You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ki...@apache.org on 2016/01/22 16:22:10 UTC

[2/2] hadoop git commit: HDFS-8898. Create API and command-line argument to get quota and quota usage without detailed content summary. Contributed by Ming Ma. (cherry picked from commit b5db1d44104a8ac4c3643cc3ac841f75f31c8345)

HDFS-8898. Create API and command-line argument to get quota and quota usage without detailed content summary. Contributed by Ming Ma.
(cherry picked from commit b5db1d44104a8ac4c3643cc3ac841f75f31c8345)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/33087668
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/33087668
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/33087668

Branch: refs/heads/branch-2.8
Commit: 330876689b54910990643b34e98e7f65126db5c7
Parents: 2097016
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Jan 22 09:20:39 2016 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Jan 22 09:20:39 2016 -0600

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/ContentSummary.java    | 241 ++++---------
 .../java/org/apache/hadoop/fs/FileSystem.java   |   7 +
 .../java/org/apache/hadoop/fs/QuotaUsage.java   | 359 +++++++++++++++++++
 .../java/org/apache/hadoop/fs/shell/Count.java  |  37 +-
 .../hadoop/fs/viewfs/ChRootedFileSystem.java    |   7 +-
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |  15 +-
 .../src/site/markdown/FileSystemShell.md        |  13 +-
 .../org/apache/hadoop/cli/CLITestHelper.java    |   2 +-
 .../apache/hadoop/fs/TestFilterFileSystem.java  |   6 +-
 .../org/apache/hadoop/fs/TestHarFileSystem.java |   1 +
 .../org/apache/hadoop/fs/TestQuotaUsage.java    | 146 ++++++++
 .../org/apache/hadoop/fs/shell/TestCount.java   | 109 +++++-
 .../src/test/resources/testConf.xml             |   2 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  28 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |  19 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  14 +
 .../ClientNamenodeProtocolTranslatorPB.java     |  14 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  79 +++-
 .../src/main/proto/ClientNamenodeProtocol.proto |  10 +
 .../src/main/proto/hdfs.proto                   |  11 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 ...tNamenodeProtocolServerSideTranslatorPB.java |  16 +
 .../server/namenode/FSDirStatAndListingOp.java  |  53 +++
 .../hdfs/server/namenode/FSNamesystem.java      |  31 ++
 .../hdfs/server/namenode/NameNodeRpcServer.java |   7 +
 .../java/org/apache/hadoop/hdfs/TestQuota.java  |  63 +++-
 .../server/namenode/ha/TestQuotasWithHA.java    |  14 +-
 27 files changed, 1072 insertions(+), 235 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
index ccd6960..3dedbcc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
@@ -30,26 +30,15 @@ import org.apache.hadoop.util.StringUtils;
 /** Store the summary of a content (a directory or a file). */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class ContentSummary implements Writable{
+public class ContentSummary extends QuotaUsage implements Writable{
   private long length;
   private long fileCount;
   private long directoryCount;
-  private long quota;
-  private long spaceConsumed;
-  private long spaceQuota;
-  private long typeConsumed[];
-  private long typeQuota[];
 
-  public static class Builder{
+  /** We don't use generics. Instead override spaceConsumed and other methods
+      in order to keep backward compatibility. */
+  public static class Builder extends QuotaUsage.Builder {
     public Builder() {
-      this.quota = -1;
-      this.spaceQuota = -1;
-
-      typeConsumed = new long[StorageType.values().length];
-      typeQuota = new long[StorageType.values().length];
-      for (int i = 0; i < typeQuota.length; i++) {
-        typeQuota[i] = -1;
-      }
     }
 
     public Builder length(long length) {
@@ -67,58 +56,57 @@ public class ContentSummary implements Writable{
       return this;
     }
 
+    @Override
     public Builder quota(long quota){
-      this.quota = quota;
+      super.quota(quota);
       return this;
     }
 
+    @Override
     public Builder spaceConsumed(long spaceConsumed) {
-      this.spaceConsumed = spaceConsumed;
+      super.spaceConsumed(spaceConsumed);
       return this;
     }
 
+    @Override
     public Builder spaceQuota(long spaceQuota) {
-      this.spaceQuota = spaceQuota;
+      super.spaceQuota(spaceQuota);
       return this;
     }
 
+    @Override
     public Builder typeConsumed(long typeConsumed[]) {
-      for (int i = 0; i < typeConsumed.length; i++) {
-        this.typeConsumed[i] = typeConsumed[i];
-      }
+      super.typeConsumed(typeConsumed);
       return this;
     }
 
+    @Override
     public Builder typeQuota(StorageType type, long quota) {
-      this.typeQuota[type.ordinal()] = quota;
+      super.typeQuota(type, quota);
       return this;
     }
 
+    @Override
     public Builder typeConsumed(StorageType type, long consumed) {
-      this.typeConsumed[type.ordinal()] = consumed;
+      super.typeConsumed(type, consumed);
       return this;
     }
 
+    @Override
     public Builder typeQuota(long typeQuota[]) {
-      for (int i = 0; i < typeQuota.length; i++) {
-        this.typeQuota[i] = typeQuota[i];
-      }
+      super.typeQuota(typeQuota);
       return this;
     }
 
     public ContentSummary build() {
-      return new ContentSummary(length, fileCount, directoryCount, quota,
-          spaceConsumed, spaceQuota, typeConsumed, typeQuota);
+      // Set it in case applications call QuotaUsage#getFileAndDirectoryCount.
+      super.fileAndDirectoryCount(this.fileCount + this.directoryCount);
+      return new ContentSummary(this);
     }
 
     private long length;
     private long fileCount;
     private long directoryCount;
-    private long quota;
-    private long spaceConsumed;
-    private long spaceQuota;
-    private long typeConsumed[];
-    private long typeQuota[];
   }
 
   /** Constructor deprecated by ContentSummary.Builder*/
@@ -143,24 +131,17 @@ public class ContentSummary implements Writable{
     this.length = length;
     this.fileCount = fileCount;
     this.directoryCount = directoryCount;
-    this.quota = quota;
-    this.spaceConsumed = spaceConsumed;
-    this.spaceQuota = spaceQuota;
+    setQuota(quota);
+    setSpaceConsumed(spaceConsumed);
+    setSpaceQuota(spaceQuota);
   }
 
   /** Constructor for ContentSummary.Builder*/
-  private ContentSummary(
-      long length, long fileCount, long directoryCount, long quota,
-      long spaceConsumed, long spaceQuota, long typeConsumed[],
-      long typeQuota[]) {
-    this.length = length;
-    this.fileCount = fileCount;
-    this.directoryCount = directoryCount;
-    this.quota = quota;
-    this.spaceConsumed = spaceConsumed;
-    this.spaceQuota = spaceQuota;
-    this.typeConsumed = typeConsumed;
-    this.typeQuota = typeQuota;
+  private ContentSummary(Builder builder) {
+    super(builder);
+    this.length = builder.length;
+    this.fileCount = builder.fileCount;
+    this.directoryCount = builder.directoryCount;
   }
 
   /** @return the length */
@@ -171,51 +152,6 @@ public class ContentSummary implements Writable{
 
   /** @return the file count */
   public long getFileCount() {return fileCount;}
-  
-  /** Return the directory quota */
-  public long getQuota() {return quota;}
-  
-  /** Retuns storage space consumed */
-  public long getSpaceConsumed() {return spaceConsumed;}
-
-  /** Returns storage space quota */
-  public long getSpaceQuota() {return spaceQuota;}
-
-  /** Returns storage type quota */
-  public long getTypeQuota(StorageType type) {
-    return (typeQuota != null) ? typeQuota[type.ordinal()] : -1;
-  }
-
-  /** Returns storage type consumed*/
-  public long getTypeConsumed(StorageType type) {
-    return (typeConsumed != null) ? typeConsumed[type.ordinal()] : 0;
-  }
-
-  /** Returns true if any storage type quota has been set*/
-  public boolean isTypeQuotaSet() {
-    if (typeQuota == null) {
-      return false;
-    }
-    for (StorageType t : StorageType.getTypesSupportingQuota()) {
-      if (typeQuota[t.ordinal()] > 0) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /** Returns true if any storage type consumption information is available*/
-  public boolean isTypeConsumedAvailable() {
-    if (typeConsumed == null) {
-      return false;
-    }
-    for (StorageType t : StorageType.getTypesSupportingQuota()) {
-      if (typeConsumed[t.ordinal()] > 0) {
-        return true;
-      }
-    }
-    return false;
-  }
 
   @Override
   @InterfaceAudience.Private
@@ -223,9 +159,9 @@ public class ContentSummary implements Writable{
     out.writeLong(length);
     out.writeLong(fileCount);
     out.writeLong(directoryCount);
-    out.writeLong(quota);
-    out.writeLong(spaceConsumed);
-    out.writeLong(spaceQuota);
+    out.writeLong(getQuota());
+    out.writeLong(getSpaceConsumed());
+    out.writeLong(getSpaceQuota());
   }
 
   @Override
@@ -234,9 +170,29 @@ public class ContentSummary implements Writable{
     this.length = in.readLong();
     this.fileCount = in.readLong();
     this.directoryCount = in.readLong();
-    this.quota = in.readLong();
-    this.spaceConsumed = in.readLong();
-    this.spaceQuota = in.readLong();
+    setQuota(in.readLong());
+    setSpaceConsumed(in.readLong());
+    setSpaceQuota(in.readLong());
+  }
+
+  @Override
+  public boolean equals(Object to) {
+    if (this == to) {
+      return true;
+    } else if (to instanceof ContentSummary) {
+      return getLength() == ((ContentSummary) to).getLength() &&
+          getFileCount() == ((ContentSummary) to).getFileCount() &&
+          getDirectoryCount() == ((ContentSummary) to).getDirectoryCount() &&
+          super.equals(to);
+    } else {
+      return super.equals(to);
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    long result = getLength() ^ getFileCount() ^ getDirectoryCount();
+    return ((int) result) ^ super.hashCode();
   }
 
   /**
@@ -245,35 +201,16 @@ public class ContentSummary implements Writable{
    *    DIR_COUNT   FILE_COUNT       CONTENT_SIZE
    */
   private static final String SUMMARY_FORMAT = "%12s %12s %18s ";
-  /**
-   * Output format:
-   * <----12----> <------15-----> <------15-----> <------15----->
-   *        QUOTA       REM_QUOTA     SPACE_QUOTA REM_SPACE_QUOTA
-   * <----12----> <----12----> <-------18------->
-   *    DIR_COUNT   FILE_COUNT       CONTENT_SIZE
-   */
-  private static final String QUOTA_SUMMARY_FORMAT = "%12s %15s ";
-  private static final String SPACE_QUOTA_SUMMARY_FORMAT = "%15s %15s ";
-
-  private static final String STORAGE_TYPE_SUMMARY_FORMAT = "%13s %17s ";
 
-  private static final String[] HEADER_FIELDS = new String[] { "DIR_COUNT",
-      "FILE_COUNT", "CONTENT_SIZE"};
-  private static final String[] QUOTA_HEADER_FIELDS = new String[] { "QUOTA",
-      "REM_QUOTA", "SPACE_QUOTA", "REM_SPACE_QUOTA" };
+  private static final String[] SUMMARY_HEADER_FIELDS =
+      new String[] {"DIR_COUNT", "FILE_COUNT", "CONTENT_SIZE"};
 
   /** The header string */
-  private static final String HEADER = String.format(
-      SUMMARY_FORMAT, (Object[]) HEADER_FIELDS);
+  private static final String SUMMARY_HEADER = String.format(
+      SUMMARY_FORMAT, (Object[]) SUMMARY_HEADER_FIELDS);
 
-  private static final String QUOTA_HEADER = String.format(
-      QUOTA_SUMMARY_FORMAT + SPACE_QUOTA_SUMMARY_FORMAT,
-      (Object[]) QUOTA_HEADER_FIELDS) +
-      HEADER;
+  private static final String ALL_HEADER = QUOTA_HEADER + SUMMARY_HEADER;
 
-  /** default quota display string */
-  private static final String QUOTA_NONE = "none";
-  private static final String QUOTA_INF = "inf";
 
   /** Return the header of the output.
    * if qOption is false, output directory count, file count, and content size;
@@ -283,28 +220,10 @@ public class ContentSummary implements Writable{
    * @return the header of the output
    */
   public static String getHeader(boolean qOption) {
-    return qOption ? QUOTA_HEADER : HEADER;
+    return qOption ? ALL_HEADER : SUMMARY_HEADER;
   }
 
-  /**
-   * return the header of with the StorageTypes
-   *
-   * @param storageTypes
-   * @return storage header string
-   */
-  public static String getStorageTypeHeader(List<StorageType> storageTypes) {
-    StringBuffer header = new StringBuffer();
-
-    for (StorageType st : storageTypes) {
-      /* the field length is 13/17 for quota and remain quota
-       * as the max length for quota name is ARCHIVE_QUOTA
-        * and remain quota name REM_ARCHIVE_QUOTA */
-      String storageName = st.toString();
-      header.append(String.format(STORAGE_TYPE_SUMMARY_FORMAT, storageName + "_QUOTA",
-          "REM_" + storageName + "_QUOTA"));
-    }
-    return header.toString();
-  }
+
 
   /**
    * Returns the names of the fields from the summary header.
@@ -312,7 +231,7 @@ public class ContentSummary implements Writable{
    * @return names of fields as displayed in the header
    */
   public static String[] getHeaderFields() {
-    return HEADER_FIELDS;
+    return SUMMARY_HEADER_FIELDS;
   }
 
   /**
@@ -370,41 +289,11 @@ public class ContentSummary implements Writable{
     String prefix = "";
 
     if (tOption) {
-      StringBuffer content = new StringBuffer();
-      for (StorageType st : types) {
-        long typeQuota = getTypeQuota(st);
-        long typeConsumed = getTypeConsumed(st);
-        String quotaStr = QUOTA_NONE;
-        String quotaRem = QUOTA_INF;
-
-        if (typeQuota > 0) {
-          quotaStr = formatSize(typeQuota, hOption);
-          quotaRem = formatSize(typeQuota - typeConsumed, hOption);
-        }
-
-        content.append(String.format(STORAGE_TYPE_SUMMARY_FORMAT,
-            quotaStr, quotaRem));
-      }
-      return content.toString();
+      return getTypesQuotaUsage(hOption, types);
     }
 
     if (qOption) {
-      String quotaStr = QUOTA_NONE;
-      String quotaRem = QUOTA_INF;
-      String spaceQuotaStr = QUOTA_NONE;
-      String spaceQuotaRem = QUOTA_INF;
-
-      if (quota>0) {
-        quotaStr = formatSize(quota, hOption);
-        quotaRem = formatSize(quota-(directoryCount+fileCount), hOption);
-      }
-      if (spaceQuota>0) {
-        spaceQuotaStr = formatSize(spaceQuota, hOption);
-        spaceQuotaRem = formatSize(spaceQuota - spaceConsumed, hOption);
-      }
-
-      prefix = String.format(QUOTA_SUMMARY_FORMAT + SPACE_QUOTA_SUMMARY_FORMAT,
-          quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
+      prefix = getQuotaUsage(hOption);
     }
 
     return prefix + String.format(SUMMARY_FORMAT,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 2102862..e0320c0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -1489,6 +1489,13 @@ public abstract class FileSystem extends Configured implements Closeable {
         spaceConsumed(summary[0]).build();
   }
 
+  /** Return the {@link QuotaUsage} of a given {@link Path}.
+   * @param f path to use
+   */
+  public QuotaUsage getQuotaUsage(Path f) throws IOException {
+    return getContentSummary(f);
+  }
+
   final private static PathFilter DEFAULT_FILTER = new PathFilter() {
       @Override
       public boolean accept(Path file) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/QuotaUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/QuotaUsage.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/QuotaUsage.java
new file mode 100644
index 0000000..9ffab2f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/QuotaUsage.java
@@ -0,0 +1,359 @@
+/**
+ * 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.fs;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.StringUtils;
+
+/** Store the quota usage of a directory. */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class QuotaUsage {
+  private long fileAndDirectoryCount;
+  // Make the followings protected so that
+  // deprecated ContentSummary constructor can use them.
+  private long quota;
+  private long spaceConsumed;
+  private long spaceQuota;
+  private long[] typeConsumed;
+  private long[] typeQuota;
+
+  /** Builder class for QuotaUsage. */
+  public static class Builder {
+    public Builder() {
+      this.quota = -1;
+      this.spaceQuota = -1;
+
+      typeConsumed = new long[StorageType.values().length];
+      typeQuota = new long[StorageType.values().length];
+      for (int i = 0; i < typeQuota.length; i++) {
+        typeQuota[i] = -1;
+      }
+    }
+
+    public Builder fileAndDirectoryCount(long count) {
+      this.fileAndDirectoryCount = count;
+      return this;
+    }
+
+    public Builder quota(long quota){
+      this.quota = quota;
+      return this;
+    }
+
+    public Builder spaceConsumed(long spaceConsumed) {
+      this.spaceConsumed = spaceConsumed;
+      return this;
+    }
+
+    public Builder spaceQuota(long spaceQuota) {
+      this.spaceQuota = spaceQuota;
+      return this;
+    }
+
+    public Builder typeConsumed(long[] typeConsumed) {
+      for (int i = 0; i < typeConsumed.length; i++) {
+        this.typeConsumed[i] = typeConsumed[i];
+      }
+      return this;
+    }
+
+    public Builder typeQuota(StorageType type, long quota) {
+      this.typeQuota[type.ordinal()] = quota;
+      return this;
+    }
+
+    public Builder typeConsumed(StorageType type, long consumed) {
+      this.typeConsumed[type.ordinal()] = consumed;
+      return this;
+    }
+
+    public Builder typeQuota(long[] typeQuota) {
+      for (int i = 0; i < typeQuota.length; i++) {
+        this.typeQuota[i] = typeQuota[i];
+      }
+      return this;
+    }
+
+    public QuotaUsage build() {
+      return new QuotaUsage(this);
+    }
+
+    private long fileAndDirectoryCount;
+    private long quota;
+    private long spaceConsumed;
+    private long spaceQuota;
+    private long[] typeConsumed;
+    private long[] typeQuota;
+  }
+
+  // Make it protected for the deprecated ContentSummary constructor.
+  protected QuotaUsage() { }
+
+  /** Build the instance based on the builder. */
+  protected QuotaUsage(Builder builder) {
+    this.fileAndDirectoryCount = builder.fileAndDirectoryCount;
+    this.quota = builder.quota;
+    this.spaceConsumed = builder.spaceConsumed;
+    this.spaceQuota = builder.spaceQuota;
+    this.typeConsumed = builder.typeConsumed;
+    this.typeQuota = builder.typeQuota;
+  }
+
+  protected void setQuota(long quota) {
+    this.quota = quota;
+  }
+
+  protected void setSpaceConsumed(long spaceConsumed) {
+    this.spaceConsumed = spaceConsumed;
+  }
+
+  protected void setSpaceQuota(long spaceQuota) {
+    this.spaceQuota = spaceQuota;
+  }
+
+  /** Return the directory count. */
+  public long getFileAndDirectoryCount() {
+    return fileAndDirectoryCount;
+  }
+
+  /** Return the directory quota. */
+  public long getQuota() {
+    return quota;
+  }
+
+  /** Return (disk) space consumed. */
+  public long getSpaceConsumed() {
+    return spaceConsumed;
+  }
+
+  /** Return (disk) space quota. */
+  public long getSpaceQuota() {
+    return spaceQuota;
+  }
+
+  /** Return storage type quota. */
+  public long getTypeQuota(StorageType type) {
+    return (typeQuota != null) ? typeQuota[type.ordinal()] : -1;
+  }
+
+  /** Return storage type consumed. */
+  public long getTypeConsumed(StorageType type) {
+    return (typeConsumed != null) ? typeConsumed[type.ordinal()] : 0;
+  }
+
+  /** Return storage type quota. */
+  private long[] getTypesQuota() {
+    return typeQuota;
+  }
+
+  /** Return storage type quota. */
+  private long[] getTypesConsumed() {
+    return typeConsumed;
+  }
+
+  /** Return true if any storage type quota has been set. */
+  public boolean isTypeQuotaSet() {
+    if (typeQuota == null) {
+      return false;
+    }
+    for (StorageType t : StorageType.getTypesSupportingQuota()) {
+      if (typeQuota[t.ordinal()] > 0) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** Return true if any storage type consumption information is available. */
+  public boolean isTypeConsumedAvailable() {
+    if (typeConsumed == null) {
+      return false;
+    }
+    for (StorageType t : StorageType.getTypesSupportingQuota()) {
+      if (typeConsumed[t.ordinal()] > 0) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public boolean equals(Object to) {
+    return (this == to || (to instanceof QuotaUsage &&
+        getFileAndDirectoryCount() ==
+        ((QuotaUsage) to).getFileAndDirectoryCount() &&
+        getQuota() == ((QuotaUsage) to).getQuota() &&
+        getSpaceConsumed() == ((QuotaUsage) to).getSpaceConsumed() &&
+        getSpaceQuota() == ((QuotaUsage) to).getSpaceQuota() &&
+        Arrays.equals(getTypesQuota(), ((QuotaUsage) to).getTypesQuota()) &&
+        Arrays.equals(getTypesConsumed(),
+        ((QuotaUsage) to).getTypesConsumed())));
+  }
+
+  @Override
+  public int hashCode() {
+    long result = (getFileAndDirectoryCount() ^ getQuota() ^
+        getSpaceConsumed() ^ getSpaceQuota());
+    if (getTypesQuota() != null) {
+      for (long quota : getTypesQuota()) {
+        result ^= quota;
+      }
+    }
+    if (getTypesConsumed() != null) {
+      for (long consumed : getTypesConsumed()) {
+        result ^= consumed;
+      }
+    }
+    return (int)result;
+  }
+
+  /**
+   * Output format:
+   * <----12----> <----15----> <----15----> <----15----> <-------18------->
+   *    QUOTA   REMAINING_QUATA SPACE_QUOTA SPACE_QUOTA_REM FILE_NAME
+   */
+  protected static final String QUOTA_STRING_FORMAT = "%12s %15s ";
+  protected static final String SPACE_QUOTA_STRING_FORMAT = "%15s %15s ";
+
+  protected static final String[] QUOTA_HEADER_FIELDS = new String[] {"QUOTA",
+      "REM_QUOTA", "SPACE_QUOTA", "REM_SPACE_QUOTA"};
+
+  protected static final String QUOTA_HEADER = String.format(
+      QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT,
+      (Object[]) QUOTA_HEADER_FIELDS);
+
+  /**
+   * Output format:
+   * <----12----> <------15-----> <------15-----> <------15----->
+   *        QUOTA       REM_QUOTA     SPACE_QUOTA REM_SPACE_QUOTA
+   * <----12----> <----12----> <-------18------->
+   *    DIR_COUNT   FILE_COUNT       CONTENT_SIZE
+   */
+  private static final String STORAGE_TYPE_SUMMARY_FORMAT = "%13s %17s ";
+
+  /** Return the header of the output.
+   * @return the header of the output
+   */
+  public static String getHeader() {
+    return QUOTA_HEADER;
+  }
+
+  /** default quota display string */
+  private static final String QUOTA_NONE = "none";
+  private static final String QUOTA_INF = "inf";
+
+  @Override
+  public String toString() {
+    return toString(false);
+  }
+
+  public String toString(boolean hOption) {
+    return toString(hOption, false, null);
+  }
+
+  /** Return the string representation of the object in the output format.
+   * if hOption is false file sizes are returned in bytes
+   * if hOption is true file sizes are returned in human readable
+   *
+   * @param hOption a flag indicating if human readable output if to be used
+   * @return the string representation of the object
+   */
+  public String toString(boolean hOption,
+      boolean tOption, List<StorageType> types) {
+    if (tOption) {
+      return getTypesQuotaUsage(hOption, types);
+    }
+    return getQuotaUsage(hOption);
+  }
+
+  protected String getQuotaUsage(boolean hOption) {
+    String quotaStr = QUOTA_NONE;
+    String quotaRem = QUOTA_INF;
+    String spaceQuotaStr = QUOTA_NONE;
+    String spaceQuotaRem = QUOTA_INF;
+
+    if (quota > 0) {
+      quotaStr = formatSize(quota, hOption);
+      quotaRem = formatSize(quota-fileAndDirectoryCount, hOption);
+    }
+    if (spaceQuota > 0) {
+      spaceQuotaStr = formatSize(spaceQuota, hOption);
+      spaceQuotaRem = formatSize(spaceQuota - spaceConsumed, hOption);
+    }
+
+    return String.format(QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT,
+        quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
+  }
+
+  protected String getTypesQuotaUsage(boolean hOption,
+      List<StorageType> types) {
+    StringBuffer content = new StringBuffer();
+    for (StorageType st : types) {
+      long typeQuota = getTypeQuota(st);
+      long typeConsumed = getTypeConsumed(st);
+      String quotaStr = QUOTA_NONE;
+      String quotaRem = QUOTA_INF;
+
+      if (typeQuota > 0) {
+        quotaStr = formatSize(typeQuota, hOption);
+        quotaRem = formatSize(typeQuota - typeConsumed, hOption);
+      }
+
+      content.append(String.format(STORAGE_TYPE_SUMMARY_FORMAT,
+          quotaStr, quotaRem));
+    }
+    return content.toString();
+  }
+
+  /**
+   * return the header of with the StorageTypes.
+   *
+   * @param storageTypes
+   * @return storage header string
+   */
+  public static String getStorageTypeHeader(List<StorageType> storageTypes) {
+    StringBuffer header = new StringBuffer();
+
+    for (StorageType st : storageTypes) {
+      /* the field length is 13/17 for quota and remain quota
+       * as the max length for quota name is ARCHIVE_QUOTA
+        * and remain quota name REM_ARCHIVE_QUOTA */
+      String storageName = st.toString();
+      header.append(String.format(STORAGE_TYPE_SUMMARY_FORMAT,
+          storageName + "_QUOTA", "REM_" + storageName + "_QUOTA"));
+    }
+    return header.toString();
+  }
+
+  /**
+   * Formats a size to be human readable or in bytes.
+   * @param size value to be formatted
+   * @param humanReadable flag indicating human readable or not
+   * @return String representation of the size
+  */
+  private String formatSize(long size, boolean humanReadable) {
+    return humanReadable
+      ? StringUtils.TraditionalBinaryPrefix.long2String(size, "", 1)
+      : String.valueOf(size);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
index c615876..2b5a4cc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 
 /**
@@ -50,11 +51,14 @@ public class Count extends FsCommand {
   private static final String OPTION_HUMAN = "h";
   private static final String OPTION_HEADER = "v";
   private static final String OPTION_TYPE = "t";
+  //return the quota, namespace count and disk space usage.
+  private static final String OPTION_QUOTA_AND_USAGE = "u";
 
   public static final String NAME = "count";
   public static final String USAGE =
       "[-" + OPTION_QUOTA + "] [-" + OPTION_HUMAN + "] [-" + OPTION_HEADER
-          + "] [-" + OPTION_TYPE + " [<storage type>]] <path> ...";
+          + "] [-" + OPTION_TYPE + " [<storage type>]] [-" +
+          OPTION_QUOTA_AND_USAGE + "] <path> ...";
   public static final String DESCRIPTION =
       "Count the number of directories, files and bytes under the paths\n" +
           "that match the specified file pattern.  The output columns are:\n" +
@@ -74,12 +78,15 @@ public class Count extends FsCommand {
           OPTION_TYPE + " option, \n" +
           "it displays the quota and usage for the specified types. \n" +
           "Otherwise, it displays the quota and usage for all the storage \n" +
-          "types that support quota";
+          "types that support quota \n" +
+          "The -" + OPTION_QUOTA_AND_USAGE + " option shows the quota and \n" +
+          "the usage against the quota without the detailed content summary.";
 
   private boolean showQuotas;
   private boolean humanReadable;
   private boolean showQuotabyType;
   private List<StorageType> storageTypes = null;
+  private boolean showQuotasAndUsageOnly;
 
   /** Constructor */
   public Count() {}
@@ -99,7 +106,7 @@ public class Count extends FsCommand {
   @Override
   protected void processOptions(LinkedList<String> args) {
     CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE,
-        OPTION_QUOTA, OPTION_HUMAN, OPTION_HEADER);
+        OPTION_QUOTA, OPTION_HUMAN, OPTION_HEADER, OPTION_QUOTA_AND_USAGE);
     cf.addOptionWithValue(OPTION_TYPE);
     cf.parse(args);
     if (args.isEmpty()) { // default path is the current working directory
@@ -107,8 +114,9 @@ public class Count extends FsCommand {
     }
     showQuotas = cf.getOpt(OPTION_QUOTA);
     humanReadable = cf.getOpt(OPTION_HUMAN);
+    showQuotasAndUsageOnly = cf.getOpt(OPTION_QUOTA_AND_USAGE);
 
-    if (showQuotas) {
+    if (showQuotas || showQuotasAndUsageOnly) {
       String types = cf.getOptValue(OPTION_TYPE);
 
       if (null != types) {
@@ -121,9 +129,13 @@ public class Count extends FsCommand {
 
     if (cf.getOpt(OPTION_HEADER)) {
       if (showQuotabyType) {
-        out.println(ContentSummary.getStorageTypeHeader(storageTypes) + "PATHNAME");
+        out.println(QuotaUsage.getStorageTypeHeader(storageTypes) + "PATHNAME");
       } else {
-        out.println(ContentSummary.getHeader(showQuotas) + "PATHNAME");
+        if (showQuotasAndUsageOnly) {
+          out.println(QuotaUsage.getHeader() + "PATHNAME");
+        } else {
+          out.println(ContentSummary.getHeader(showQuotas) + "PATHNAME");
+        }
       }
     }
   }
@@ -145,11 +157,16 @@ public class Count extends FsCommand {
 
   @Override
   protected void processPath(PathData src) throws IOException {
-    ContentSummary summary = src.fs.getContentSummary(src.path);
-    out.println(summary.toString(showQuotas, isHumanReadable(),
-        showQuotabyType, storageTypes) + src);
+    if (showQuotasAndUsageOnly || showQuotabyType) {
+      QuotaUsage usage = src.fs.getQuotaUsage(src.path);
+      out.println(usage.toString(isHumanReadable(), showQuotabyType,
+          storageTypes) + src);
+    } else {
+      ContentSummary summary = src.fs.getContentSummary(src.path);
+      out.println(summary.toString(showQuotas, isHumanReadable()) + src);
+    }
   }
-  
+
   /**
    * Should quotas get shown as part of the report?
    * @return if quotas should be shown then true otherwise false

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
index dd28c58..46e8f2d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Progressable;
@@ -388,7 +389,11 @@ class ChRootedFileSystem extends FilterFileSystem {
   public ContentSummary getContentSummary(Path f) throws IOException {
     return fs.getContentSummary(fullPath(f));
   }
-  
+
+  @Override
+  public QuotaUsage getQuotaUsage(Path f) throws IOException {
+    return fs.getQuotaUsage(fullPath(f));
+  }
 
   private static Path rootPath = new Path(Path.SEPARATOR);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 8605064..5f42e70 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclUtil;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.viewfs.InodeTree.INode;
 import org.apache.hadoop.fs.viewfs.InodeTree.INodeLink;
 import org.apache.hadoop.security.AccessControlException;
@@ -689,12 +690,19 @@ public class ViewFileSystem extends FileSystem {
 
   @Override
   public ContentSummary getContentSummary(Path f) throws IOException {
-    InodeTree.ResolveResult<FileSystem> res = 
+    InodeTree.ResolveResult<FileSystem> res =
       fsState.resolve(getUriPath(f), true);
     return res.targetFileSystem.getContentSummary(res.remainingPath);
   }
 
   @Override
+  public QuotaUsage getQuotaUsage(Path f) throws IOException {
+    InodeTree.ResolveResult<FileSystem> res =
+        fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.getQuotaUsage(res.remainingPath);
+  }
+
+  @Override
   public void setWriteChecksum(final boolean writeChecksum) { 
     List<InodeTree.MountPoint<FileSystem>> mountPoints = 
         fsState.getMountPoints();
@@ -1066,5 +1074,10 @@ public class ViewFileSystem extends FileSystem {
       checkPathIsSlash(path);
       throw readOnlyMountTable("deleteSnapshot", path);
     }
+
+    @Override
+    public QuotaUsage getQuotaUsage(Path f) throws IOException {
+      throw new NotInMountpointException(f, "getQuotaUsage");
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
index 685e954..4f12b9d 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
@@ -170,12 +170,16 @@ Similar to get command, except that the destination is restricted to a local fil
 count
 -----
 
-Usage: `hadoop fs -count [-q] [-h] [-v] <paths> `
+Usage: `hadoop fs -count [-q] [-h] [-v] [-t [<storage type>]] [-u] <paths> `
 
-Count the number of directories, files and bytes under the paths that match the specified file pattern. The output columns with -count are: DIR\_COUNT, FILE\_COUNT, CONTENT\_SIZE, PATHNAME
+Count the number of directories, files and bytes under the paths that match the specified file pattern. Get the quota and the usage. The output columns with -count are: DIR\_COUNT, FILE\_COUNT, CONTENT\_SIZE, PATHNAME
 
 The output columns with -count -q are: QUOTA, REMAINING\_QUOTA, SPACE\_QUOTA, REMAINING\_SPACE\_QUOTA, DIR\_COUNT, FILE\_COUNT, CONTENT\_SIZE, PATHNAME
 
+The output columns with -count -u are: QUOTA, REMAINING\_QUOTA, SPACE\_QUOTA, REMAINING\_SPACE\_QUOTA
+
+The -t option shows the quota and usage for each storage type.
+
 The -h option shows sizes in human readable format.
 
 The -v option displays a header line.
@@ -185,7 +189,10 @@ Example:
 * `hadoop fs -count hdfs://nn1.example.com/file1 hdfs://nn2.example.com/file2`
 * `hadoop fs -count -q hdfs://nn1.example.com/file1`
 * `hadoop fs -count -q -h hdfs://nn1.example.com/file1`
-* `hdfs dfs -count -q -h -v hdfs://nn1.example.com/file1`
+* `hadoop fs -count -q -h -v hdfs://nn1.example.com/file1`
+* `hadoop fs -count -u hdfs://nn1.example.com/file1`
+* `hadoop fs -count -u -h hdfs://nn1.example.com/file1`
+* `hadoop fs -count -u -h -v hdfs://nn1.example.com/file1`
 
 Exit Code:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
index d39727b..b08af16 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
@@ -68,7 +68,7 @@ public class CLITestHelper {
   protected String clitestDataDir = null;
   protected String username = null;
   /**
-   * Read the test config file - testConfig.xml
+   * Read the test config file - testConf.xml
    */
   protected void readTestConfigFile() {
     String testConfigFile = getTestFile();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
index abb75f5..c42ebac 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
@@ -126,13 +126,13 @@ public class TestFilterFileSystem {
     public Token<?> getDelegationToken(String renewer) throws IOException;
     public boolean deleteOnExit(Path f) throws IOException;
     public boolean cancelDeleteOnExit(Path f) throws IOException;
-    public Token<?>[] addDelegationTokens(String renewer, Credentials creds)
-        throws IOException;
+    public Token<?>[] addDelegationTokens(String renewer, Credentials creds);
     public String getScheme();
     public Path fixRelativePart(Path p);
     public ContentSummary getContentSummary(Path f);
+    public QuotaUsage getQuotaUsage(Path f);
   }
-  
+
   @Test
   public void testFilterFileSystem() throws Exception {
     int errors = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
index 8ec9d80..26923a8 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
@@ -112,6 +112,7 @@ public class TestHarFileSystem {
     public short getReplication(Path src);
     public void processDeleteOnExit();
     public ContentSummary getContentSummary(Path f);
+    public QuotaUsage getQuotaUsage(Path f);
     public FsStatus getStatus();
     public FileStatus[] listStatus(Path f, PathFilter filter);
     public FileStatus[] listStatus(Path[] files);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestQuotaUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestQuotaUsage.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestQuotaUsage.java
new file mode 100644
index 0000000..e3e2002
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestQuotaUsage.java
@@ -0,0 +1,146 @@
+/**
+ * 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.fs;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+public class TestQuotaUsage {
+
+  // check the empty constructor correctly initialises the object
+  @Test
+  public void testConstructorEmpty() {
+    QuotaUsage quotaUsage = new QuotaUsage.Builder().build();
+    assertEquals("getQuota", -1, quotaUsage.getQuota());
+    assertEquals("getSpaceConsumed", 0, quotaUsage.getSpaceConsumed());
+    assertEquals("getSpaceQuota", -1, quotaUsage.getSpaceQuota());
+  }
+
+  // check the full constructor with quota information
+  @Test
+  public void testConstructorWithQuota() {
+    long fileAndDirCount = 22222;
+    long quota = 44444;
+    long spaceConsumed = 55555;
+    long spaceQuota = 66666;
+
+    QuotaUsage quotaUsage = new QuotaUsage.Builder().
+        fileAndDirectoryCount(fileAndDirCount).quota(quota).
+        spaceConsumed(spaceConsumed).spaceQuota(spaceQuota).build();
+    assertEquals("getFileAndDirectoryCount", fileAndDirCount,
+        quotaUsage.getFileAndDirectoryCount());
+    assertEquals("getQuota", quota, quotaUsage.getQuota());
+    assertEquals("getSpaceConsumed", spaceConsumed,
+        quotaUsage.getSpaceConsumed());
+    assertEquals("getSpaceQuota", spaceQuota, quotaUsage.getSpaceQuota());
+  }
+
+  // check the constructor with quota information
+  @Test
+  public void testConstructorNoQuota() {
+    long spaceConsumed = 11111;
+    long fileAndDirCount = 22222;
+    QuotaUsage quotaUsage = new QuotaUsage.Builder().
+        fileAndDirectoryCount(fileAndDirCount).
+        spaceConsumed(spaceConsumed).build();
+    assertEquals("getFileAndDirectoryCount", fileAndDirCount,
+        quotaUsage.getFileAndDirectoryCount());
+    assertEquals("getQuota", -1, quotaUsage.getQuota());
+    assertEquals("getSpaceConsumed", spaceConsumed,
+        quotaUsage.getSpaceConsumed());
+    assertEquals("getSpaceQuota", -1, quotaUsage.getSpaceQuota());
+  }
+
+  // check the header
+  @Test
+  public void testGetHeader() {
+    String header = "       QUOTA       REM_QUOTA     SPACE_QUOTA "
+        + "REM_SPACE_QUOTA ";
+    assertEquals(header, QuotaUsage.getHeader());
+  }
+
+  // check the toString method with quotas
+  @Test
+  public void testToStringWithQuota() {
+    long fileAndDirCount = 55555;
+    long quota = 44444;
+    long spaceConsumed = 55555;
+    long spaceQuota = 66665;
+
+    QuotaUsage quotaUsage = new QuotaUsage.Builder().
+        fileAndDirectoryCount(fileAndDirCount).quota(quota).
+        spaceConsumed(spaceConsumed).spaceQuota(spaceQuota).build();
+    String expected ="       44444          -11111           66665" +
+        "           11110 ";
+    assertEquals(expected, quotaUsage.toString());
+  }
+
+  // check the toString method with quotas
+  @Test
+  public void testToStringNoQuota() {
+    QuotaUsage quotaUsage = new QuotaUsage.Builder().
+        fileAndDirectoryCount(1234).build();
+    String expected = "        none             inf            none"
+        + "             inf ";
+    assertEquals(expected, quotaUsage.toString());
+  }
+
+  // check the toString method with quotas
+  @Test
+  public void testToStringHumanWithQuota() {
+    long fileAndDirCount = 222255555;
+    long quota = 222256578;
+    long spaceConsumed = 1073741825;
+    long spaceQuota = 1;
+
+    QuotaUsage quotaUsage = new QuotaUsage.Builder().
+        fileAndDirectoryCount(fileAndDirCount).quota(quota).
+        spaceConsumed(spaceConsumed).spaceQuota(spaceQuota).build();
+    String expected = "     212.0 M            1023               1 "
+        + "           -1 G ";
+    assertEquals(expected, quotaUsage.toString(true));
+  }
+
+  // check the equality
+  @Test
+  public void testCompareQuotaUsage() {
+    long fileAndDirCount = 222255555;
+    long quota = 222256578;
+    long spaceConsumed = 1073741825;
+    long spaceQuota = 1;
+    long SSDspaceConsumed = 100000;
+    long SSDQuota = 300000;
+
+    QuotaUsage quotaUsage1 = new QuotaUsage.Builder().
+        fileAndDirectoryCount(fileAndDirCount).quota(quota).
+        spaceConsumed(spaceConsumed).spaceQuota(spaceQuota).
+        typeConsumed(StorageType.SSD, SSDQuota).
+        typeQuota(StorageType.SSD, SSDQuota).
+        build();
+
+    QuotaUsage quotaUsage2 = new QuotaUsage.Builder().
+        fileAndDirectoryCount(fileAndDirCount).quota(quota).
+        spaceConsumed(spaceConsumed).spaceQuota(spaceQuota).
+        typeConsumed(StorageType.SSD, SSDQuota).
+        typeQuota(StorageType.SSD, SSDQuota).
+        build();
+
+    assertEquals(quotaUsage1, quotaUsage2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
index 1517359..f8f08d6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FilterFileSystem;
@@ -47,6 +48,7 @@ public class TestCount {
   private static final String NO_QUOTAS = "Content summary without quotas";
   private static final String HUMAN = "human: ";
   private static final String BYTES = "bytes: ";
+  private static final String QUOTAS_AND_USAGE = "quotas and usage";
   private static Configuration conf;
   private static FileSystem mockFs;
   private static FileStatus fileStat;
@@ -344,7 +346,20 @@ public class TestCount {
   }
 
   @Test
-  public void processPathWithQuotasByMultipleStorageTypesContent() throws Exception {
+  public void processPathWithQuotasByMultipleStorageTypesContent()
+      throws Exception {
+    processMultipleStorageTypesContent(false);
+  }
+
+  @Test
+  public void processPathWithQuotaUsageByMultipleStorageTypesContent()
+      throws Exception {
+    processMultipleStorageTypesContent(true);
+  }
+
+  // "-q -t" is the same as "-u -t"; only return the storage quota and usage.
+  private void processMultipleStorageTypesContent(boolean quotaUsageOnly)
+    throws Exception {
     Path path = new Path("mockfs:/test");
 
     when(mockFs.getFileStatus(eq(path))).thenReturn(fileStat);
@@ -356,7 +371,7 @@ public class TestCount {
     count.out = out;
 
     LinkedList<String> options = new LinkedList<String>();
-    options.add("-q");
+    options.add(quotaUsageOnly ? "-u" : "-q");
     options.add("-t");
     options.add("SSD,DISK");
     options.add("dummy");
@@ -431,7 +446,8 @@ public class TestCount {
   public void getUsage() {
     Count count = new Count();
     String actual = count.getUsage();
-    String expected = "-count [-q] [-h] [-v] [-t [<storage type>]] <path> ...";
+    String expected =
+        "-count [-q] [-h] [-v] [-t [<storage type>]] [-u] <path> ...";
     assertEquals("Count.getUsage", expected, actual);
   }
 
@@ -454,11 +470,47 @@ public class TestCount {
         + "If a comma-separated list of storage types is given after the -t option, \n"
         + "it displays the quota and usage for the specified types. \n"
         + "Otherwise, it displays the quota and usage for all the storage \n"
-        + "types that support quota";
+        + "types that support quota \n"
+        + "The -u option shows the quota and \n"
+        + "the usage against the quota without the detailed content summary.";
 
     assertEquals("Count.getDescription", expected, actual);
   }
 
+  @Test
+  public void processPathWithQuotaUsageHuman() throws Exception {
+    processPathWithQuotaUsage(false);
+  }
+
+  @Test
+  public void processPathWithQuotaUsageRawBytes() throws Exception {
+    processPathWithQuotaUsage(true);
+  }
+
+  private void processPathWithQuotaUsage(boolean rawBytes) throws Exception {
+    Path path = new Path("mockfs:/test");
+
+    when(mockFs.getFileStatus(eq(path))).thenReturn(fileStat);
+    PathData pathData = new PathData(path.toString(), conf);
+
+    PrintStream out = mock(PrintStream.class);
+
+    Count count = new Count();
+    count.out = out;
+
+    LinkedList<String> options = new LinkedList<String>();
+    if (!rawBytes) {
+      options.add("-h");
+    }
+    options.add("-u");
+    options.add("dummy");
+    count.processOptions(options);
+    count.processPath(pathData);
+    String withStorageType = (rawBytes ? BYTES : HUMAN) + QUOTAS_AND_USAGE +
+        pathData.toString();
+    verify(out).println(withStorageType);
+    verifyNoMoreInteractions(out);
+  }
 
   // mock content system
   static class MockContentSummary extends ContentSummary {
@@ -468,19 +520,7 @@ public class TestCount {
     }
 
     @Override
-    public String toString(boolean qOption, boolean hOption,
-                           boolean tOption, List<StorageType> types) {
-      if (tOption) {
-        StringBuffer result = new StringBuffer();
-        result.append(hOption ? HUMAN : BYTES);
-
-        for (StorageType type : types) {
-          result.append(type.toString());
-          result.append(" ");
-        }
-        return result.toString();
-      }
-
+    public String toString(boolean qOption, boolean hOption) {
       if (qOption) {
         if (hOption) {
           return (HUMAN + WITH_QUOTAS);
@@ -497,6 +537,36 @@ public class TestCount {
     }
   }
 
+  // mock content system
+  static class MockQuotaUsage extends QuotaUsage {
+
+    @SuppressWarnings("deprecation")
+    // suppress warning on the usage of deprecated ContentSummary constructor
+    public MockQuotaUsage() {
+    }
+
+    @Override
+    public String toString(boolean hOption,
+        boolean tOption, List<StorageType> types) {
+      if (tOption) {
+        StringBuffer result = new StringBuffer();
+        result.append(hOption ? HUMAN : BYTES);
+
+        for (StorageType type : types) {
+          result.append(type.toString());
+          result.append(" ");
+        }
+        return result.toString();
+      }
+
+      if (hOption) {
+        return (HUMAN + QUOTAS_AND_USAGE);
+      } else {
+        return (BYTES + QUOTAS_AND_USAGE);
+      }
+    }
+  }
+
   // mock file system for use in testing
   static class MockFileSystem extends FilterFileSystem {
     Configuration conf;
@@ -524,5 +594,10 @@ public class TestCount {
     public Configuration getConf() {
       return conf;
     }
+
+    @Override
+    public QuotaUsage getQuotaUsage(Path f) throws IOException {
+      return new MockQuotaUsage();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
index 693ba98..3c34171 100644
--- a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
+++ b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
@@ -270,7 +270,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-count \[-q\] \[-h\] \[-v\] \[-t \[&lt;storage type&gt;\]\] &lt;path&gt; \.\.\. :( )*</expected-output>
+          <expected-output>^-count \[-q\] \[-h\] \[-v\] \[-t \[&lt;storage type&gt;\]\] \[-u\] &lt;path&gt; \.\.\. :( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index ef5ec2d..4c0f3a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -90,6 +90,7 @@ import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.VolumeId;
@@ -167,7 +168,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.RpcInvocationHandler;
+import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
@@ -2491,6 +2492,31 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /**
+   * Get {@link org.apache.hadoop.fs.QuotaUsage} rooted at the specified directory.
+   * @param src The string representation of the path
+   *
+   * @see ClientProtocol#getQuotaUsage(String)
+   */
+  QuotaUsage getQuotaUsage(String src) throws IOException {
+    checkOpen();
+    try (TraceScope ignored = newPathTraceScope("getQuotaUsage", src)) {
+      return namenode.getQuotaUsage(src);
+    } catch(RemoteException re) {
+      IOException ioe = re.unwrapRemoteException(AccessControlException.class,
+          FileNotFoundException.class,
+          UnresolvedPathException.class,
+          RpcNoSuchMethodException.class);
+      if (ioe instanceof RpcNoSuchMethodException) {
+        LOG.debug("The version of namenode doesn't support getQuotaUsage API." +
+            " Fall back to use getContentSummary API.");
+        return getContentSummary(src);
+      } else {
+        throw ioe;
+      }
+    }
+  }
+
+  /**
    * Sets or resets quotas for a directory.
    * @see ClientProtocol#setQuota(String, long, long, StorageType)
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 366e4e1..32e700d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
@@ -759,6 +760,24 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
   }
 
+  @Override
+  public QuotaUsage getQuotaUsage(Path f) throws IOException {
+    statistics.incrementReadOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<QuotaUsage>() {
+      @Override
+      public QuotaUsage doCall(final Path p)
+              throws IOException, UnresolvedLinkException {
+        return dfs.getQuotaUsage(getPathName(p));
+      }
+      @Override
+      public QuotaUsage next(final FileSystem fs, final Path p)
+              throws IOException {
+        return fs.getQuotaUsage(p);
+      }
+    }.resolve(this, absF);
+  }
+
   /** Set a directory's quotas
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String,
    * long, long, StorageType)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 436aee7..dcf7c7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
@@ -1482,4 +1483,17 @@ public interface ClientProtocol {
    */
   @Idempotent
   EventBatchList getEditsFromTxid(long txid) throws IOException;
+
+  /**
+   * Get {@link QuotaUsage} rooted at the specified directory.
+   * @param path The string representation of the path
+   *
+   * @throws AccessControlException permission denied
+   * @throws java.io.FileNotFoundException file <code>path</code> is not found
+   * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>path</code>
+   *         contains a symlink.
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  QuotaUsage getQuotaUsage(String path) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index a042ceb..c94d515 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
@@ -110,6 +111,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLin
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
@@ -1479,4 +1481,16 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public QuotaUsage getQuotaUsage(String path) throws IOException {
+    GetQuotaUsageRequestProto req =
+        GetQuotaUsageRequestProto.newBuilder().setPath(path).build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getQuotaUsage(null, req)
+          .getUsage());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index d64bdd9..ff6f892 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
@@ -135,6 +136,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.File
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.QuotaUsageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
@@ -1364,12 +1366,37 @@ public class PBHelperClient {
         spaceConsumed(cs.getSpaceConsumed()).
         spaceQuota(cs.getSpaceQuota());
     if (cs.hasTypeQuotaInfos()) {
-      for (HdfsProtos.StorageTypeQuotaInfoProto info :
-          cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
-        StorageType type = convertStorageType(info.getType());
-        builder.typeConsumed(type, info.getConsumed());
-        builder.typeQuota(type, info.getQuota());
-      }
+      addStorageTypes(cs.getTypeQuotaInfos(), builder);
+    }
+    return builder.build();
+  }
+
+  public static QuotaUsage convert(QuotaUsageProto qu) {
+    if (qu == null) {
+      return null;
+    }
+    QuotaUsage.Builder builder = new QuotaUsage.Builder();
+    builder.fileAndDirectoryCount(qu.getFileAndDirectoryCount()).
+        quota(qu.getQuota()).
+        spaceConsumed(qu.getSpaceConsumed()).
+        spaceQuota(qu.getSpaceQuota());
+    if (qu.hasTypeQuotaInfos()) {
+      addStorageTypes(qu.getTypeQuotaInfos(), builder);
+    }
+    return builder.build();
+  }
+
+  public static QuotaUsageProto convert(QuotaUsage qu) {
+    if (qu == null) {
+      return null;
+    }
+    QuotaUsageProto.Builder builder = QuotaUsageProto.newBuilder();
+    builder.setFileAndDirectoryCount(qu.getFileAndDirectoryCount()).
+        setQuota(qu.getQuota()).
+        setSpaceConsumed(qu.getSpaceConsumed()).
+        setSpaceQuota(qu.getSpaceQuota());
+    if (qu.isTypeQuotaSet() || qu.isTypeConsumedAvailable()) {
+      builder.setTypeQuotaInfos(getBuilder(qu));
     }
     return builder.build();
   }
@@ -1927,22 +1954,38 @@ public class PBHelperClient {
         setSpaceQuota(cs.getSpaceQuota());
 
     if (cs.isTypeQuotaSet() || cs.isTypeConsumedAvailable()) {
-      HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
-          HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
-      for (StorageType t: StorageType.getTypesSupportingQuota()) {
-        HdfsProtos.StorageTypeQuotaInfoProto info =
-            HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
-                setType(convertStorageType(t)).
-                setConsumed(cs.getTypeConsumed(t)).
-                setQuota(cs.getTypeQuota(t)).
-                build();
-        isb.addTypeQuotaInfo(info);
-      }
-      builder.setTypeQuotaInfos(isb);
+      builder.setTypeQuotaInfos(getBuilder(cs));
     }
     return builder.build();
   }
 
+  private static void addStorageTypes(
+      HdfsProtos.StorageTypeQuotaInfosProto typeQuotaInfos,
+      QuotaUsage.Builder builder) {
+    for (HdfsProtos.StorageTypeQuotaInfoProto info :
+        typeQuotaInfos.getTypeQuotaInfoList()) {
+      StorageType type = convertStorageType(info.getType());
+      builder.typeConsumed(type, info.getConsumed());
+      builder.typeQuota(type, info.getQuota());
+    }
+  }
+
+  private static HdfsProtos.StorageTypeQuotaInfosProto.Builder getBuilder(
+      QuotaUsage qu) {
+    HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
+            HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
+    for (StorageType t: StorageType.getTypesSupportingQuota()) {
+      HdfsProtos.StorageTypeQuotaInfoProto info =
+          HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
+              setType(convertStorageType(t)).
+              setConsumed(qu.getTypeConsumed(t)).
+              setQuota(qu.getTypeQuota(t)).
+              build();
+      isb.addTypeQuotaInfo(info);
+    }
+    return isb;
+  }
+
   public static DatanodeStorageProto convert(DatanodeStorage s) {
     return DatanodeStorageProto.newBuilder()
         .setState(convertState(s.getState()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index ba4f9ee..0752297 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -578,6 +578,14 @@ message GetContentSummaryResponseProto {
   required ContentSummaryProto summary = 1;
 }
 
+message GetQuotaUsageRequestProto {
+  required string path = 1;
+}
+
+message GetQuotaUsageResponseProto {
+  required QuotaUsageProto usage = 1;
+}
+
 message SetQuotaRequestProto {
   required string path = 1;
   required uint64 namespaceQuota = 2;
@@ -869,4 +877,6 @@ service ClientNamenodeProtocol {
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
       returns(GetEditsFromTxidResponseProto);
+  rpc getQuotaUsage(GetQuotaUsageRequestProto)
+      returns(GetQuotaUsageResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index 0e2d541..5d7b601 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -139,6 +139,17 @@ message ContentSummaryProto {
 }
 
 /**
+ * Summary of quota usage of a directory
+ */
+message QuotaUsageProto {
+  required uint64 fileAndDirectoryCount = 1;
+  required uint64 quota = 2;
+  required uint64 spaceConsumed = 3;
+  required uint64 spaceQuota = 4;
+  optional StorageTypeQuotaInfosProto typeQuotaInfos = 5;
+}
+
+/**
  * Storage type quota and usage information of a file or directory
  */
 message StorageTypeQuotaInfosProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e0b78bb..77db003 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -882,6 +882,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9655. NN should start JVM pause monitor before loading fsimage.
     (John Zhuge via Lei (Eddy) Xu)
 
+    HDFS-8898. Create API and command-line argument to get quota and quota
+    usage without detailed content summary. (Ming Ma via kihwal)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 054570a..a88307f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -120,6 +121,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLis
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
@@ -1516,4 +1519,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetQuotaUsageResponseProto getQuotaUsage(
+      RpcController controller, GetQuotaUsageRequestProto req)
+      throws ServiceException {
+    try {
+      QuotaUsage result = server.getQuotaUsage(req.getPath());
+      return GetQuotaUsageResponseProto.newBuilder()
+          .setUsage(PBHelperClient.convert(result)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index c5c2fb4..0dd6aa7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@@ -611,6 +612,58 @@ class FSDirStatAndListingOp {
     }
   }
 
+  static QuotaUsage getQuotaUsage(
+      FSDirectory fsd, String src) throws IOException {
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    final INodesInPath iip;
+    fsd.readLock();
+    try {
+      src = fsd.resolvePath(pc, src, pathComponents);
+      iip = fsd.getINodesInPath(src, false);
+      if (fsd.isPermissionEnabled()) {
+        fsd.checkPermission(pc, iip, false, null, null, null,
+            FsAction.READ_EXECUTE);
+      }
+    } finally {
+      fsd.readUnlock();
+    }
+    QuotaUsage usage = getQuotaUsageInt(fsd, iip);
+    if (usage != null) {
+      return usage;
+    } else {
+      //If quota isn't set, fall back to getContentSummary.
+      return getContentSummaryInt(fsd, iip);
+    }
+  }
+
+  private static QuotaUsage getQuotaUsageInt(FSDirectory fsd, INodesInPath iip)
+    throws IOException {
+    fsd.readLock();
+    try {
+      INode targetNode = iip.getLastINode();
+      QuotaUsage usage = null;
+      if (targetNode.isDirectory()) {
+        DirectoryWithQuotaFeature feature =
+            targetNode.asDirectory().getDirectoryWithQuotaFeature();
+        if (feature != null) {
+          QuotaCounts counts = feature.getSpaceConsumed();
+          QuotaCounts quotas = feature.getQuota();
+          usage = new QuotaUsage.Builder().
+              fileAndDirectoryCount(counts.getNameSpace()).
+              quota(quotas.getNameSpace()).
+              spaceConsumed(counts.getStorageSpace()).
+              spaceQuota(quotas.getStorageSpace()).
+              typeConsumed(counts.getTypeSpaces().asArray()).
+              typeQuota(quotas.getTypeSpaces().asArray()).build();
+        }
+      }
+      return usage;
+    } finally {
+      fsd.readUnlock();
+    }
+  }
+
   static class GetBlockLocationsResult {
     final boolean updateAccessTime;
     final LocatedBlocks blocks;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 384bbf6..9ec848e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -164,6 +164,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -2912,6 +2913,36 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
+   * Get the quota usage for a specific file/dir.
+   *
+   * @param src The string representation of the path to the file
+   *
+   * @throws AccessControlException if access is denied
+   * @throws UnresolvedLinkException if a symlink is encountered.
+   * @throws FileNotFoundException if no file exists
+   * @throws StandbyException
+   * @throws IOException for issues with writing to the audit log
+   *
+   * @return object containing information regarding the file
+   *         or null if file not found
+   */
+  QuotaUsage getQuotaUsage(final String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+    readLock();
+    boolean success = true;
+    try {
+      checkOperation(OperationCategory.READ);
+      return FSDirStatAndListingOp.getQuotaUsage(dir, src);
+    } catch (AccessControlException ace) {
+      success = false;
+      throw ace;
+    } finally {
+      readUnlock();
+      logAuditEvent(success, "quotaUsage", src);
+    }
+  }
+
+  /**
    * Set the namespace quota and storage space quota for a directory.
    * See {@link ClientProtocol#setQuota(String, long, long, StorageType)} for the
    * contract.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33087668/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 7a5e99b..0c8554d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.ha.HAServiceStatus;
 import org.apache.hadoop.ha.HealthCheckFailedException;
 import org.apache.hadoop.ha.ServiceFailedException;
@@ -1259,6 +1260,12 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
+  public QuotaUsage getQuotaUsage(String path) throws IOException {
+    checkNNStartup();
+    return namesystem.getQuotaUsage(path);
+  }
+
+  @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
                        StorageType type)
       throws IOException {