You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/05/19 17:13:44 UTC

[44/50] [abbrv] hbase git commit: HBASE-17955 Various reviewboard improvements to space quota work

HBASE-17955 Various reviewboard improvements to space quota work

Most notable change is to cache SpaceViolationPolicyEnforcement objects
in the write path. When a table has no quota or there is not SpaceQuotaSnapshot
for that table (yet), we want to avoid creating lots of
SpaceViolationPolicyEnforcement instances, caching one instance
instead. This will help reduce GC pressure.


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

Branch: refs/heads/HBASE-16961
Commit: 97dcaffa9831c3a49a70457e57390667a41fe292
Parents: a69b68e
Author: Josh Elser <el...@apache.org>
Authored: Tue Apr 18 16:43:40 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri May 19 12:28:50 2017 -0400

----------------------------------------------------------------------
 .../hbase/quotas/QuotaSettingsFactory.java      |  10 +-
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |   7 +-
 .../hadoop/hbase/quotas/SpaceLimitSettings.java |  26 +-
 .../hadoop/hbase/quotas/SpaceQuotaSnapshot.java |  34 +-
 .../hbase/quotas/SpaceViolationPolicy.java      |   5 +-
 .../hbase/master/MetricsMasterQuotaSource.java  |  13 +-
 .../MetricsRegionServerQuotaSource.java         |  10 +-
 .../MetricsMasterQuotaSourceFactoryImpl.java    |   2 +-
 .../master/MetricsMasterQuotaSourceImpl.java    |  10 +-
 .../shaded/protobuf/generated/QuotaProtos.java  | 637 ++++++++++---------
 .../generated/RegionServerStatusProtos.java     | 340 +++++-----
 .../src/main/protobuf/Quota.proto               |   8 +-
 .../src/main/protobuf/RegionServerStatus.proto  |   4 +-
 .../hbase/protobuf/generated/QuotaProtos.java   | 463 +++++++-------
 hbase-protocol/src/main/protobuf/Quota.proto    |   8 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   4 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   9 +-
 .../hadoop/hbase/master/MetricsMaster.java      |  13 +-
 .../hbase/master/MetricsMasterWrapperImpl.java  |   4 +-
 .../hbase/quotas/ActivePolicyEnforcement.java   |  54 +-
 .../quotas/FileSystemUtilizationChore.java      |   4 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java |   8 +-
 .../hbase/quotas/MasterSpaceQuotaObserver.java  |   4 +-
 .../quotas/NamespaceQuotaSnapshotStore.java     |   2 +-
 .../hadoop/hbase/quotas/QuotaObserverChore.java |  62 +-
 .../quotas/RegionServerSpaceQuotaManager.java   |  16 +-
 .../hbase/quotas/SpaceLimitingException.java    |   6 +-
 .../hbase/quotas/SpaceQuotaRefresherChore.java  |   2 +-
 .../SpaceViolationPolicyEnforcementFactory.java |  20 +-
 .../hbase/quotas/TableQuotaSnapshotStore.java   |   2 +-
 .../AbstractViolationPolicyEnforcement.java     |  45 +-
 ...LoadVerifyingViolationPolicyEnforcement.java |  50 --
 .../DefaultViolationPolicyEnforcement.java      |  90 +++
 .../DisableTableViolationPolicyEnforcement.java |   2 +-
 ...ssingSnapshotViolationPolicyEnforcement.java |  63 ++
 .../NoInsertsViolationPolicyEnforcement.java    |   2 +-
 .../NoWritesViolationPolicyEnforcement.java     |   2 +-
 .../hbase/regionserver/CompactSplitThread.java  |   2 +-
 .../hbase/regionserver/HRegionServer.java       |   4 +-
 .../hbase/regionserver/RSRpcServices.java       |   9 +-
 .../resources/hbase-webapps/master/table.jsp    |   8 +-
 .../hbase/quotas/SpaceQuotaHelperForTests.java  |  66 +-
 .../quotas/TestActivePolicyEnforcement.java     |  62 +-
 .../quotas/TestMasterSpaceQuotaObserver.java    |  28 +-
 .../TestQuotaObserverChoreRegionReports.java    |   6 +-
 .../TestQuotaObserverChoreWithMiniCluster.java  |  31 +-
 .../hbase/quotas/TestQuotaStatusRPCs.java       |  15 +-
 .../TestRegionServerSpaceQuotaManager.java      |   4 +-
 .../hadoop/hbase/quotas/TestSpaceQuotas.java    |  30 +-
 .../TestTableSpaceQuotaViolationNotifier.java   |   8 +-
 ...kLoadCheckingViolationPolicyEnforcement.java |   2 +-
 .../TestRegionServerRegionSpaceUseReport.java   |   4 +-
 52 files changed, 1280 insertions(+), 1040 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
index 184277d..a99235f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
@@ -127,11 +127,11 @@ public class QuotaSettingsFactory {
   }
 
   static QuotaSettings fromSpace(TableName table, String namespace, SpaceQuota protoQuota) {
-    if ((null == table && null == namespace) || (null != table && null != namespace)) {
+    if ((table == null && namespace == null) || (table != null && namespace != null)) {
       throw new IllegalArgumentException(
           "Can only construct SpaceLimitSettings for a table or namespace.");
     }
-    if (null != table) {
+    if (table != null) {
       return SpaceLimitSettings.fromSpaceQuota(table, protoQuota);
     } else {
       // namespace must be non-null
@@ -323,7 +323,7 @@ public class QuotaSettingsFactory {
    * @return A {@link QuotaSettings} object.
    */
   public static QuotaSettings removeTableSpaceLimit(TableName tableName) {
-    return new SpaceLimitSettings(tableName, true);
+    return new SpaceLimitSettings(tableName);
   }
 
   /**
@@ -343,12 +343,12 @@ public class QuotaSettingsFactory {
 
   /**
    * Creates a {@link QuotaSettings} object to remove the FileSystem space quota for the given
-	 * namespace.
+   * namespace.
    *
    * @param namespace The namespace to remove the quota on.
    * @return A {@link QuotaSettings} object.
    */
   public static QuotaSettings removeNamespaceSpaceLimit(String namespace) {
-    return new SpaceLimitSettings(namespace, true);
+    return new SpaceLimitSettings(namespace);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
index c008702..ec480c4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
@@ -251,12 +251,12 @@ public class QuotaTableUtil {
   public static void extractQuotaSnapshot(
       Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {
     byte[] row = Objects.requireNonNull(result).getRow();
-    if (null == row) {
+    if (row == null) {
       throw new IllegalArgumentException("Provided result had a null row");
     }
     final TableName targetTableName = getTableFromRowKey(row);
     Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
-    if (null == c) {
+    if (c == null) {
       throw new IllegalArgumentException("Result did not contain the expected column "
           + QUOTA_POLICY_COLUMN + ", " + result.toString());
     }
@@ -398,7 +398,8 @@ public class QuotaTableUtil {
   }
 
   /**
-   * Creates a {@link Put} to enable the given <code>policy</code> on the <code>table</code>.
+   * Creates a {@link Put} to store the given {@code snapshot} for the given {@code tableName} in
+   * the quota table.
    */
   public static Put createPutSpaceSnapshot(TableName tableName, SpaceQuotaSnapshot snapshot) {
     Put p = new Put(getTableRowKey(tableName));

http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
index 8ff7623..e6d2db8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceLimitR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
 
 /**
- * A {@link QuotaSettings} implementation for implementing filesystem-use quotas.
+ * A {@link QuotaSettings} implementation for configuring filesystem-use quotas.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -38,33 +38,33 @@ class SpaceLimitSettings extends QuotaSettings {
 
   SpaceLimitSettings(TableName tableName, long sizeLimit, SpaceViolationPolicy violationPolicy) {
     super(null, Objects.requireNonNull(tableName), null);
-    if (0L > sizeLimit) {
+    if (sizeLimit < 0L) {
       throw new IllegalArgumentException("Size limit must be a non-negative value.");
     }
     proto = buildProtoAddQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
   }
 
-  SpaceLimitSettings(TableName tableName, boolean remove) {
+  /**
+   * Constructs a {@code SpaceLimitSettings} to remove a space quota on the given {@code tableName}.
+   */
+  SpaceLimitSettings(TableName tableName) {
     super(null, Objects.requireNonNull(tableName), null);
-    if (!remove) {
-      throw new IllegalArgumentException("A value of 'false' for removing a quota makes no sense");
-    }
     proto = buildProtoRemoveQuota();
   }
 
   SpaceLimitSettings(String namespace, long sizeLimit, SpaceViolationPolicy violationPolicy) {
     super(null, null, Objects.requireNonNull(namespace));
-    if (0L > sizeLimit) {
+    if (sizeLimit < 0L) {
       throw new IllegalArgumentException("Size limit must be a non-negative value.");
     }
     proto = buildProtoAddQuota(sizeLimit, Objects.requireNonNull(violationPolicy));
   }
 
-  SpaceLimitSettings(String namespace, boolean remove) {
+  /**
+   * Constructs a {@code SpaceLimitSettings} to remove a space quota on the given {@code namespace}.
+   */
+  SpaceLimitSettings(String namespace) {
     super(null, null, Objects.requireNonNull(namespace));
-    if (!remove) {
-      throw new IllegalArgumentException("A value of 'false' for removing a quota makes no sense");
-    }
     proto = buildProtoRemoveQuota();
   }
 
@@ -183,10 +183,10 @@ class SpaceLimitSettings extends QuotaSettings {
   public String toString() {
     StringBuilder sb = new StringBuilder();
     sb.append("TYPE => SPACE");
-    if (null != getTableName()) {
+    if (getTableName() != null) {
       sb.append(", TABLE => ").append(getTableName());
     }
-    if (null != getNamespace()) {
+    if (getNamespace() != null) {
       sb.append(", NAMESPACE => ").append(getNamespace());
     }
     if (proto.getQuota().getRemove()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
index 86d64d3..5c64880 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaSnapshot.java
@@ -36,7 +36,8 @@ public class SpaceQuotaSnapshot {
 
   /**
    * Encapsulates the state of a quota on a table. The quota may or may not be in violation.
-   * If it is in violation, there will be a non-null violation policy.
+   * If the quota is not in violation, the violation may be null. If the quota is in violation,
+   * there is guaranteed to be a non-null violation policy.
    */
   @InterfaceAudience.Private
   public static class SpaceQuotaStatus {
@@ -44,9 +45,17 @@ public class SpaceQuotaSnapshot {
     final SpaceViolationPolicy policy;
     final boolean inViolation;
 
+    /**
+     * Constructs a {@code SpaceQuotaSnapshot} which is in violation of the provided {@code policy}.
+     *
+     * Use {@link #notInViolation()} to obtain an instance of this class for the cases when the
+     * quota is not in violation.
+     *
+     * @param policy The non-null policy being violated.
+     */
     public SpaceQuotaStatus(SpaceViolationPolicy policy) {
-      this.policy = Objects.requireNonNull(policy);
-      this.inViolation = true;
+      // If the caller is instantiating a status, the policy must be non-null
+      this (Objects.requireNonNull(policy), true);
     }
 
     private SpaceQuotaStatus(SpaceViolationPolicy policy, boolean inViolation) {
@@ -55,16 +64,15 @@ public class SpaceQuotaSnapshot {
     }
 
     /**
-     * The violation policy which may be null. Is guaranteed to be non-null if
-     * {@link #isInViolation()} is <code>true</code>, and <code>false</code>
-     * otherwise.
+     * Returns the violation policy, which may be null. It is guaranteed to be non-null if
+     * {@link #isInViolation()} is {@code true}, but may be null otherwise.
      */
     public SpaceViolationPolicy getPolicy() {
       return policy;
     }
 
     /**
-     * <code>true</code> if the quota is being violated, <code>false</code> otherwise.
+     * @return {@code true} if the quota is being violated, {@code false} otherwise.
      */
     public boolean isInViolation() {
       return inViolation;
@@ -104,14 +112,14 @@ public class SpaceQuotaSnapshot {
       QuotaProtos.SpaceQuotaStatus.Builder builder = QuotaProtos.SpaceQuotaStatus.newBuilder();
       builder.setInViolation(status.inViolation);
       if (status.isInViolation()) {
-        builder.setPolicy(ProtobufUtil.toProtoViolationPolicy(status.getPolicy()));
+        builder.setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(status.getPolicy()));
       }
       return builder.build();
     }
 
     public static SpaceQuotaStatus toStatus(QuotaProtos.SpaceQuotaStatus proto) {
       if (proto.getInViolation()) {
-        return new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(proto.getPolicy()));
+        return new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(proto.getViolationPolicy()));
       } else {
         return NOT_IN_VIOLATION;
       }
@@ -173,14 +181,14 @@ public class SpaceQuotaSnapshot {
 
   // ProtobufUtil is in hbase-client, and this doesn't need to be public.
   public static SpaceQuotaSnapshot toSpaceQuotaSnapshot(QuotaProtos.SpaceQuotaSnapshot proto) {
-    return new SpaceQuotaSnapshot(SpaceQuotaStatus.toStatus(proto.getStatus()),
-        proto.getUsage(), proto.getLimit());
+    return new SpaceQuotaSnapshot(SpaceQuotaStatus.toStatus(proto.getQuotaStatus()),
+        proto.getQuotaUsage(), proto.getQuotaLimit());
   }
 
   public static QuotaProtos.SpaceQuotaSnapshot toProtoSnapshot(SpaceQuotaSnapshot snapshot) {
     return QuotaProtos.SpaceQuotaSnapshot.newBuilder()
-        .setStatus(SpaceQuotaStatus.toProto(snapshot.getQuotaStatus()))
-        .setUsage(snapshot.getUsage()).setLimit(snapshot.getLimit()).build();
+        .setQuotaStatus(SpaceQuotaStatus.toProto(snapshot.getQuotaStatus()))
+        .setQuotaUsage(snapshot.getUsage()).setQuotaLimit(snapshot.getLimit()).build();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java
index c63acb0..34d2542 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceViolationPolicy.java
@@ -21,6 +21,9 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Enumeration that represents the action HBase will take when a space quota is violated.
+ *
+ * The target for a violation policy is either an HBase table or namespace. In the case of a
+ * namespace, it is treated as a collection of tables (all tables are subject to the same policy).
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
@@ -40,5 +43,5 @@ public enum SpaceViolationPolicy {
   /**
    * Disallows any updates (but allows deletes and compactions) on the table(s).
    */
-  NO_INSERTS,
+  NO_INSERTS;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
index be579be..7625f84 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSource.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.master;
 import org.apache.hadoop.hbase.metrics.BaseSource;
 
 /**
- * A collection of exposed metrics for HBase quotas from the HBase Master.
+ * A collection of exposed metrics for space quotas from the HBase Master.
  */
 public interface MetricsMasterQuotaSource extends BaseSource {
 
@@ -46,30 +46,41 @@ public interface MetricsMasterQuotaSource extends BaseSource {
 
   /**
    * Updates the metric tracking the number of space quotas defined in the system.
+   *
+   * @param numSpaceQuotas The number of space quotas defined
    */
   void updateNumSpaceQuotas(long numSpaceQuotas);
 
   /**
    * Updates the metric tracking the number of tables the master has computed to be in
    * violation of their space quota.
+   *
+   * @param numTablesInViolation The number of tables violating a space quota
    */
   void updateNumTablesInSpaceQuotaViolation(long numTablesInViolation);
 
   /**
    * Updates the metric tracking the number of namespaces the master has computed to be in
    * violation of their space quota.
+   *
+   * @param numNamespacesInViolation The number of namespaces violating a space quota
    */
   void updateNumNamespacesInSpaceQuotaViolation(long numNamespacesInViolation);
 
   /**
    * Updates the metric tracking the number of region size reports the master is currently
    * retaining in memory.
+   *
+   * @param numCurrentRegionSizeReports The number of region size reports the master is holding in
+   *    memory
    */
   void updateNumCurrentSpaceQuotaRegionSizeReports(long numCurrentRegionSizeReports);
 
   /**
    * Updates the metric tracking the amount of time taken by the {@code QuotaObserverChore}
    * which runs periodically.
+   *
+   * @param time The execution time of the chore in milliseconds
    */
   void incrementSpaceQuotaObserverChoreTime(long time);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java
index 12fa66d..3175aad 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerQuotaSource.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
 import org.apache.hadoop.hbase.metrics.BaseSource;
 
 /**
- * A collection of exposed metrics for HBase quotas from an HBase RegionServer.
+ * A collection of exposed metrics for space quotas from an HBase RegionServer.
  */
 public interface MetricsRegionServerQuotaSource extends BaseSource {
 
@@ -37,18 +37,24 @@ public interface MetricsRegionServerQuotaSource extends BaseSource {
   /**
    * Updates the metric tracking how many tables this RegionServer has received
    * {@code SpaceQuotaSnapshot}s for.
+   *
+   * @param numSnapshots The number of {@code SpaceQuotaSnapshot}s received from the Master.
    */
   void updateNumTableSpaceQuotaSnapshots(long numSnapshots);
 
   /**
    * Updates the metric tracking how much time was spent scanning the filesystem to compute
    * the size of each region hosted by this RegionServer.
+   *
+   * @param time The execution time of the chore in milliseconds.
    */
   void incrementSpaceQuotaFileSystemScannerChoreTime(long time);
 
   /**
    * Updates the metric tracking how much time was spent updating the RegionServer with the
-   * lastest information on space quotas from the {@code hbase:quota} table.
+   * latest information on space quotas from the {@code hbase:quota} table.
+   *
+   * @param time The execution time of the chore in milliseconds.
    */
   void incrementSpaceQuotaRefresherChoreTime(long time);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java
index 96a57c4..7655160 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceFactoryImpl.java
@@ -28,7 +28,7 @@ public class MetricsMasterQuotaSourceFactoryImpl implements MetricsMasterQuotaSo
 
   @Override
   public synchronized MetricsMasterQuotaSource create(MetricsMasterWrapper masterWrapper) {
-    if (null == quotaSource) {
+    if (quotaSource == null) {
       quotaSource = new MetricsMasterQuotaSourceImpl(masterWrapper);
     }
     return quotaSource;

http://git-wip-us.apache.org/repos/asf/hbase/blob/97dcaffa/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
index 064f7fc..342aa1e 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterQuotaSourceImpl.java
@@ -27,6 +27,10 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 
+/**
+ * Implementation of {@link MetricsMasterQuotaSource} which writes the values passed in via the
+ * interface to the metrics backend.
+ */
 @InterfaceAudience.Private
 public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements MetricsMasterQuotaSource {
   private final MetricsMasterWrapper wrapper;
@@ -87,11 +91,11 @@ public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements Metr
   @Override
   public void getMetrics(MetricsCollector metricsCollector, boolean all) {
     MetricsRecordBuilder record = metricsCollector.addRecord(metricsRegistry.info());
-    if (null != wrapper) {
+    if (wrapper != null) {
       // Summarize the tables
       Map<String,Entry<Long,Long>> tableUsages = wrapper.getTableSpaceUtilization();
       String tableSummary = "[]";
-      if (null != tableUsages && !tableUsages.isEmpty()) {
+      if (tableUsages != null && !tableUsages.isEmpty()) {
         tableSummary = generateJsonQuotaSummary(tableUsages.entrySet(), "table");
       }
       record.tag(Interns.info(TABLE_QUOTA_USAGE_NAME, TABLE_QUOTA_USAGE_DESC), tableSummary);
@@ -99,7 +103,7 @@ public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements Metr
       // Summarize the namespaces
       String nsSummary = "[]";
       Map<String,Entry<Long,Long>> namespaceUsages = wrapper.getNamespaceSpaceUtilization();
-      if (null != namespaceUsages && !namespaceUsages.isEmpty()) {
+      if (namespaceUsages != null && !namespaceUsages.isEmpty()) {
         nsSummary = generateJsonQuotaSummary(namespaceUsages.entrySet(), "namespace");
       }
       record.tag(Interns.info(NS_QUOTA_USAGE_NAME, NS_QUOTA_USAGE_DESC), nsSummary);