Package org.apache.hadoop.hbase.util

Source Code of org.apache.hadoop.hbase.util.FSUtils$DirFilter

/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements.  See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership.  The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License.  You may obtain a copy of the License at
*
*     http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;

import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.Method;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.exceptions.FileSystemVersionException;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;

import com.google.common.primitives.Ints;
import com.google.protobuf.InvalidProtocolBufferException;

/**
* Utility methods for interacting with the underlying file system.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public abstract class FSUtils {
  private static final Log LOG = LogFactory.getLog(FSUtils.class);

  /** Full access permissions (starting point for a umask) */
  private static final String FULL_RWX_PERMISSIONS = "777";

  protected FSUtils() {
    super();
  }

  /**
   * Compare of path component. Does not consider schema; i.e. if schemas different but <code>path
   * <code> starts with <code>rootPath<code>, then the function returns true
   * @param rootPath
   * @param path
   * @return True if <code>path</code> starts with <code>rootPath</code>
   */
  public static boolean isStartingWithPath(final Path rootPath, final String path) {
    String uriRootPath = rootPath.toUri().getPath();
    String tailUriPath = (new Path(path)).toUri().getPath();
    return tailUriPath.startsWith(uriRootPath);
  }

  /**
   * Compare path component of the Path URI; e.g. if hdfs://a/b/c and /a/b/c, it will compare the
   * '/a/b/c' part. Does not consider schema; i.e. if schemas different but path or subpath matches,
   * the two will equate.
   * @param pathToSearch Path we will be trying to match.
   * @param pathTail
   * @return True if <code>pathTail</code> is tail on the path of <code>pathToSearch</code>
   */
  public static boolean isMatchingTail(final Path pathToSearch, String pathTail) {
    return isMatchingTail(pathToSearch, new Path(pathTail));
  }

  /**
   * Compare path component of the Path URI; e.g. if hdfs://a/b/c and /a/b/c, it will compare the
   * '/a/b/c' part. If you passed in 'hdfs://a/b/c and b/c, it would return true.  Does not consider
   * schema; i.e. if schemas different but path or subpath matches, the two will equate.
   * @param pathToSearch Path we will be trying to match.
   * @param pathTail
   * @return True if <code>pathTail</code> is tail on the path of <code>pathToSearch</code>
   */
  public static boolean isMatchingTail(final Path pathToSearch, final Path pathTail) {
    if (pathToSearch.depth() != pathTail.depth()) return false;
    Path tailPath = pathTail;
    String tailName;
    Path toSearch = pathToSearch;
    String toSearchName;
    boolean result = false;
    do {
      tailName = tailPath.getName();
      if (tailName == null || tailName.length() <= 0) {
        result = true;
        break;
      }
      toSearchName = toSearch.getName();
      if (toSearchName == null || toSearchName.length() <= 0) break;
      // Move up a parent on each path for next go around.  Path doesn't let us go off the end.
      tailPath = tailPath.getParent();
      toSearch = toSearch.getParent();
    } while(tailName.equals(toSearchName));
    return result;
  }

  public static FSUtils getInstance(FileSystem fs, Configuration conf) {
    String scheme = fs.getUri().getScheme();
    if (scheme == null) {
      LOG.warn("Could not find scheme for uri " +
          fs.getUri() + ", default to hdfs");
      scheme = "hdfs";
    }
    Class<?> fsUtilsClass = conf.getClass("hbase.fsutil." +
        scheme + ".impl", FSHDFSUtils.class); // Default to HDFS impl
    FSUtils fsUtils = (FSUtils)ReflectionUtils.newInstance(fsUtilsClass, conf);
    return fsUtils;
  }

  /**
   * Delete if exists.
   * @param fs filesystem object
   * @param dir directory to delete
   * @return True if deleted <code>dir</code>
   * @throws IOException e
   */
  public static boolean deleteDirectory(final FileSystem fs, final Path dir)
  throws IOException {
    return fs.exists(dir) && fs.delete(dir, true);
  }


  /**
   * Create the specified file on the filesystem. By default, this will:
   * <ol>
   * <li>overwrite the file if it exists</li>
   * <li>apply the umask in the configuration (if it is enabled)</li>
   * <li>use the fs configured buffer size (or 4096 if not set)</li>
   * <li>use the default replication</li>
   * <li>use the default block size</li>
   * <li>not track progress</li>
   * </ol>
   *
   * @param fs {@link FileSystem} on which to write the file
   * @param path {@link Path} to the file to write
   * @return output stream to the created file
   * @throws IOException if the file cannot be created
   */
  public static FSDataOutputStream create(FileSystem fs, Path path,
      FsPermission perm) throws IOException {
    return create(fs, path, perm, true);
  }

  /**
   * Create the specified file on the filesystem. By default, this will:
   * <ol>
   * <li>apply the umask in the configuration (if it is enabled)</li>
   * <li>use the fs configured buffer size (or 4096 if not set)</li>
   * <li>use the default replication</li>
   * <li>use the default block size</li>
   * <li>not track progress</li>
   * </ol>
   *
   * @param fs {@link FileSystem} on which to write the file
   * @param path {@link Path} to the file to write
   * @param perm
   * @param overwrite Whether or not the created file should be overwritten.
   * @return output stream to the created file
   * @throws IOException if the file cannot be created
   */
  public static FSDataOutputStream create(FileSystem fs, Path path,
      FsPermission perm, boolean overwrite) throws IOException {
    LOG.debug("Creating file=" + path + " with permission=" + perm);

    return fs.create(path, perm, overwrite,
        fs.getConf().getInt("io.file.buffer.size", 4096),
        fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
  }

  /**
   * Get the file permissions specified in the configuration, if they are
   * enabled.
   *
   * @param fs filesystem that the file will be created on.
   * @param conf configuration to read for determining if permissions are
   *          enabled and which to use
   * @param permssionConfKey property key in the configuration to use when
   *          finding the permission
   * @return the permission to use when creating a new file on the fs. If
   *         special permissions are not specified in the configuration, then
   *         the default permissions on the the fs will be returned.
   */
  public static FsPermission getFilePermissions(final FileSystem fs,
      final Configuration conf, final String permssionConfKey) {
    boolean enablePermissions = conf.getBoolean(
        HConstants.ENABLE_DATA_FILE_UMASK, false);

    if (enablePermissions) {
      try {
        FsPermission perm = new FsPermission(FULL_RWX_PERMISSIONS);
        // make sure that we have a mask, if not, go default.
        String mask = conf.get(permssionConfKey);
        if (mask == null)
          return FsPermission.getDefault();
        // appy the umask
        FsPermission umask = new FsPermission(mask);
        return perm.applyUMask(umask);
      } catch (IllegalArgumentException e) {
        LOG.warn(
            "Incorrect umask attempted to be created: "
                + conf.get(permssionConfKey)
                + ", using default file permissions.", e);
        return FsPermission.getDefault();
      }
    }
    return FsPermission.getDefault();
  }

  /**
   * Checks to see if the specified file system is available
   *
   * @param fs filesystem
   * @throws IOException e
   */
  public static void checkFileSystemAvailable(final FileSystem fs)
  throws IOException {
    if (!(fs instanceof DistributedFileSystem)) {
      return;
    }
    IOException exception = null;
    DistributedFileSystem dfs = (DistributedFileSystem) fs;
    try {
      if (dfs.exists(new Path("/"))) {
        return;
      }
    } catch (IOException e) {
      exception = RemoteExceptionHandler.checkIOException(e);
    }
    try {
      fs.close();
    } catch (Exception e) {
      LOG.error("file system close failed: ", e);
    }
    IOException io = new IOException("File system is not available");
    io.initCause(exception);
    throw io;
  }

  /**
   * We use reflection because {@link DistributedFileSystem#setSafeMode(
   * FSConstants.SafeModeAction action, boolean isChecked)} is not in hadoop 1.1
   *
   * @param dfs
   * @return whether we're in safe mode
   * @throws IOException
   */
  private static boolean isInSafeMode(DistributedFileSystem dfs) throws IOException {
    boolean inSafeMode = false;
    try {
      Method m = DistributedFileSystem.class.getMethod("setSafeMode", new Class<?> []{
          org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction.class, boolean.class});
      inSafeMode = (Boolean) m.invoke(dfs,
        org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction.SAFEMODE_GET, true);
    } catch (Exception e) {
      if (e instanceof IOException) throw (IOException) e;
     
      // Check whether dfs is on safemode.
      inSafeMode = dfs.setSafeMode(
        org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction.SAFEMODE_GET);     
    }
    return inSafeMode;   
  }
 
  /**
   * Check whether dfs is in safemode.
   * @param conf
   * @throws IOException
   */
  public static void checkDfsSafeMode(final Configuration conf)
  throws IOException {
    boolean isInSafeMode = false;
    FileSystem fs = FileSystem.get(conf);
    if (fs instanceof DistributedFileSystem) {
      DistributedFileSystem dfs = (DistributedFileSystem)fs;
      isInSafeMode = isInSafeMode(dfs);
    }
    if (isInSafeMode) {
      throw new IOException("File system is in safemode, it can't be written now");
    }
  }

  /**
   * Verifies current version of file system
   *
   * @param fs filesystem object
   * @param rootdir root hbase directory
   * @return null if no version file exists, version string otherwise.
   * @throws IOException e
   * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
   */
  public static String getVersion(FileSystem fs, Path rootdir)
  throws IOException, DeserializationException {
    Path versionFile = new Path(rootdir, HConstants.VERSION_FILE_NAME);
    FileStatus[] status = null;
    try {
      // hadoop 2.0 throws FNFE if directory does not exist. 
      // hadoop 1.0 returns null if directory does not exist.
      status = fs.listStatus(versionFile);
    } catch (FileNotFoundException fnfe) {
      return null;
    }
    if (status == null || status.length == 0) return null;
    String version = null;
    byte [] content = new byte [(int)status[0].getLen()];
    FSDataInputStream s = fs.open(versionFile);
    try {
      IOUtils.readFully(s, content, 0, content.length);
      if (ProtobufUtil.isPBMagicPrefix(content)) {
        version = parseVersionFrom(content);
      } else {
        // Presume it pre-pb format.
        InputStream is = new ByteArrayInputStream(content);
        DataInputStream dis = new DataInputStream(is);
        try {
          version = dis.readUTF();
        } finally {
          dis.close();
        }
        // Update the format
        LOG.info("Updating the hbase.version file format with version=" + version);
        setVersion(fs, rootdir, version, 0, HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS);
      }
    } catch (EOFException eof) {
      LOG.warn("Version file was empty, odd, will try to set it.");
    } finally {
      s.close();
    }
    return version;
  }

  /**
   * Parse the content of the ${HBASE_ROOTDIR}/hbase.version file.
   * @param bytes The byte content of the hbase.version file.
   * @return The version found in the file as a String.
   * @throws DeserializationException
   */
  static String parseVersionFrom(final byte [] bytes)
  throws DeserializationException {
    ProtobufUtil.expectPBMagicPrefix(bytes);
    int pblen = ProtobufUtil.lengthOfPBMagic();
    FSProtos.HBaseVersionFileContent.Builder builder =
      FSProtos.HBaseVersionFileContent.newBuilder();
    FSProtos.HBaseVersionFileContent fileContent;
    try {
      fileContent = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
      return fileContent.getVersion();
    } catch (InvalidProtocolBufferException e) {
      // Convert
      throw new DeserializationException(e);
    }
  }

  /**
   * Create the content to write into the ${HBASE_ROOTDIR}/hbase.version file.
   * @param version Version to persist
   * @return Serialized protobuf with <code>version</code> content and a bit of pb magic for a prefix.
   */
  static byte [] toVersionByteArray(final String version) {
    FSProtos.HBaseVersionFileContent.Builder builder =
      FSProtos.HBaseVersionFileContent.newBuilder();
    return ProtobufUtil.prependPBMagic(builder.setVersion(version).build().toByteArray());
  }

  /**
   * Verifies current version of file system
   *
   * @param fs file system
   * @param rootdir root directory of HBase installation
   * @param message if true, issues a message on System.out
   *
   * @throws IOException e
   * @throws DeserializationException
   */
  public static void checkVersion(FileSystem fs, Path rootdir, boolean message)
  throws IOException, DeserializationException {
    checkVersion(fs, rootdir, message, 0, HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS);
  }

  /**
   * Verifies current version of file system
   *
   * @param fs file system
   * @param rootdir root directory of HBase installation
   * @param message if true, issues a message on System.out
   * @param wait wait interval
   * @param retries number of times to retry
   *
   * @throws IOException e
   * @throws DeserializationException
   */
  public static void checkVersion(FileSystem fs, Path rootdir,
      boolean message, int wait, int retries)
  throws IOException, DeserializationException {
    String version = getVersion(fs, rootdir);
    if (version == null) {
      if (!metaRegionExists(fs, rootdir)) {
        // rootDir is empty (no version file and no root region)
        // just create new version file (HBASE-1195)
        setVersion(fs, rootdir, wait, retries);
        return;
      }
    } else if (version.compareTo(HConstants.FILE_SYSTEM_VERSION) == 0) return;

    // version is deprecated require migration
    // Output on stdout so user sees it in terminal.
    String msg = "File system needs to be upgraded."
      + "  You have version " + version
      + " and I want version " + HConstants.FILE_SYSTEM_VERSION
      + ".  Run the '${HBASE_HOME}/bin/hbase migrate' script.";
    if (message) {
      System.out.println("WARNING! " + msg);
    }
    throw new FileSystemVersionException(msg);
  }

  /**
   * Sets version of file system
   *
   * @param fs filesystem object
   * @param rootdir hbase root
   * @throws IOException e
   */
  public static void setVersion(FileSystem fs, Path rootdir)
  throws IOException {
    setVersion(fs, rootdir, HConstants.FILE_SYSTEM_VERSION, 0,
      HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS);
  }

  /**
   * Sets version of file system
   *
   * @param fs filesystem object
   * @param rootdir hbase root
   * @param wait time to wait for retry
   * @param retries number of times to retry before failing
   * @throws IOException e
   */
  public static void setVersion(FileSystem fs, Path rootdir, int wait, int retries)
  throws IOException {
    setVersion(fs, rootdir, HConstants.FILE_SYSTEM_VERSION, wait, retries);
  }


  /**
   * Sets version of file system
   *
   * @param fs filesystem object
   * @param rootdir hbase root directory
   * @param version version to set
   * @param wait time to wait for retry
   * @param retries number of times to retry before throwing an IOException
   * @throws IOException e
   */
  public static void setVersion(FileSystem fs, Path rootdir, String version,
      int wait, int retries) throws IOException {
    Path versionFile = new Path(rootdir, HConstants.VERSION_FILE_NAME);
    while (true) {
      try {
        FSDataOutputStream s = fs.create(versionFile);
        s.write(toVersionByteArray(version));
        s.close();
        LOG.debug("Created version file at " + rootdir.toString() + " with version=" + version);
        return;
      } catch (IOException e) {
        if (retries > 0) {
          LOG.warn("Unable to create version file at " + rootdir.toString() + ", retrying", e);
          fs.delete(versionFile, false);
          try {
            if (wait > 0) {
              Thread.sleep(wait);
            }
          } catch (InterruptedException ex) {
            // ignore
          }
          retries--;
        } else {
          throw e;
        }
      }
    }
  }

  /**
   * Checks that a cluster ID file exists in the HBase root directory
   * @param fs the root directory FileSystem
   * @param rootdir the HBase root directory in HDFS
   * @param wait how long to wait between retries
   * @return <code>true</code> if the file exists, otherwise <code>false</code>
   * @throws IOException if checking the FileSystem fails
   */
  public static boolean checkClusterIdExists(FileSystem fs, Path rootdir,
      int wait) throws IOException {
    while (true) {
      try {
        Path filePath = new Path(rootdir, HConstants.CLUSTER_ID_FILE_NAME);
        return fs.exists(filePath);
      } catch (IOException ioe) {
        if (wait > 0) {
          LOG.warn("Unable to check cluster ID file in " + rootdir.toString() +
              ", retrying in "+wait+"msec: "+StringUtils.stringifyException(ioe));
          try {
            Thread.sleep(wait);
          } catch (InterruptedException ie) {
            Thread.interrupted();
            break;
          }
        } else {
          throw ioe;
        }
      }
    }
    return false;
  }

  /**
   * Returns the value of the unique cluster ID stored for this HBase instance.
   * @param fs the root directory FileSystem
   * @param rootdir the path to the HBase root directory
   * @return the unique cluster identifier
   * @throws IOException if reading the cluster ID file fails
   */
  public static ClusterId getClusterId(FileSystem fs, Path rootdir)
  throws IOException {
    Path idPath = new Path(rootdir, HConstants.CLUSTER_ID_FILE_NAME);
    ClusterId clusterId = null;
    FileStatus status = fs.exists(idPath)? fs.getFileStatus(idPath)null;
    if (status != null) {
      int len = Ints.checkedCast(status.getLen());
      byte [] content = new byte[len];
      FSDataInputStream in = fs.open(idPath);
      try {
        in.readFully(content);
      } catch (EOFException eof) {
        LOG.warn("Cluster ID file " + idPath.toString() + " was empty");
      } finally{
        in.close();
      }
      try {
        clusterId = ClusterId.parseFrom(content);
      } catch (DeserializationException e) {
        throw new IOException("content=" + Bytes.toString(content), e);
      }
      // If not pb'd, make it so.
      if (!ProtobufUtil.isPBMagicPrefix(content)) rewriteAsPb(fs, rootdir, idPath, clusterId);
      return clusterId;
    } else {
      LOG.warn("Cluster ID file does not exist at " + idPath.toString());
    }
    return clusterId;
  }

  /**
   * @param cid
   * @throws IOException
   */
  private static void rewriteAsPb(final FileSystem fs, final Path rootdir, final Path p,
      final ClusterId cid)
  throws IOException {
    // Rewrite the file as pb.  Move aside the old one first, write new
    // then delete the moved-aside file.
    Path movedAsideName = new Path(p + "." + System.currentTimeMillis());
    if (!fs.rename(p, movedAsideName)) throw new IOException("Failed rename of " + p);
    setClusterId(fs, rootdir, cid, 100);
    if (!fs.delete(movedAsideName, false)) {
      throw new IOException("Failed delete of " + movedAsideName);
    }
    LOG.debug("Rewrote the hbase.id file as pb");
  }

  /**
   * Writes a new unique identifier for this cluster to the "hbase.id" file
   * in the HBase root directory
   * @param fs the root directory FileSystem
   * @param rootdir the path to the HBase root directory
   * @param clusterId the unique identifier to store
   * @param wait how long (in milliseconds) to wait between retries
   * @throws IOException if writing to the FileSystem fails and no wait value
   */
  public static void setClusterId(FileSystem fs, Path rootdir, ClusterId clusterId,
      int wait) throws IOException {
    while (true) {
      try {
        Path filePath = new Path(rootdir, HConstants.CLUSTER_ID_FILE_NAME);
        FSDataOutputStream s = fs.create(filePath);
        try {
          s.write(clusterId.toByteArray());
        } finally {
          s.close();
        }
        if (LOG.isDebugEnabled()) {
          LOG.debug("Created cluster ID file at " + filePath.toString() + " with ID: " + clusterId);
        }
        return;
      } catch (IOException ioe) {
        if (wait > 0) {
          LOG.warn("Unable to create cluster ID file in " + rootdir.toString() +
              ", retrying in " + wait + "msec: " + StringUtils.stringifyException(ioe));
          try {
            Thread.sleep(wait);
          } catch (InterruptedException ie) {
            Thread.interrupted();
            break;
          }
        } else {
          throw ioe;
        }
      }
    }
  }

  /**
   * Verifies root directory path is a valid URI with a scheme
   *
   * @param root root directory path
   * @return Passed <code>root</code> argument.
   * @throws IOException if not a valid URI with a scheme
   */
  public static Path validateRootPath(Path root) throws IOException {
    try {
      URI rootURI = new URI(root.toString());
      String scheme = rootURI.getScheme();
      if (scheme == null) {
        throw new IOException("Root directory does not have a scheme");
      }
      return root;
    } catch (URISyntaxException e) {
      IOException io = new IOException("Root directory path is not a valid " +
        "URI -- check your " + HConstants.HBASE_DIR + " configuration");
      io.initCause(e);
      throw io;
    }
  }

  /**
   * Checks for the presence of the root path (using the provided conf object) in the given path. If
   * it exists, this method removes it and returns the String representation of remaining relative path.
   * @param path
   * @param conf
   * @return String representation of the remaining relative path
   * @throws IOException
   */
  public static String removeRootPath(Path path, final Configuration conf) throws IOException {
    Path root = FSUtils.getRootDir(conf);
    String pathStr = path.toString();
    // check that the path is absolute... it has the root path in it.
    if (!pathStr.startsWith(root.toString())) return pathStr;
    // if not, return as it is.
    return pathStr.substring(root.toString().length() + 1);// remove the "/" too.
  }

  /**
   * If DFS, check safe mode and if so, wait until we clear it.
   * @param conf configuration
   * @param wait Sleep between retries
   * @throws IOException e
   */
  public static void waitOnSafeMode(final Configuration conf,
    final long wait)
  throws IOException {
    FileSystem fs = FileSystem.get(conf);
    if (!(fs instanceof DistributedFileSystem)) return;
    DistributedFileSystem dfs = (DistributedFileSystem)fs;
    // Make sure dfs is not in safe mode
    while (isInSafeMode(dfs)) {
      LOG.info("Waiting for dfs to exit safe mode...");
      try {
        Thread.sleep(wait);
      } catch (InterruptedException e) {
        //continue
      }
    }
  }

  /**
   * Return the 'path' component of a Path.  In Hadoop, Path is an URI.  This
   * method returns the 'path' component of a Path's URI: e.g. If a Path is
   * <code>hdfs://example.org:9000/hbase_trunk/TestTable/compaction.dir</code>,
   * this method returns <code>/hbase_trunk/TestTable/compaction.dir</code>.
   * This method is useful if you want to print out a Path without qualifying
   * Filesystem instance.
   * @param p Filesystem Path whose 'path' component we are to return.
   * @return Path portion of the Filesystem
   */
  public static String getPath(Path p) {
    return p.toUri().getPath();
  }

  /**
   * @param c configuration
   * @return Path to hbase root directory: i.e. <code>hbase.rootdir</code> from
   * configuration as a qualified Path.
   * @throws IOException e
   */
  public static Path getRootDir(final Configuration c) throws IOException {
    Path p = new Path(c.get(HConstants.HBASE_DIR));
    FileSystem fs = p.getFileSystem(c);
    return p.makeQualified(fs);
  }

  public static void setRootDir(final Configuration c, final Path root) throws IOException {
    c.set(HConstants.HBASE_DIR, root.toString());
  }

  public static void setFsDefault(final Configuration c, final Path root) throws IOException {
    c.set("fs.defaultFS", root.toString());    // for hadoop 0.21+
    c.set("fs.default.name", root.toString()); // for hadoop 0.20
  }

  /**
   * Checks if root region exists
   *
   * @param fs file system
   * @param rootdir root directory of HBase installation
   * @return true if exists
   * @throws IOException e
   */
  public static boolean metaRegionExists(FileSystem fs, Path rootdir)
  throws IOException {
    Path rootRegionDir =
      HRegion.getRegionDir(rootdir, HRegionInfo.FIRST_META_REGIONINFO);
    return fs.exists(rootRegionDir);
  }

  /**
   * Compute HDFS blocks distribution of a given file, or a portion of the file
   * @param fs file system
   * @param status file status of the file
   * @param start start position of the portion
   * @param length length of the portion
   * @return The HDFS blocks distribution
   */
  static public HDFSBlocksDistribution computeHDFSBlocksDistribution(
    final FileSystem fs, FileStatus status, long start, long length)
    throws IOException {
    HDFSBlocksDistribution blocksDistribution = new HDFSBlocksDistribution();
    BlockLocation [] blockLocations =
      fs.getFileBlockLocations(status, start, length);
    for(BlockLocation bl : blockLocations) {
      String [] hosts = bl.getHosts();
      long len = bl.getLength();
      blocksDistribution.addHostsAndBlockWeight(hosts, len);
    }

    return blocksDistribution;
  }



  /**
   * Runs through the hbase rootdir and checks all stores have only
   * one file in them -- that is, they've been major compacted.  Looks
   * at root and meta tables too.
   * @param fs filesystem
   * @param hbaseRootDir hbase root directory
   * @return True if this hbase install is major compacted.
   * @throws IOException e
   */
  public static boolean isMajorCompacted(final FileSystem fs,
      final Path hbaseRootDir)
  throws IOException {
    // Presumes any directory under hbase.rootdir is a table.
    FileStatus [] tableDirs = fs.listStatus(hbaseRootDir, new DirFilter(fs));
    for (FileStatus tableDir : tableDirs) {
      // Skip the .log directory.  All others should be tables.  Inside a table,
      // there are compaction.dir directories to skip.  Otherwise, all else
      // should be regions.  Then in each region, should only be family
      // directories.  Under each of these, should be one file only.
      Path d = tableDir.getPath();
      if (d.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
        continue;
      }
      FileStatus[] regionDirs = fs.listStatus(d, new DirFilter(fs));
      for (FileStatus regionDir : regionDirs) {
        Path dd = regionDir.getPath();
        if (dd.getName().equals(HConstants.HREGION_COMPACTIONDIR_NAME)) {
          continue;
        }
        // Else its a region name.  Now look in region for families.
        FileStatus[] familyDirs = fs.listStatus(dd, new DirFilter(fs));
        for (FileStatus familyDir : familyDirs) {
          Path family = familyDir.getPath();
          // Now in family make sure only one file.
          FileStatus[] familyStatus = fs.listStatus(family);
          if (familyStatus.length > 1) {
            LOG.debug(family.toString() + " has " + familyStatus.length +
                " files.");
            return false;
          }
        }
      }
    }
    return true;
  }

  // TODO move this method OUT of FSUtils. No dependencies to HMaster
  /**
   * Returns the total overall fragmentation percentage. Includes .META. and
   * -ROOT- as well.
   *
   * @param master  The master defining the HBase root and file system.
   * @return A map for each table and its percentage.
   * @throws IOException When scanning the directory fails.
   */
  public static int getTotalTableFragmentation(final HMaster master)
  throws IOException {
    Map<String, Integer> map = getTableFragmentation(master);
    return map != null && map.size() > 0 ? map.get("-TOTAL-") : -1;
  }

  /**
   * Runs through the HBase rootdir and checks how many stores for each table
   * have more than one file in them. Checks -ROOT- and .META. too. The total
   * percentage across all tables is stored under the special key "-TOTAL-".
   *
   * @param master  The master defining the HBase root and file system.
   * @return A map for each table and its percentage.
   *
   * @throws IOException When scanning the directory fails.
   */
  public static Map<String, Integer> getTableFragmentation(
    final HMaster master)
  throws IOException {
    Path path = getRootDir(master.getConfiguration());
    // since HMaster.getFileSystem() is package private
    FileSystem fs = path.getFileSystem(master.getConfiguration());
    return getTableFragmentation(fs, path);
  }

  /**
   * Runs through the HBase rootdir and checks how many stores for each table
   * have more than one file in them. Checks -ROOT- and .META. too. The total
   * percentage across all tables is stored under the special key "-TOTAL-".
   *
   * @param fs  The file system to use.
   * @param hbaseRootDir  The root directory to scan.
   * @return A map for each table and its percentage.
   * @throws IOException When scanning the directory fails.
   */
  public static Map<String, Integer> getTableFragmentation(
    final FileSystem fs, final Path hbaseRootDir)
  throws IOException {
    Map<String, Integer> frags = new HashMap<String, Integer>();
    int cfCountTotal = 0;
    int cfFragTotal = 0;
    DirFilter df = new DirFilter(fs);
    // presumes any directory under hbase.rootdir is a table
    FileStatus [] tableDirs = fs.listStatus(hbaseRootDir, df);
    for (FileStatus tableDir : tableDirs) {
      // Skip the .log directory.  All others should be tables.  Inside a table,
      // there are compaction.dir directories to skip.  Otherwise, all else
      // should be regions.  Then in each region, should only be family
      // directories.  Under each of these, should be one file only.
      Path d = tableDir.getPath();
      if (d.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
        continue;
      }
      int cfCount = 0;
      int cfFrag = 0;
      FileStatus[] regionDirs = fs.listStatus(d, df);
      for (FileStatus regionDir : regionDirs) {
        Path dd = regionDir.getPath();
        if (dd.getName().equals(HConstants.HREGION_COMPACTIONDIR_NAME)) {
          continue;
        }
        // else its a region name, now look in region for families
        FileStatus[] familyDirs = fs.listStatus(dd, df);
        for (FileStatus familyDir : familyDirs) {
          cfCount++;
          cfCountTotal++;
          Path family = familyDir.getPath();
          // now in family make sure only one file
          FileStatus[] familyStatus = fs.listStatus(family);
          if (familyStatus.length > 1) {
            cfFrag++;
            cfFragTotal++;
          }
        }
      }
      // compute percentage per table and store in result list
      frags.put(d.getName(), Math.round((float) cfFrag / cfCount * 100));
    }
    // set overall percentage for all tables
    frags.put("-TOTAL-", Math.round((float) cfFragTotal / cfCountTotal * 100));
    return frags;
  }

  /**
   * Expects to find -ROOT- directory.
   * @param fs filesystem
   * @param hbaseRootDir hbase root directory
   * @return True if this a pre020 layout.
   * @throws IOException e
   */
  public static boolean isPre020FileLayout(final FileSystem fs,
    final Path hbaseRootDir)
  throws IOException {
    Path mapfiles = new Path(new Path(new Path(new Path(hbaseRootDir, "-ROOT-"),
      "70236052"), "info"), "mapfiles");
    return fs.exists(mapfiles);
  }

  /**
   * Runs through the hbase rootdir and checks all stores have only
   * one file in them -- that is, they've been major compacted.  Looks
   * at root and meta tables too.  This version differs from
   * {@link #isMajorCompacted(FileSystem, Path)} in that it expects a
   * pre-0.20.0 hbase layout on the filesystem.  Used migrating.
   * @param fs filesystem
   * @param hbaseRootDir hbase root directory
   * @return True if this hbase install is major compacted.
   * @throws IOException e
   */
  public static boolean isMajorCompactedPre020(final FileSystem fs,
      final Path hbaseRootDir)
  throws IOException {
    // Presumes any directory under hbase.rootdir is a table.
    FileStatus [] tableDirs = fs.listStatus(hbaseRootDir, new DirFilter(fs));
    for (FileStatus tableDir : tableDirs) {
      // Inside a table, there are compaction.dir directories to skip.
      // Otherwise, all else should be regions.  Then in each region, should
      // only be family directories.  Under each of these, should be a mapfile
      // and info directory and in these only one file.
      Path d = tableDir.getPath();
      if (d.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
        continue;
      }
      FileStatus[] regionDirs = fs.listStatus(d, new DirFilter(fs));
      for (FileStatus regionDir : regionDirs) {
        Path dd = regionDir.getPath();
        if (dd.getName().equals(HConstants.HREGION_COMPACTIONDIR_NAME)) {
          continue;
        }
        // Else its a region name.  Now look in region for families.
        FileStatus[] familyDirs = fs.listStatus(dd, new DirFilter(fs));
        for (FileStatus familyDir : familyDirs) {
          Path family = familyDir.getPath();
          FileStatus[] infoAndMapfile = fs.listStatus(family);
          // Assert that only info and mapfile in family dir.
          if (infoAndMapfile.length != 0 && infoAndMapfile.length != 2) {
            LOG.debug(family.toString() +
                " has more than just info and mapfile: " + infoAndMapfile.length);
            return false;
          }
          // Make sure directory named info or mapfile.
          for (int ll = 0; ll < 2; ll++) {
            if (infoAndMapfile[ll].getPath().getName().equals("info") ||
                infoAndMapfile[ll].getPath().getName().equals("mapfiles"))
              continue;
            LOG.debug("Unexpected directory name: " +
                infoAndMapfile[ll].getPath());
            return false;
          }
          // Now in family, there are 'mapfile' and 'info' subdirs.  Just
          // look in the 'mapfile' subdir.
          FileStatus[] familyStatus =
              fs.listStatus(new Path(family, "mapfiles"));
          if (familyStatus.length > 1) {
            LOG.debug(family.toString() + " has " + familyStatus.length +
                " files.");
            return false;
          }
        }
      }
    }
    return true;
  }

  /**
   * A {@link PathFilter} that returns only regular files.
   */
  static class FileFilter implements PathFilter {
    private final FileSystem fs;

    public FileFilter(final FileSystem fs) {
      this.fs = fs;
    }

    @Override
    public boolean accept(Path p) {
      try {
        return fs.isFile(p);
      } catch (IOException e) {
        LOG.debug("unable to verify if path=" + p + " is a regular file", e);
        return false;
      }
    }
  }

  /**
   * A {@link PathFilter} that returns directories.
   */
  public static class DirFilter implements PathFilter {
    private final FileSystem fs;

    public DirFilter(final FileSystem fs) {
      this.fs = fs;
    }

    @Override
    public boolean accept(Path p) {
      boolean isValid = false;
      try {
        if (HConstants.HBASE_NON_USER_TABLE_DIRS.contains(p.toString())) {
          isValid = false;
        } else {
          isValid = fs.getFileStatus(p).isDir();
        }
      } catch (IOException e) {
        LOG.warn("An error occurred while verifying if [" + p.toString() +
                 "] is a valid directory. Returning 'not valid' and continuing.", e);
      }
      return isValid;
    }
  }

  /**
   * Heuristic to determine whether is safe or not to open a file for append
   * Looks both for dfs.support.append and use reflection to search
   * for SequenceFile.Writer.syncFs() or FSDataOutputStream.hflush()
   * @param conf
   * @return True if append support
   */
  public static boolean isAppendSupported(final Configuration conf) {
    boolean append = conf.getBoolean("dfs.support.append", false);
    if (append) {
      try {
        // TODO: The implementation that comes back when we do a createWriter
        // may not be using SequenceFile so the below is not a definitive test.
        // Will do for now (hdfs-200).
        SequenceFile.Writer.class.getMethod("syncFs", new Class<?> []{});
        append = true;
      } catch (SecurityException e) {
      } catch (NoSuchMethodException e) {
        append = false;
      }
    }
    if (!append) {
      // Look for the 0.21, 0.22, new-style append evidence.
      try {
        FSDataOutputStream.class.getMethod("hflush", new Class<?> []{});
        append = true;
      } catch (NoSuchMethodException e) {
        append = false;
      }
    }
    return append;
  }

  /**
   * @param conf
   * @return True if this filesystem whose scheme is 'hdfs'.
   * @throws IOException
   */
  public static boolean isHDFS(final Configuration conf) throws IOException {
    FileSystem fs = FileSystem.get(conf);
    String scheme = fs.getUri().getScheme();
    return scheme.equalsIgnoreCase("hdfs");
  }

  /**
   * Recover file lease. Used when a file might be suspect
   * to be had been left open by another process.
   * @param fs FileSystem handle
   * @param p Path of file to recover lease
   * @param conf Configuration handle
   * @throws IOException
   */
  public abstract void recoverFileLease(final FileSystem fs, final Path p,
      Configuration conf) throws IOException;

  /**
   * @param fs
   * @param rootdir
   * @return All the table directories under <code>rootdir</code>. Ignore non table hbase folders such as
   * .logs, .oldlogs, .corrupt, .META., and -ROOT- folders.
   * @throws IOException
   */
  public static List<Path> getTableDirs(final FileSystem fs, final Path rootdir)
  throws IOException {
    // presumes any directory under hbase.rootdir is a table
    FileStatus [] dirs = fs.listStatus(rootdir, new DirFilter(fs));
    List<Path> tabledirs = new ArrayList<Path>(dirs.length);
    for (FileStatus dir: dirs) {
      Path p = dir.getPath();
      String tableName = p.getName();
      if (!HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName)) {
        tabledirs.add(p);
      }
    }
    return tabledirs;
  }

  public static Path getTablePath(Path rootdir, byte [] tableName) {
    return getTablePath(rootdir, Bytes.toString(tableName));
  }

  public static Path getTablePath(Path rootdir, final String tableName) {
    return new Path(rootdir, tableName);
  }

  /**
   * Filter for all dirs that don't start with '.'
   */
  public static class RegionDirFilter implements PathFilter {
    // This pattern will accept 0.90+ style hex region dirs and older numeric region dir names.
    final public static Pattern regionDirPattern = Pattern.compile("^[0-9a-f]*$");
    final FileSystem fs;

    public RegionDirFilter(FileSystem fs) {
      this.fs = fs;
    }

    @Override
    public boolean accept(Path rd) {
      if (!regionDirPattern.matcher(rd.getName()).matches()) {
        return false;
      }

      try {
        return fs.getFileStatus(rd).isDir();
      } catch (IOException ioe) {
        // Maybe the file was moved or the fs was disconnected.
        LOG.warn("Skipping file " + rd +" due to IOException", ioe);
        return false;
      }
    }
  }

  /**
   * Given a particular table dir, return all the regiondirs inside it, excluding files such as
   * .tableinfo
   * @param fs A file system for the Path
   * @param tableDir Path to a specific table directory <hbase.rootdir>/<tabledir>
   * @return List of paths to valid region directories in table dir.
   * @throws IOException
   */
  public static List<Path> getRegionDirs(final FileSystem fs, final Path tableDir) throws IOException {
    // assumes we are in a table dir.
    FileStatus[] rds = fs.listStatus(tableDir, new RegionDirFilter(fs));
    List<Path> regionDirs = new ArrayList<Path>(rds.length);
    for (FileStatus rdfs: rds) {
      Path rdPath = rdfs.getPath();
      regionDirs.add(rdPath);
    }
    return regionDirs;
  }

  /**
   * Filter for all dirs that are legal column family names.  This is generally used for colfam
   * dirs <hbase.rootdir>/<tabledir>/<regiondir>/<colfamdir>.
   */
  public static class FamilyDirFilter implements PathFilter {
    final FileSystem fs;

    public FamilyDirFilter(FileSystem fs) {
      this.fs = fs;
    }

    @Override
    public boolean accept(Path rd) {
      try {
        // throws IAE if invalid
        HColumnDescriptor.isLegalFamilyName(Bytes.toBytes(rd.getName()));
      } catch (IllegalArgumentException iae) {
        // path name is an invalid family name and thus is excluded.
        return false;
      }

      try {
        return fs.getFileStatus(rd).isDir();
      } catch (IOException ioe) {
        // Maybe the file was moved or the fs was disconnected.
        LOG.warn("Skipping file " + rd +" due to IOException", ioe);
        return false;
      }
    }
  }

  /**
   * Given a particular region dir, return all the familydirs inside it
   *
   * @param fs A file system for the Path
   * @param regionDir Path to a specific region directory
   * @return List of paths to valid family directories in region dir.
   * @throws IOException
   */
  public static List<Path> getFamilyDirs(final FileSystem fs, final Path regionDir) throws IOException {
    // assumes we are in a region dir.
    FileStatus[] fds = fs.listStatus(regionDir, new FamilyDirFilter(fs));
    List<Path> familyDirs = new ArrayList<Path>(fds.length);
    for (FileStatus fdfs: fds) {
      Path fdPath = fdfs.getPath();
      familyDirs.add(fdPath);
    }
    return familyDirs;
  }

  /**
   * Filter for HFiles that excludes reference files.
   */
  public static class HFileFilter implements PathFilter {
    // This pattern will accept 0.90+ style hex hfies files but reject reference files
    final public static Pattern hfilePattern = Pattern.compile("^([0-9a-f]+)$");

    final FileSystem fs;

    public HFileFilter(FileSystem fs) {
      this.fs = fs;
    }

    @Override
    public boolean accept(Path rd) {
      if (!hfilePattern.matcher(rd.getName()).matches()) {
        return false;
      }

      try {
        // only files
        return !fs.getFileStatus(rd).isDir();
      } catch (IOException ioe) {
        // Maybe the file was moved or the fs was disconnected.
        LOG.warn("Skipping file " + rd +" due to IOException", ioe);
        return false;
      }
    }
  }

  /**
   * @param conf
   * @return Returns the filesystem of the hbase rootdir.
   * @throws IOException
   */
  public static FileSystem getCurrentFileSystem(Configuration conf)
  throws IOException {
    return getRootDir(conf).getFileSystem(conf);
  }

  /**
   * Runs through the HBase rootdir and creates a reverse lookup map for
   * table StoreFile names to the full Path.
   * <br>
   * Example...<br>
   * Key = 3944417774205889744  <br>
   * Value = hdfs://localhost:51169/user/userid/-ROOT-/70236052/info/3944417774205889744
   *
   * @param fs  The file system to use.
   * @param hbaseRootDir  The root directory to scan.
   * @return Map keyed by StoreFile name with a value of the full Path.
   * @throws IOException When scanning the directory fails.
   */
  public static Map<String, Path> getTableStoreFilePathMap(
    final FileSystem fs, final Path hbaseRootDir)
  throws IOException {
    Map<String, Path> map = new HashMap<String, Path>();

    // if this method looks similar to 'getTableFragmentation' that is because
    // it was borrowed from it.
   
    DirFilter df = new DirFilter(fs);
    // presumes any directory under hbase.rootdir is a table
    FileStatus [] tableDirs = fs.listStatus(hbaseRootDir, df);
    for (FileStatus tableDir : tableDirs) {
      // Skip the .log and other non-table directories.  All others should be tables.
      // Inside a table, there are compaction.dir directories to skip.  Otherwise, all else
      // should be regions.
      Path d = tableDir.getPath();
      if (HConstants.HBASE_NON_TABLE_DIRS.contains(d.getName())) {
        continue;
      }
      FileStatus[] regionDirs = fs.listStatus(d, df);
      for (FileStatus regionDir : regionDirs) {
        Path dd = regionDir.getPath();
        if (dd.getName().equals(HConstants.HREGION_COMPACTIONDIR_NAME)) {
          continue;
        }
        // else its a region name, now look in region for families
        FileStatus[] familyDirs = fs.listStatus(dd, df);
        for (FileStatus familyDir : familyDirs) {
          Path family = familyDir.getPath();
          // now in family, iterate over the StoreFiles and
          // put in map
          FileStatus[] familyStatus = fs.listStatus(family);
          for (FileStatus sfStatus : familyStatus) {
            Path sf = sfStatus.getPath();
            map.put( sf.getName(), sf);
          }

        }
      }
    }
      return map;
  }

  /**
   * Calls fs.listStatus() and treats FileNotFoundException as non-fatal
   * This accommodates differences between hadoop versions
   *
   * @param fs file system
   * @param dir directory
   * @param filter path filter
   * @return null if tabledir doesn't exist, otherwise FileStatus array
   */
  public static FileStatus [] listStatus(final FileSystem fs,
      final Path dir, final PathFilter filter) throws IOException {
    FileStatus [] status = null;
    try {
      status = filter == null ? fs.listStatus(dir) : fs.listStatus(dir, filter);
    } catch (FileNotFoundException fnfe) {
      // if directory doesn't exist, return null
      LOG.debug(dir + " doesn't exist");
    }
    if (status == null || status.length < 1) return null;
    return status;
  }

  /**
   * Calls fs.listStatus() and treats FileNotFoundException as non-fatal
   * This would accommodates differences between hadoop versions
   *
   * @param fs file system
   * @param dir directory
   * @return null if tabledir doesn't exist, otherwise FileStatus array
   */
  public static FileStatus[] listStatus(final FileSystem fs, final Path dir) throws IOException {
    return listStatus(fs, dir, null);
  }

  /**
   * Calls fs.delete() and returns the value returned by the fs.delete()
   *
   * @param fs
   * @param path
   * @param recursive
   * @return the value returned by the fs.delete()
   * @throws IOException
   */
  public static boolean delete(final FileSystem fs, final Path path, final boolean recursive)
      throws IOException {
    return fs.delete(path, recursive);
  }

  /**
   * Calls fs.exists(). Checks if the specified path exists
   *
   * @param fs
   * @param path
   * @return the value returned by fs.exists()
   * @throws IOException
   */
  public static boolean isExists(final FileSystem fs, final Path path) throws IOException {
    return fs.exists(path);
  }

  /**
   * Throw an exception if an action is not permitted by a user on a file.
   *
   * @param ugi
   *          the user
   * @param file
   *          the file
   * @param action
   *          the action
   */
  public static void checkAccess(UserGroupInformation ugi, FileStatus file,
      FsAction action) throws AccessControlException {
    if (ugi.getShortUserName().equals(file.getOwner())) {
      if (file.getPermission().getUserAction().implies(action)) {
        return;
      }
    } else if (contains(ugi.getGroupNames(), file.getGroup())) {
      if (file.getPermission().getGroupAction().implies(action)) {
        return;
      }
    } else if (file.getPermission().getOtherAction().implies(action)) {
      return;
    }
    throw new AccessControlException("Permission denied:" + " action=" + action
        + " path=" + file.getPath() + " user=" + ugi.getShortUserName());
  }

  private static boolean contains(String[] groups, String user) {
    for (String group : groups) {
      if (group.equals(user)) {
        return true;
      }
    }
    return false;
  }

  /**
   * Log the current state of the filesystem from a certain root directory
   * @param fs filesystem to investigate
   * @param root root file/directory to start logging from
   * @param LOG log to output information
   * @throws IOException if an unexpected exception occurs
   */
  public static void logFileSystemState(final FileSystem fs, final Path root, Log LOG)
      throws IOException {
    LOG.debug("Current file system:");
    logFSTree(LOG, fs, root, "|-");
  }

  /**
   * Recursive helper to log the state of the FS
   *
   * @see #logFileSystemState(FileSystem, Path, Log)
   */
  private static void logFSTree(Log LOG, final FileSystem fs, final Path root, String prefix)
      throws IOException {
    FileStatus[] files = FSUtils.listStatus(fs, root, null);
    if (files == null) return;

    for (FileStatus file : files) {
      if (file.isDir()) {
        LOG.debug(prefix + file.getPath().getName() + "/");
        logFSTree(LOG, fs, file.getPath(), prefix + "---");
      } else {
        LOG.debug(prefix + file.getPath().getName());
      }
    }
  }
}
TOP

Related Classes of org.apache.hadoop.hbase.util.FSUtils$DirFilter

TOP
Copyright © 2018 www.massapi.com. All rights reserved.
All source code are property of their respective owners. Java is a trademark of Sun Microsystems, Inc and owned by ORACLE Inc. Contact coftware#gmail.com.