Browse Source

HDFS-12719. Ozone: Fix checkstyle, javac, whitespace issues in HDFS-7240 branch. Contributed by Mukul Kumar Singh.

Anu Engineer 7 years ago
parent
commit
740a06cdd7
30 changed files with 136 additions and 97 deletions
  1. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/OzoneConfiguration.java
  2. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/ServiceRuntimeInfo.java
  3. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
  4. 8 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/ContainerInfo.java
  5. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh
  6. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-ozone.sh
  7. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh
  8. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-ozone.sh
  9. 24 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/jscsiHelper/cache/impl/CBlockLocalCache.java
  10. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
  11. 12 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java
  12. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/placement/metrics/LongMetric.java
  13. 13 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/placement/metrics/SCMNodeStat.java
  14. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java
  15. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/StorageContainerDatanodeProtocol.proto
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/TestBufferManager.java
  17. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/TestCBlockReadWrite.java
  18. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/TestLocalBlockCache.java
  19. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
  20. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java
  21. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java
  22. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
  23. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java
  24. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java
  25. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestVolumeStructs.java
  26. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
  27. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
  28. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
  29. 1 1
      hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/Constants.java
  30. 1 1
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/OzoneConfiguration.java

@@ -148,8 +148,8 @@ public class OzoneConfiguration extends Configuration {
 
     @Override
     public boolean equals(Object obj) {
-      return (obj instanceof Property) && (((Property) obj).getName()).equals
-          (this.getName());
+      return (obj instanceof Property) && (((Property) obj).getName())
+          .equals(this.getName());
     }
   }
 }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/ServiceRuntimeInfo.java

@@ -41,14 +41,14 @@ public interface ServiceRuntimeInfo {
   String getVersion();
 
   /**
-   * Get the version of software running on the Namenode
+   * Get the version of software running on the Namenode.
    *
    * @return a string representing the version
    */
   String getSoftwareVersion();
 
   /**
-   * Get the compilation information which contains date, user and branch
+   * Get the compilation information which contains date, user and branch.
    *
    * @return the compilation information, as a JSON string.
    */

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java

@@ -332,8 +332,8 @@ public class ChunkGroupOutputStream extends OutputStream {
       return this;
     }
 
-    public Builder setType(ReplicationType type) {
-      this.type = type;
+    public Builder setType(ReplicationType replicationType) {
+      this.type = replicationType;
       return this;
     }
 

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/ContainerInfo.java

@@ -249,8 +249,8 @@ public class ContainerInfo
       return this;
     }
 
-    public Builder setPipeline(Pipeline pipeline) {
-      this.pipeline = pipeline;
+    public Builder setPipeline(Pipeline containerPipeline) {
+      this.pipeline = containerPipeline;
       return this;
     }
 
@@ -269,18 +269,18 @@ public class ContainerInfo
       return this;
     }
 
-    public Builder setStateEnterTime(long stateEnterTime) {
-      this.stateEnterTime = stateEnterTime;
+    public Builder setStateEnterTime(long time) {
+      this.stateEnterTime = time;
       return this;
     }
 
-    public Builder setOwner(OzoneProtos.Owner owner) {
-      this.owner = owner;
+    public Builder setOwner(OzoneProtos.Owner containerOwner) {
+      this.owner = containerOwner;
       return this;
     }
 
-    public Builder setContainerName(String containerName) {
-      this.containerName = containerName;
+    public Builder setContainerName(String container) {
+      this.containerName = container;
       return this;
     }
 

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh

@@ -17,6 +17,10 @@
 
 MYNAME="${BASH_SOURCE-$0}"
 
+## @description  usage info
+## @audience     private
+## @stability    evolving
+## @replaceable  no
 function hadoop_usage
 {
   hadoop_add_option "--buildpaths" "attempt to add class files from build tree"

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-ozone.sh

@@ -17,7 +17,10 @@
 
 # Start hadoop hdfs and ozone daemons.
 # Run this on master node.
-
+## @description  usage info
+## @audience     private
+## @stability    evolving
+## @replaceable  no
 function hadoop_usage
 {
   echo "Usage: start-ozone.sh"

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh

@@ -17,6 +17,10 @@
 
 MYNAME="${BASH_SOURCE-$0}"
 
+## @description  usage info
+## @audience     private
+## @stability    evolving
+## @replaceable  no
 function hadoop_usage
 {
   hadoop_add_option "--buildpaths" "attempt to add class files from build tree"

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-ozone.sh

@@ -17,7 +17,10 @@
 
 # Stop hdfs and ozone daemons.
 # Run this on master node.
-
+## @description  usage info
+## @audience     private
+## @stability    evolving
+## @replaceable  no
 function hadoop_usage
 {
   echo "Usage: stop-ozone.sh"

+ 24 - 24
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/jscsiHelper/cache/impl/CBlockLocalCache.java

@@ -434,88 +434,88 @@ public class CBlockLocalCache implements CacheModule {
     /**
      * Sets the Config to be used by this cache.
      *
-     * @param configuration - Config
+     * @param conf - Config
      * @return Builder
      */
-    public Builder setConfiguration(Configuration configuration) {
-      this.configuration = configuration;
+    public Builder setConfiguration(Configuration conf) {
+      this.configuration = conf;
       return this;
     }
 
     /**
      * Sets the user name who is the owner of this volume.
      *
-     * @param userName - name of the owner, please note this is not the current
+     * @param user - name of the owner, please note this is not the current
      * user name.
      * @return - Builder
      */
-    public Builder setUserName(String userName) {
-      this.userName = userName;
+    public Builder setUserName(String user) {
+      this.userName = user;
       return this;
     }
 
     /**
      * Sets the VolumeName.
      *
-     * @param volumeName - Name of the volume
+     * @param volume - Name of the volume
      * @return Builder
      */
-    public Builder setVolumeName(String volumeName) {
-      this.volumeName = volumeName;
+    public Builder setVolumeName(String volume) {
+      this.volumeName = volume;
       return this;
     }
 
     /**
      * Sets the Pipelines that form this volume.
      *
-     * @param pipelines - list of pipelines
+     * @param pipelineList - list of pipelines
      * @return Builder
      */
-    public Builder setPipelines(List<Pipeline> pipelines) {
-      this.pipelines = pipelines;
+    public Builder setPipelines(List<Pipeline> pipelineList) {
+      this.pipelines = pipelineList;
       return this;
     }
 
     /**
      * Sets the Client Manager that manages the communication with containers.
      *
-     * @param clientManager - clientManager.
+     * @param xceiverClientManager - clientManager.
      * @return - Builder
      */
-    public Builder setClientManager(XceiverClientManager clientManager) {
-      this.clientManager = clientManager;
+    public Builder setClientManager(XceiverClientManager xceiverClientManager) {
+      this.clientManager = xceiverClientManager;
       return this;
     }
 
     /**
      * Sets the block size -- Typical sizes are 4KB, 8KB etc.
      *
-     * @param blockSize - BlockSize.
+     * @param size - BlockSize.
      * @return - Builder
      */
-    public Builder setBlockSize(int blockSize) {
-      this.blockSize = blockSize;
+    public Builder setBlockSize(int size) {
+      this.blockSize = size;
       return this;
     }
 
     /**
      * Sets the volumeSize.
      *
-     * @param volumeSize - VolumeSize
+     * @param size - VolumeSize
      * @return - Builder
      */
-    public Builder setVolumeSize(long volumeSize) {
-      this.volumeSize = volumeSize;
+    public Builder setVolumeSize(long size) {
+      this.volumeSize = size;
       return this;
     }
 
     /**
      * Set flusher.
-     * @param flusher - cache Flusher
+     * @param containerCacheFlusher - cache Flusher
      * @return Builder.
      */
-    public Builder setFlusher(ContainerCacheFlusher flusher) {
-      this.flusher = flusher;
+    public Builder setFlusher(ContainerCacheFlusher containerCacheFlusher) {
+      this.flusher = containerCacheFlusher;
       return this;
     }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java

@@ -321,9 +321,9 @@ public class DatanodeStateMachine implements Closeable {
   /**
    * Create a command handler thread.
    *
-   * @param conf
+   * @param config
    */
-  private void initCommandHandlerThread(Configuration conf) {
+  private void initCommandHandlerThread(Configuration config) {
 
     /**
      * Task that periodically checks if we have any outstanding commands.

+ 12 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java

@@ -125,37 +125,37 @@ public final class CommandDispatcher {
     /**
      * Add the OzoneContainer.
      *
-     * @param container - ozone container.
+     * @param ozoneContainer - ozone container.
      * @return Builder
      */
-    public Builder setContainer(OzoneContainer container) {
-      Preconditions.checkNotNull(container);
-      this.container = container;
+    public Builder setContainer(OzoneContainer ozoneContainer) {
+      Preconditions.checkNotNull(ozoneContainer);
+      this.container = ozoneContainer;
       return this;
     }
 
     /**
      * Set the Connection Manager.
      *
-     * @param connectionManager
+     * @param scmConnectionManager
      * @return this
      */
     public Builder setConnectionManager(SCMConnectionManager
-        connectionManager) {
-      Preconditions.checkNotNull(connectionManager);
-      this.connectionManager = connectionManager;
+        scmConnectionManager) {
+      Preconditions.checkNotNull(scmConnectionManager);
+      this.connectionManager = scmConnectionManager;
       return this;
     }
 
     /**
      * Sets the Context.
      *
-     * @param context - StateContext
+     * @param stateContext - StateContext
      * @return this
      */
-    public Builder setContext(StateContext context) {
-      Preconditions.checkNotNull(context);
-      this.context = context;
+    public Builder setContext(StateContext stateContext) {
+      Preconditions.checkNotNull(stateContext);
+      this.context = stateContext;
       return this;
     }
 

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/placement/metrics/LongMetric.java

@@ -92,32 +92,32 @@ public class LongMetric implements DatanodeMetric<Long, Long> {
   /**
    * Sets the value of this metric.
    *
-   * @param value - value of the metric.
+   * @param setValue - value of the metric.
    */
   @Override
-  public void set(Long value) {
-    this.value = value;
+  public void set(Long setValue) {
+    this.value = setValue;
 
   }
 
   /**
    * Adds a value of to the base.
    *
-   * @param value - value
+   * @param addValue - value
    */
   @Override
-  public void add(Long value) {
-    this.value += value;
+  public void add(Long addValue) {
+    this.value += addValue;
   }
 
   /**
    * subtract a value.
    *
-   * @param value value
+   * @param subValue value
    */
   @Override
-  public void subtract(Long value) {
-    this.value -= value;
+  public void subtract(Long subValue) {
+    this.value -= subValue;
   }
 
   /**

+ 13 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/placement/metrics/SCMNodeStat.java

@@ -73,26 +73,26 @@ public class SCMNodeStat implements NodeStat {
   /**
    * Set the capacity, used and remaining space on a datanode.
    *
-   * @param capacity in bytes
-   * @param used in bytes
-   * @param remaining in bytes
+   * @param newCapacity in bytes
+   * @param newUsed in bytes
+   * @param newRemaining in bytes
    */
   @VisibleForTesting
-  public void set(long capacity, long used, long remaining) {
-    Preconditions.checkNotNull(capacity, "Capacity cannot be null");
-    Preconditions.checkNotNull(used, "used cannot be null");
-    Preconditions.checkNotNull(remaining, "remaining cannot be null");
+  public void set(long newCapacity, long newUsed, long newRemaining) {
+    Preconditions.checkNotNull(newCapacity, "Capacity cannot be null");
+    Preconditions.checkNotNull(newUsed, "used cannot be null");
+    Preconditions.checkNotNull(newRemaining, "remaining cannot be null");
 
-    Preconditions.checkArgument(capacity >= 0, "Capacity cannot be " +
+    Preconditions.checkArgument(newCapacity >= 0, "Capacity cannot be " +
         "negative.");
-    Preconditions.checkArgument(used >= 0, "used space cannot be " +
+    Preconditions.checkArgument(newUsed >= 0, "used space cannot be " +
         "negative.");
-    Preconditions.checkArgument(remaining >= 0, "remaining cannot be " +
+    Preconditions.checkArgument(newRemaining >= 0, "remaining cannot be " +
         "negative");
 
-    this.capacity = new LongMetric(capacity);
-    this.scmUsed = new LongMetric(used);
-    this.remaining = new LongMetric(remaining);
+    this.capacity = new LongMetric(newCapacity);
+    this.scmUsed = new LongMetric(newUsed);
+    this.remaining = new LongMetric(newRemaining);
   }
 
   /**

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java

@@ -65,9 +65,9 @@ public interface Keys {
    */
   @PUT
   @Consumes(MediaType.WILDCARD)
-  @ApiOperation(value = "Adds a key to an existing bucket.",notes = "If the "
-      + "object already exists this call will overwrite or add with new version "
-      + "number if the bucket versioning is turned on.")
+  @ApiOperation(value = "Adds a key to an existing bucket.", notes = "If the "
+      + "object already exists this call will overwrite or add with new version"
+      + " number if the bucket versioning is turned on.")
   @ApiImplicitParams({
       @ApiImplicitParam(name = "x-ozone-version", example = "v1", required =
           true, paramType = "header"),

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/StorageContainerDatanodeProtocol.proto

@@ -268,9 +268,9 @@ message ContainerBlocksDeletionACKProto {
  * Here is a simple state diagram that shows how a datanode would boot up and
  * communicate with SCM.
  *
- *           ----------------------- 
+ *           -----------------------
  *          |         Start         |
- *           ---------- ------------ 
+ *           ---------- ------------
  *                     |
  *                     |
  *                     |
@@ -278,19 +278,19 @@ message ContainerBlocksDeletionACKProto {
  *                     |
  *                     |
  *                     |
- *           ----------v------------- 
- *          |   Searching for  SCM    ------------ 
+ *           ----------v-------------
+ *          |   Searching for  SCM    ------------
  *           ---------- -------------             |
  *                     |                          |
  *                     |                          |
- *                     |                ----------v------------- 
+ *                     |                ----------v-------------
  *                     |               | Register if needed     |
- *                     |                ----------- ------------ 
+ *                     |                ----------- ------------
  *                     |                           |
  *                     v                           |
  *            ----------- ----------------         |
- *  ---------   Heartbeat state           <-------- 
- * |          --------^------------------- 
+ *  ---------   Heartbeat state           <--------
+ * |          --------^-------------------
  * |                  |
  * |                  |
  * |                  |
@@ -298,7 +298,7 @@ message ContainerBlocksDeletionACKProto {
  * |                  |
  * |                  |
  * |                  |
- *  ------------------ 
+ *  ------------------
  *
  *
  *

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/TestBufferManager.java

@@ -83,7 +83,7 @@ public class TestBufferManager {
     if (cluster != null) {
       cluster.shutdown();
     }
-    IOUtils.cleanup(null, storageContainerLocationClient, cluster);
+    IOUtils.cleanupWithLogger(null, storageContainerLocationClient, cluster);
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/TestCBlockReadWrite.java

@@ -90,7 +90,7 @@ public class TestCBlockReadWrite {
     if (cluster != null) {
       cluster.shutdown();
     }
-    IOUtils.cleanup(null, storageContainerLocationClient, cluster);
+    IOUtils.cleanupWithLogger(null, storageContainerLocationClient, cluster);
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/TestLocalBlockCache.java

@@ -96,7 +96,7 @@ public class TestLocalBlockCache {
     if (cluster != null) {
       cluster.shutdown();
     }
-    IOUtils.cleanup(null, storageContainerLocationClient, cluster);
+    IOUtils.cleanupWithLogger(null, storageContainerLocationClient, cluster);
   }
 
   /**

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java

@@ -169,10 +169,10 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
   @Override
   public StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto
       sendHeartbeat(DatanodeID datanodeID, SCMNodeReport nodeReport,
-      ReportState reportState) throws IOException {
+      ReportState scmReportState) throws IOException {
     rpcCount.incrementAndGet();
     heartbeatCount.incrementAndGet();
-    this.reportState = reportState;
+    this.reportState = scmReportState;
     sleepIfNeeded();
     List<SCMCommandResponseProto>
         cmdResponses = new LinkedList<>();

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestBucketInfo.java

@@ -29,6 +29,9 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
+/**
+ * Test Ozone Bucket Info operation.
+ */
 public class TestBucketInfo {
   @Test
   public void testBucketInfoJson() throws IOException {

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestErrorCode.java

@@ -25,6 +25,9 @@ import org.junit.Test;
 import static junit.framework.TestCase.assertEquals;
 import static org.apache.hadoop.ozone.web.utils.OzoneUtils.getRequestID;
 
+/**
+ * Test Ozone Error Codes.
+ */
 public class TestErrorCode {
   /**
    * Test Error Generator functions.

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java

@@ -46,6 +46,9 @@ import static java.net.HttpURLConnection.HTTP_CREATED;
 import static org.apache.hadoop.ozone.web.utils.OzoneUtils.getRequestID;
 import static org.junit.Assert.assertEquals;
 
+/**
+ * Test Ozone Access through REST protocol.
+ */
 public class TestOzoneWebAccess {
   /**
    * Set the timeout for every test.

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java

@@ -29,7 +29,9 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-
+/**
+ * Test Ozone Volume Quota.
+ */
 public class TestQuota {
   @Test
   public void testParseQuota() {

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestUtils.java

@@ -29,6 +29,9 @@ import static org.apache.hadoop.ozone.web.utils.OzoneUtils.verifyResourceName;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Test Ozone Utility operations like verifying resource name.
+ */
 public class TestUtils {
 
   /**

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestVolumeStructs.java

@@ -28,6 +28,9 @@ import java.io.IOException;
 
 import static org.junit.Assert.assertEquals;
 
+/**
+ * Test Ozone Volume info structure.
+ */
 public class TestVolumeStructs {
 
   @Test

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java

@@ -44,7 +44,9 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-
+/**
+ * Test Ozone Bucket Lifecycle.
+ */
 public class TestBuckets {
   /**
    * Set the timeout for every test.

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java

@@ -74,6 +74,9 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Test Ozone Key Lifecycle.
+ */
 public class TestKeys {
   /**
    * Set the timeout for every test.

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java

@@ -58,6 +58,9 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+/**
+ * Test Ozone Volumes Lifecycle.
+ */
 public class TestVolume {
   private static MiniOzoneCluster cluster = null;
   private static OzoneRestClient ozoneRestClient = null;

+ 1 - 1
hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/Constants.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.fs.ozone;
 /**
  * Constants for Ozone FileSystem implementation.
  */
-public class Constants {
+public final class Constants {
 
   public static final String OZONE_URI_SCHEME = "o3";
 

+ 1 - 1
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java

@@ -48,7 +48,7 @@ class OzoneContract extends AbstractFSContract {
   private static StorageHandler storageHandler;
   private static final String CONTRACT_XML = "contract/ozone.xml";
 
-  public OzoneContract(Configuration conf) {
+  OzoneContract(Configuration conf) {
     super(conf);
     //insert the base features
     addConfResource(CONTRACT_XML);