Browse Source

HDDS-965. Ozone: checkstyle improvements and code quality scripts. Contributed by Elek, Marton.

Márton Elek 6 years ago
parent
commit
32cf0412ce
94 changed files with 903 additions and 525 deletions
  1. 1 7
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
  2. 1 0
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
  3. 17 20
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
  4. 1 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/HddsVersionProvider.java
  5. 3 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/HddsConfServlet.java
  6. 1 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java
  7. 1 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/UnknownPipelineStateException.java
  8. 0 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
  9. 29 95
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java
  10. 11 17
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStoreBuilder.java
  11. 1 0
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
  12. 2 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
  13. 9 9
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java
  14. 148 97
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
  15. 1 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfo.java
  16. 8 5
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
  17. 4 4
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java
  18. 0 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java
  19. 10 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteBlockCommandStatus.java
  20. 9 13
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
  21. 2 2
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
  22. 10 2
      hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventWatcher.java
  23. 3 0
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/PendingDeleteHandler.java
  24. 1 3
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
  25. 2 0
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ObjectsMap.java
  26. 5 3
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2PipelineMap.java
  27. 1 0
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
  28. 5 3
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java
  29. 2 2
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java
  30. 3 0
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java
  31. 3 0
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java
  32. 2 2
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java
  33. 3 0
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeReportHandler.java
  34. 105 0
      hadoop-ozone/Jenkinsfile
  35. 2 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
  36. 1 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java
  37. 4 2
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
  38. 1 1
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
  39. 4 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
  40. 20 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/hdds/protocol/package-info.java
  41. 2 4
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
  42. 2 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/freon/OzoneGetConf.java
  43. 2 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmBucketInfoCodec.java
  44. 2 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmKeyInfoCodec.java
  45. 1 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmMultipartKeyInfoCodec.java
  46. 2 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmVolumeArgsCodec.java
  47. 2 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/VolumeListCodec.java
  48. 3 4
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
  49. 8 9
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
  50. 1 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
  51. 1 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
  52. 33 104
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/OzoneVersionInfo.java
  53. 22 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/package-info.java
  54. 1 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeArgs.java
  55. 6 5
      hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/acl/TestOzoneObjInfo.java
  56. 18 0
      hadoop-ozone/dev-support/checks/acceptance.sh
  57. 22 0
      hadoop-ozone/dev-support/checks/author.sh
  58. 18 0
      hadoop-ozone/dev-support/checks/build.sh
  59. 23 0
      hadoop-ozone/dev-support/checks/checkstyle.sh
  60. 35 0
      hadoop-ozone/dev-support/checks/findbugs.sh
  61. 24 0
      hadoop-ozone/dev-support/checks/isolation.sh
  62. 24 0
      hadoop-ozone/dev-support/checks/rat.sh
  63. 24 0
      hadoop-ozone/dev-support/checks/unit.sh
  64. 66 0
      hadoop-ozone/dev-support/docker/Dockerfile
  65. 4 3
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestNode2PipelineMap.java
  66. 3 3
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
  67. 6 6
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
  68. 2 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java
  69. 7 2
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
  70. 2 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
  71. 1 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneDatanodeShell.java
  72. 1 0
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
  73. 5 4
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java
  74. 2 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
  75. 22 21
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
  76. 1 0
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
  77. 40 38
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
  78. 1 0
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
  79. 1 0
      hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/Bucket.java
  80. 1 0
      hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
  81. 1 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManager.java
  82. 1 2
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStorage.java
  83. 6 7
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
  84. 1 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManager.java
  85. 2 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisHelper.java
  86. 1 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
  87. 1 1
      hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/SignedChunksInputStream.java
  88. 1 0
      hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
  89. 0 1
      hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectMultiDelete.java
  90. 1 1
      hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/audit/parser/common/DatabaseHelper.java
  91. 0 1
      hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
  92. 3 0
      hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkMetadataStoreReads.java
  93. 3 1
      hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkMetadataStoreWrites.java
  94. 5 1
      hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkRocksDbStore.java

+ 1 - 7
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java

@@ -190,12 +190,6 @@ public final class XceiverClientRatis extends XceiverClientSpi {
     return minIndex.isPresent() ? minIndex.getAsLong() : 0;
     return minIndex.isPresent() ? minIndex.getAsLong() : 0;
   }
   }
 
 
-  private void getFailedServer(
-      Collection<RaftProtos.CommitInfoProto> commitInfos) {
-    for (RaftProtos.CommitInfoProto proto : commitInfos) {
-
-    }
-  }
 
 
   @Override
   @Override
   public long watchForCommit(long index, long timeout)
   public long watchForCommit(long index, long timeout)
@@ -217,7 +211,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
         .sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED);
         .sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED);
     RaftClientReply reply;
     RaftClientReply reply;
     try {
     try {
-      reply = replyFuture.get(timeout, TimeUnit.MILLISECONDS);
+      replyFuture.get(timeout, TimeUnit.MILLISECONDS);
     } catch (TimeoutException toe) {
     } catch (TimeoutException toe) {
       LOG.warn("3 way commit failed ", toe);
       LOG.warn("3 way commit failed ", toe);
 
 

+ 1 - 0
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java

@@ -122,6 +122,7 @@ public class BlockOutputStream extends OutputStream {
    * @param watchTimeout          watch timeout
    * @param watchTimeout          watch timeout
    * @param checksum              checksum
    * @param checksum              checksum
    */
    */
+  @SuppressWarnings("parameternumber")
   public BlockOutputStream(BlockID blockID, String key,
   public BlockOutputStream(BlockID blockID, String key,
       XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
       XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
       String traceID, int chunkSize, long streamBufferFlushSize,
       String traceID, int chunkSize, long streamBufferFlushSize,

+ 17 - 20
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java

@@ -18,22 +18,6 @@
 
 
 package org.apache.hadoop.hdds;
 package org.apache.hadoop.hdds;
 
 
-import com.google.common.base.Strings;
-import com.google.common.net.HostAndPort;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.metrics2.util.MBeans;
-import org.apache.hadoop.net.DNS;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import javax.management.ObjectName;
 import javax.management.ObjectName;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Method;
@@ -47,13 +31,26 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Optional;
 import java.util.TimeZone;
 import java.util.TimeZone;
 
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys
-    .DFS_DATANODE_DNS_INTERFACE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys
-    .DFS_DATANODE_DNS_NAMESERVER_KEY;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.net.NetUtils;
+
+import com.google.common.base.Strings;
+import com.google.common.net.HostAndPort;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED_DEFAULT;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
 /**
  * HDDS specific stateless utility functions.
  * HDDS specific stateless utility functions.

+ 1 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/HddsVersionProvider.java

@@ -28,7 +28,7 @@ public class HddsVersionProvider implements IVersionProvider {
   @Override
   @Override
   public String[] getVersion() throws Exception {
   public String[] getVersion() throws Exception {
     String[] result = new String[] {
     String[] result = new String[] {
-        HddsVersionInfo.getBuildVersion()
+        HddsVersionInfo.HDDS_VERSION_INFO.getBuildVersion()
     };
     };
     return result;
     return result;
   }
   }

+ 3 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/HddsConfServlet.java

@@ -139,6 +139,9 @@ public class HddsConfServlet extends HttpServlet {
     }
     }
   }
   }
 
 
+  /**
+   * Exception for signal bad content type.
+   */
   public static class BadFormatException extends Exception {
   public static class BadFormatException extends Exception {
 
 
     private static final long serialVersionUID = 1L;
     private static final long serialVersionUID = 1L;

+ 1 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java

@@ -81,6 +81,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
   @JsonIgnore
   @JsonIgnore
   private byte[] data;
   private byte[] data;
 
 
+  @SuppressWarnings("parameternumber")
   ContainerInfo(
   ContainerInfo(
       long containerID,
       long containerID,
       HddsProtos.LifeCycleState state,
       HddsProtos.LifeCycleState state,

+ 1 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/UnknownPipelineStateException.java

@@ -18,7 +18,7 @@
 
 
 package org.apache.hadoop.hdds.scm.pipeline;
 package org.apache.hadoop.hdds.scm.pipeline;
 
 
-    import java.io.IOException;
+import java.io.IOException;
 
 
 /**
 /**
  * Signals that a pipeline state is not recognized.
  * Signals that a pipeline state is not recognized.

+ 0 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java

@@ -67,7 +67,6 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
-import java.util.stream.Collectors;
 
 
 /**
 /**
  * This class is the client-side translator to translate the requests made on
  * This class is the client-side translator to translate the requests made on

+ 29 - 95
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java

@@ -36,9 +36,13 @@ import java.util.Properties;
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class HddsVersionInfo {
 public class HddsVersionInfo {
+
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
       HddsVersionInfo.class);
       HddsVersionInfo.class);
 
 
+  public static final HddsVersionInfo HDDS_VERSION_INFO =
+      new HddsVersionInfo("hdds");
+
   private Properties info;
   private Properties info;
 
 
   protected HddsVersionInfo(String component) {
   protected HddsVersionInfo(String component) {
@@ -46,7 +50,8 @@ public class HddsVersionInfo {
     String versionInfoFile = component + "-version-info.properties";
     String versionInfoFile = component + "-version-info.properties";
     InputStream is = null;
     InputStream is = null;
     try {
     try {
-      is = ThreadUtil.getResourceAsStream(HddsVersionInfo.class.getClassLoader(),
+      is = ThreadUtil.getResourceAsStream(
+          HddsVersionInfo.class.getClassLoader(),
           versionInfoFile);
           versionInfoFile);
       info.load(is);
       info.load(is);
     } catch (IOException ex) {
     } catch (IOException ex) {
@@ -57,127 +62,56 @@ public class HddsVersionInfo {
     }
     }
   }
   }
 
 
-  protected String _getVersion() {
+  protected String getVersion() {
     return info.getProperty("version", "Unknown");
     return info.getProperty("version", "Unknown");
   }
   }
 
 
-  protected String _getRevision() {
+  protected String getRevision() {
     return info.getProperty("revision", "Unknown");
     return info.getProperty("revision", "Unknown");
   }
   }
 
 
-  protected String _getBranch() {
+  protected String getBranch() {
     return info.getProperty("branch", "Unknown");
     return info.getProperty("branch", "Unknown");
   }
   }
 
 
-  protected String _getDate() {
+  protected String getDate() {
     return info.getProperty("date", "Unknown");
     return info.getProperty("date", "Unknown");
   }
   }
 
 
-  protected String _getUser() {
+  protected String getUser() {
     return info.getProperty("user", "Unknown");
     return info.getProperty("user", "Unknown");
   }
   }
 
 
-  protected String _getUrl() {
+  protected String getUrl() {
     return info.getProperty("url", "Unknown");
     return info.getProperty("url", "Unknown");
   }
   }
 
 
-  protected String _getSrcChecksum() {
+  protected String getSrcChecksum() {
     return info.getProperty("srcChecksum", "Unknown");
     return info.getProperty("srcChecksum", "Unknown");
   }
   }
 
 
-  protected String _getBuildVersion(){
-    return _getVersion() +
-      " from " + _getRevision() +
-      " by " + _getUser() +
-      " source checksum " + _getSrcChecksum();
+  public String getBuildVersion() {
+    return HDDS_VERSION_INFO.getVersion() +
+        " from " + HDDS_VERSION_INFO.getRevision() +
+        " by " + getUser() +
+        " source checksum " + getSrcChecksum();
   }
   }
 
 
-  protected String _getProtocVersion() {
+  protected String getProtocVersion() {
     return info.getProperty("protocVersion", "Unknown");
     return info.getProperty("protocVersion", "Unknown");
   }
   }
 
 
-  private static final HddsVersionInfo HDDS_VERSION_INFO =
-      new HddsVersionInfo("hdds");
-  /**
-   * Get the HDDS version.
-   * @return the Hdds version string, eg. "0.6.3-dev"
-   */
-  public static String getVersion() {
-    return HDDS_VERSION_INFO._getVersion();
-  }
-
-  /**
-   * Get the Git commit hash of the repository when compiled.
-   * @return the commit hash, eg. "18f64065d5db6208daf50b02c1b5ed4ee3ce547a"
-   */
-  public static String getRevision() {
-    return HDDS_VERSION_INFO._getRevision();
-  }
-
-  /**
-   * Get the branch on which this originated.
-   * @return The branch name, e.g. "trunk" or "branches/branch-0.20"
-   */
-  public static String getBranch() {
-    return HDDS_VERSION_INFO._getBranch();
-  }
-
-  /**
-   * The date that HDDS was compiled.
-   * @return the compilation date in unix date format
-   */
-  public static String getDate() {
-    return HDDS_VERSION_INFO._getDate();
-  }
-
-  /**
-   * The user that compiled HDDS.
-   * @return the username of the user
-   */
-  public static String getUser() {
-    return HDDS_VERSION_INFO._getUser();
-  }
-
-  /**
-   * Get the URL for the HDDS repository.
-   * @return the URL of the Hdds repository
-   */
-  public static String getUrl() {
-    return HDDS_VERSION_INFO._getUrl();
-  }
-
-  /**
-   * Get the checksum of the source files from which HDDS was built.
-   * @return the checksum of the source files
-   */
-  public static String getSrcChecksum() {
-    return HDDS_VERSION_INFO._getSrcChecksum();
-  }
-
-  /**
-   * Returns the buildVersion which includes version,
-   * revision, user and date.
-   * @return the buildVersion
-   */
-  public static String getBuildVersion(){
-    return HDDS_VERSION_INFO._getBuildVersion();
-  }
-
-  /**
-   * Returns the protoc version used for the build.
-   * @return the protoc version
-   */
-  public static String getProtocVersion(){
-    return HDDS_VERSION_INFO._getProtocVersion();
-  }
-
   public static void main(String[] args) {
   public static void main(String[] args) {
-    System.out.println("Using HDDS " + getVersion());
-    System.out.println("Source code repository " + getUrl() + " -r " +
-        getRevision());
-    System.out.println("Compiled by " + getUser() + " on " + getDate());
-    System.out.println("Compiled with protoc " + getProtocVersion());
-    System.out.println("From source with checksum " + getSrcChecksum());
+    System.out.println("Using HDDS " + HDDS_VERSION_INFO.getVersion());
+    System.out.println(
+        "Source code repository " + HDDS_VERSION_INFO.getUrl() + " -r " +
+            HDDS_VERSION_INFO.getRevision());
+    System.out.println("Compiled by " + HDDS_VERSION_INFO.getUser() + " on "
+        + HDDS_VERSION_INFO.getDate());
+    System.out.println(
+        "Compiled with protoc " + HDDS_VERSION_INFO.getProtocVersion());
+    System.out.println(
+        "From source with checksum " + HDDS_VERSION_INFO.getSrcChecksum());
     LOG.debug("This command was run using " +
     LOG.debug("This command was run using " +
         ClassUtil.findContainingJar(HddsVersionInfo.class));
         ClassUtil.findContainingJar(HddsVersionInfo.class));
   }
   }

+ 11 - 17
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStoreBuilder.java

@@ -18,10 +18,20 @@
 
 
 package org.apache.hadoop.utils;
 package org.apache.hadoop.utils;
 
 
-import com.google.common.annotations.VisibleForTesting;
+import java.io.File;
+import java.io.IOException;
+import java.util.Optional;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
+
+import com.google.common.annotations.VisibleForTesting;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_ROCKSDB_STATISTICS_OFF;
 import org.iq80.leveldb.Options;
 import org.iq80.leveldb.Options;
 import org.rocksdb.BlockBasedTableConfig;
 import org.rocksdb.BlockBasedTableConfig;
 import org.rocksdb.Statistics;
 import org.rocksdb.Statistics;
@@ -29,22 +39,6 @@ import org.rocksdb.StatsLevel;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Optional;
-import java.util.function.Supplier;
-
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_METADATA_STORE_IMPL_LEVELDB;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_METADATA_STORE_IMPL_ROCKSDB;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_METADATA_STORE_ROCKSDB_STATISTICS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_METADATA_STORE_ROCKSDB_STATISTICS_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_METADATA_STORE_ROCKSDB_STATISTICS_OFF;
-
 /**
 /**
  * Builder for metadata store.
  * Builder for metadata store.
  */
  */

+ 1 - 0
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
  * Dispatcher sends ContainerCommandRequests to Handler. Each Container Type
  * Dispatcher sends ContainerCommandRequests to Handler. Each Container Type
  * should have an implementation for Handler.
  * should have an implementation for Handler.
  */
  */
+@SuppressWarnings("visibilitymodifier")
 public abstract class Handler {
 public abstract class Handler {
 
 
   protected final Configuration conf;
   protected final Configuration conf;

+ 2 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java

@@ -86,7 +86,8 @@ public class VersionEndpointTask implements
           Preconditions.checkNotNull(clusterId, "Reply from SCM: clusterId " +
           Preconditions.checkNotNull(clusterId, "Reply from SCM: clusterId " +
               "cannot be null");
               "cannot be null");
 
 
-          // If version file does not exist create version file and also set scmId
+          // If version file does not exist
+          // create version file and also set scmId
 
 
           for (Map.Entry<String, HddsVolume> entry : volumeMap.entrySet()) {
           for (Map.Entry<String, HddsVolume> entry : volumeMap.entrySet()) {
             HddsVolume hddsVolume = entry.getValue();
             HddsVolume hddsVolume = entry.getValue();

+ 9 - 9
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-public class DispatcherContext {
+public final class DispatcherContext {
   /**
   /**
    * Determines which stage of writeChunk a write chunk request is for.
    * Determines which stage of writeChunk a write chunk request is for.
    */
    */
@@ -82,8 +82,8 @@ public class DispatcherContext {
      * @param stage WriteChunk Stage
      * @param stage WriteChunk Stage
      * @return DispatcherContext.Builder
      * @return DispatcherContext.Builder
      */
      */
-    public Builder setStage(WriteChunkStage stage) {
-      this.stage = stage;
+    public Builder setStage(WriteChunkStage writeChunkStage) {
+      this.stage = writeChunkStage;
       return this;
       return this;
     }
     }
 
 
@@ -93,8 +93,8 @@ public class DispatcherContext {
      * @param readFromTmpFile whether to read from tmp chunk file or not
      * @param readFromTmpFile whether to read from tmp chunk file or not
      * @return DispatcherContext.Builder
      * @return DispatcherContext.Builder
      */
      */
-    public Builder setReadFromTmpFile(boolean readFromTmpFile) {
-      this.readFromTmpFile = readFromTmpFile;
+    public Builder setReadFromTmpFile(boolean setReadFromTmpFile) {
+      this.readFromTmpFile = setReadFromTmpFile;
       return this;
       return this;
     }
     }
 
 
@@ -104,8 +104,8 @@ public class DispatcherContext {
      * @param term current term
      * @param term current term
      * @return DispatcherContext.Builder
      * @return DispatcherContext.Builder
      */
      */
-    public Builder setTerm(long term) {
-      this.term = term;
+    public Builder setTerm(long currentTerm) {
+      this.term = currentTerm;
       return this;
       return this;
     }
     }
 
 
@@ -115,8 +115,8 @@ public class DispatcherContext {
      * @param logIndex log index
      * @param logIndex log index
      * @return DispatcherContext.Builder
      * @return DispatcherContext.Builder
      */
      */
-    public Builder setLogIndex(long logIndex) {
-      this.logIndex = logIndex;
+    public Builder setLogIndex(long index) {
+      this.logIndex = index;
       return this;
       return this;
     }
     }
 
 

+ 148 - 97
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java

@@ -156,58 +156,19 @@ public final class XceiverServerRatis implements XceiverServerSpi {
     final RaftProperties properties = new RaftProperties();
     final RaftProperties properties = new RaftProperties();
 
 
     // Set rpc type
     // Set rpc type
-    final String rpcType = conf.get(
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
-    final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
-    RaftConfigKeys.Rpc.setType(properties, rpc);
+    final RpcType rpc = setRpcType(conf, properties);
 
 
     // set raft segment size
     // set raft segment size
-    final int raftSegmentSize = (int)conf.getStorageSize(
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY,
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT,
-        StorageUnit.BYTES);
-    RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
-        SizeInBytes.valueOf(raftSegmentSize));
+    setRaftSegmentSize(conf, properties);
 
 
     // set raft segment pre-allocated size
     // set raft segment pre-allocated size
-    final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
-        StorageUnit.BYTES);
-    int logAppenderQueueNumElements = conf.getInt(
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS,
-        OzoneConfigKeys
-            .DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT);
-    final int logAppenderQueueByteLimit = (int) conf.getStorageSize(
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
-        OzoneConfigKeys
-            .DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
-        StorageUnit.BYTES);
-    RaftServerConfigKeys.Log.Appender
-        .setBufferElementLimit(properties, logAppenderQueueNumElements);
-    RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties,
-        SizeInBytes.valueOf(logAppenderQueueByteLimit));
-    RaftServerConfigKeys.Log.setPreallocatedSize(properties,
-        SizeInBytes.valueOf(raftSegmentPreallocatedSize));
+    final int raftSegmentPreallocatedSize =
+        setRaftSegmentPreallocatedSize(conf, properties);
 
 
     // Set max write buffer size, which is the scm chunk size
     // Set max write buffer size, which is the scm chunk size
-    final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
-    RaftServerConfigKeys.Log.setWriteBufferSize(properties,
-        SizeInBytes.valueOf(maxChunkSize));
-
-    // Set the client requestTimeout
-    TimeUnit timeUnit =
-        OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
-            .getUnit();
-    long duration = conf.getTimeDuration(
-        OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
-        OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
-            .getDuration(), timeUnit);
-    final TimeDuration clientRequestTimeout =
-        TimeDuration.valueOf(duration, timeUnit);
-    RaftClientConfigKeys.Rpc
-        .setRequestTimeout(properties, clientRequestTimeout);
+    final int maxChunkSize = setMaxWriteBuffer(conf, properties);
+    TimeUnit timeUnit;
+    long duration;
 
 
     // set the configs enable and set the stateMachineData sync timeout
     // set the configs enable and set the stateMachineData sync timeout
     RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true);
     RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true);
@@ -224,66 +185,19 @@ public final class XceiverServerRatis implements XceiverServerSpi {
         .setSyncTimeout(properties, dataSyncTimeout);
         .setSyncTimeout(properties, dataSyncTimeout);
 
 
     // Set the server Request timeout
     // Set the server Request timeout
-    timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
-        .getUnit();
-    duration = conf.getTimeDuration(
-        OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_KEY,
-        OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
-            .getDuration(), timeUnit);
-    final TimeDuration serverRequestTimeout =
-        TimeDuration.valueOf(duration, timeUnit);
-    RaftServerConfigKeys.Rpc
-        .setRequestTimeout(properties, serverRequestTimeout);
+    setServerRequestTimeout(conf, properties);
 
 
     // set timeout for a retry cache entry
     // set timeout for a retry cache entry
-    timeUnit =
-        OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
-            .getUnit();
-    duration = conf.getTimeDuration(
-        OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY,
-        OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
-            .getDuration(), timeUnit);
-    final TimeDuration retryCacheTimeout =
-        TimeDuration.valueOf(duration, timeUnit);
-    RaftServerConfigKeys.RetryCache
-        .setExpiryTime(properties, retryCacheTimeout);
+    setTimeoutForRetryCache(conf, properties);
 
 
     // Set the ratis leader election timeout
     // Set the ratis leader election timeout
-    TimeUnit leaderElectionMinTimeoutUnit =
-        OzoneConfigKeys.
-            DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
-            .getUnit();
-    duration = conf.getTimeDuration(
-        OzoneConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY,
-        OzoneConfigKeys.
-            DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
-            .getDuration(), leaderElectionMinTimeoutUnit);
-    final TimeDuration leaderElectionMinTimeout =
-        TimeDuration.valueOf(duration, leaderElectionMinTimeoutUnit);
-    RaftServerConfigKeys.Rpc
-        .setTimeoutMin(properties, leaderElectionMinTimeout);
-    long leaderElectionMaxTimeout =
-        leaderElectionMinTimeout.toLong(TimeUnit.MILLISECONDS) + 200;
-    RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
-        TimeDuration.valueOf(leaderElectionMaxTimeout, TimeUnit.MILLISECONDS));
+    setRatisLeaderElectionTimeout(conf, properties);
 
 
     // Set the maximum cache segments
     // Set the maximum cache segments
     RaftServerConfigKeys.Log.setMaxCachedSegmentNum(properties, 2);
     RaftServerConfigKeys.Log.setMaxCachedSegmentNum(properties, 2);
 
 
     // set the node failure timeout
     // set the node failure timeout
-    timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
-        .getUnit();
-    duration = conf.getTimeDuration(
-        OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_KEY,
-        OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
-            .getDuration(), timeUnit);
-    final TimeDuration nodeFailureTimeout =
-        TimeDuration.valueOf(duration, timeUnit);
-    RaftServerConfigKeys.setLeaderElectionTimeout(properties,
-        nodeFailureTimeout);
-    RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
-        nodeFailureTimeout);
-    nodeFailureTimeoutMs = nodeFailureTimeout.toLong(TimeUnit.MILLISECONDS);
+    setNodeFailureTimeout(conf, properties);
 
 
     // Set the ratis storage directory
     // Set the ratis storage directory
     String storageDir = HddsServerUtil.getOzoneDatanodeRatisDirectory(conf);
     String storageDir = HddsServerUtil.getOzoneDatanodeRatisDirectory(conf);
@@ -331,6 +245,143 @@ public final class XceiverServerRatis implements XceiverServerSpi {
     return properties;
     return properties;
   }
   }
 
 
+  private void setNodeFailureTimeout(Configuration conf,
+                                     RaftProperties properties) {
+    TimeUnit timeUnit;
+    long duration;
+    timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
+        .getUnit();
+    duration = conf.getTimeDuration(
+        OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_KEY,
+        OzoneConfigKeys.DFS_RATIS_SERVER_FAILURE_DURATION_DEFAULT
+            .getDuration(), timeUnit);
+    final TimeDuration nodeFailureTimeout =
+        TimeDuration.valueOf(duration, timeUnit);
+    RaftServerConfigKeys.setLeaderElectionTimeout(properties,
+        nodeFailureTimeout);
+    RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
+        nodeFailureTimeout);
+    nodeFailureTimeoutMs = nodeFailureTimeout.toLong(TimeUnit.MILLISECONDS);
+  }
+
+  private void setRatisLeaderElectionTimeout(Configuration conf,
+                                             RaftProperties properties) {
+    long duration;
+    TimeUnit leaderElectionMinTimeoutUnit =
+        OzoneConfigKeys.
+            DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
+            .getUnit();
+    duration = conf.getTimeDuration(
+        OzoneConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY,
+        OzoneConfigKeys.
+            DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
+            .getDuration(), leaderElectionMinTimeoutUnit);
+    final TimeDuration leaderElectionMinTimeout =
+        TimeDuration.valueOf(duration, leaderElectionMinTimeoutUnit);
+    RaftServerConfigKeys.Rpc
+        .setTimeoutMin(properties, leaderElectionMinTimeout);
+    long leaderElectionMaxTimeout =
+        leaderElectionMinTimeout.toLong(TimeUnit.MILLISECONDS) + 200;
+    RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
+        TimeDuration.valueOf(leaderElectionMaxTimeout, TimeUnit.MILLISECONDS));
+  }
+
+  private void setTimeoutForRetryCache(Configuration conf,
+                                       RaftProperties properties) {
+    TimeUnit timeUnit;
+    long duration;
+    timeUnit =
+        OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
+            .getUnit();
+    duration = conf.getTimeDuration(
+        OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY,
+        OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
+            .getDuration(), timeUnit);
+    final TimeDuration retryCacheTimeout =
+        TimeDuration.valueOf(duration, timeUnit);
+    RaftServerConfigKeys.RetryCache
+        .setExpiryTime(properties, retryCacheTimeout);
+  }
+
+  private void setServerRequestTimeout(Configuration conf,
+                                       RaftProperties properties) {
+    TimeUnit timeUnit;
+    long duration;
+    timeUnit = OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
+        .getUnit();
+    duration = conf.getTimeDuration(
+        OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_KEY,
+        OzoneConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_DEFAULT
+            .getDuration(), timeUnit);
+    final TimeDuration serverRequestTimeout =
+        TimeDuration.valueOf(duration, timeUnit);
+    RaftServerConfigKeys.Rpc
+        .setRequestTimeout(properties, serverRequestTimeout);
+  }
+
+  private int setMaxWriteBuffer(Configuration conf, RaftProperties properties) {
+    final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
+    RaftServerConfigKeys.Log.setWriteBufferSize(properties,
+        SizeInBytes.valueOf(maxChunkSize));
+
+    // Set the client requestTimeout
+    TimeUnit timeUnit =
+        OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
+            .getUnit();
+    long duration = conf.getTimeDuration(
+        OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
+        OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
+            .getDuration(), timeUnit);
+    final TimeDuration clientRequestTimeout =
+        TimeDuration.valueOf(duration, timeUnit);
+    RaftClientConfigKeys.Rpc
+        .setRequestTimeout(properties, clientRequestTimeout);
+    return maxChunkSize;
+  }
+
+  private int setRaftSegmentPreallocatedSize(Configuration conf,
+                                             RaftProperties properties) {
+    final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
+        StorageUnit.BYTES);
+    int logAppenderQueueNumElements = conf.getInt(
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS,
+        OzoneConfigKeys
+            .DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT);
+    final int logAppenderQueueByteLimit = (int) conf.getStorageSize(
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
+        OzoneConfigKeys
+            .DFS_CONTAINER_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
+        StorageUnit.BYTES);
+    RaftServerConfigKeys.Log.Appender
+        .setBufferElementLimit(properties, logAppenderQueueNumElements);
+    RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties,
+        SizeInBytes.valueOf(logAppenderQueueByteLimit));
+    RaftServerConfigKeys.Log.setPreallocatedSize(properties,
+        SizeInBytes.valueOf(raftSegmentPreallocatedSize));
+    return raftSegmentPreallocatedSize;
+  }
+
+  private void setRaftSegmentSize(Configuration conf,
+                                  RaftProperties properties) {
+    final int raftSegmentSize = (int)conf.getStorageSize(
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY,
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT,
+        StorageUnit.BYTES);
+    RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
+        SizeInBytes.valueOf(raftSegmentSize));
+  }
+
+  private RpcType setRpcType(Configuration conf, RaftProperties properties) {
+    final String rpcType = conf.get(
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
+    final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
+    RaftConfigKeys.Rpc.setType(properties, rpc);
+    return rpc;
+  }
+
   public static XceiverServerRatis newXceiverServerRatis(
   public static XceiverServerRatis newXceiverServerRatis(
       DatanodeDetails datanodeDetails, Configuration ozoneConf,
       DatanodeDetails datanodeDetails, Configuration ozoneConf,
       ContainerDispatcher dispatcher, StateContext context) throws IOException {
       ContainerDispatcher dispatcher, StateContext context) throws IOException {

+ 1 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfo.java

@@ -31,7 +31,7 @@ import java.io.IOException;
 /**
 /**
  * Stores information about a disk/volume.
  * Stores information about a disk/volume.
  */
  */
-public class VolumeInfo {
+public final class VolumeInfo {
 
 
   private static final Logger LOG = LoggerFactory.getLogger(VolumeInfo.class);
   private static final Logger LOG = LoggerFactory.getLogger(VolumeInfo.class);
 
 

+ 8 - 5
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java

@@ -41,10 +41,12 @@ public final class KeyValueContainerLocationUtil {
    * @return containerMetadata Path to container metadata location where
    * @return containerMetadata Path to container metadata location where
    * .container file will be stored.
    * .container file will be stored.
    */
    */
-  public static File getContainerMetaDataPath(String hddsVolumeDir, String scmId,
+  public static File getContainerMetaDataPath(String hddsVolumeDir,
+                                              String scmId,
                                               long containerId) {
                                               long containerId) {
-    String containerMetaDataPath = getBaseContainerLocation(hddsVolumeDir, scmId,
-        containerId);
+    String containerMetaDataPath =
+        getBaseContainerLocation(hddsVolumeDir, scmId,
+            containerId);
     containerMetaDataPath = containerMetaDataPath + File.separator +
     containerMetaDataPath = containerMetaDataPath + File.separator +
         OzoneConsts.CONTAINER_META_PATH;
         OzoneConsts.CONTAINER_META_PATH;
     return new File(containerMetaDataPath);
     return new File(containerMetaDataPath);
@@ -72,8 +74,9 @@ public final class KeyValueContainerLocationUtil {
    * @param containerId
    * @param containerId
    * @return base directory for container.
    * @return base directory for container.
    */
    */
-  private static String getBaseContainerLocation(String hddsVolumeDir, String scmId,
-                                        long containerId) {
+  private static String getBaseContainerLocation(String hddsVolumeDir,
+                                                 String scmId,
+                                                 long containerId) {
     Preconditions.checkNotNull(hddsVolumeDir, "Base Directory cannot be null");
     Preconditions.checkNotNull(hddsVolumeDir, "Base Directory cannot be null");
     Preconditions.checkNotNull(scmId, "scmUuid cannot be null");
     Preconditions.checkNotNull(scmId, "scmUuid cannot be null");
     Preconditions.checkState(containerId >= 0,
     Preconditions.checkState(containerId >= 0,

+ 4 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java

@@ -57,10 +57,10 @@ public class GrpcReplicationService extends
     LOG.info("Streaming container data ({}) to other datanode",
     LOG.info("Streaming container data ({}) to other datanode",
         request.getContainerID());
         request.getContainerID());
     try {
     try {
-        GrpcOutputStream outputStream =
-            new GrpcOutputStream(responseObserver, request.getContainerID());
-        containerReplicationSource
-            .copyData(request.getContainerID(), outputStream);
+      GrpcOutputStream outputStream =
+          new GrpcOutputStream(responseObserver, request.getContainerID());
+      containerReplicationSource
+          .copyData(request.getContainerID(), outputStream);
 
 
     } catch (IOException e) {
     } catch (IOException e) {
       LOG.error("Can't stream the container data", e);
       LOG.error("Can't stream the container data", e);

+ 0 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java

@@ -30,7 +30,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.function.Function;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;

+ 10 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteBlockCommandStatus.java

@@ -24,13 +24,17 @@ import org.apache.hadoop.hdds.protocol.proto
 import org.apache.hadoop.hdds.protocol.proto
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
     .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
 
 
+/**
+ * Command status to report about block deletion.
+ */
 public class DeleteBlockCommandStatus extends CommandStatus {
 public class DeleteBlockCommandStatus extends CommandStatus {
 
 
   private ContainerBlocksDeletionACKProto blocksDeletionAck = null;
   private ContainerBlocksDeletionACKProto blocksDeletionAck = null;
 
 
   public DeleteBlockCommandStatus(Type type, Long cmdId,
   public DeleteBlockCommandStatus(Type type, Long cmdId,
       StorageContainerDatanodeProtocolProtos.CommandStatus.Status status,
       StorageContainerDatanodeProtocolProtos.CommandStatus.Status status,
-      String msg, ContainerBlocksDeletionACKProto blocksDeletionAck) {
+      String msg,
+      ContainerBlocksDeletionACKProto blocksDeletionAck) {
     super(type, cmdId, status, msg);
     super(type, cmdId, status, msg);
     this.blocksDeletionAck = blocksDeletionAck;
     this.blocksDeletionAck = blocksDeletionAck;
   }
   }
@@ -53,7 +57,8 @@ public class DeleteBlockCommandStatus extends CommandStatus {
   }
   }
 
 
   @Override
   @Override
-  public StorageContainerDatanodeProtocolProtos.CommandStatus getProtoBufMessage() {
+  public StorageContainerDatanodeProtocolProtos.CommandStatus
+      getProtoBufMessage() {
     StorageContainerDatanodeProtocolProtos.CommandStatus.Builder builder =
     StorageContainerDatanodeProtocolProtos.CommandStatus.Builder builder =
         StorageContainerDatanodeProtocolProtos.CommandStatus.newBuilder()
         StorageContainerDatanodeProtocolProtos.CommandStatus.newBuilder()
             .setCmdId(this.getCmdId())
             .setCmdId(this.getCmdId())
@@ -68,6 +73,9 @@ public class DeleteBlockCommandStatus extends CommandStatus {
     return builder.build();
     return builder.build();
   }
   }
 
 
+  /**
+   * Builder for DeleteBlockCommandStatus.
+   */
   public static final class DeleteBlockCommandStatusBuilder
   public static final class DeleteBlockCommandStatusBuilder
       extends CommandStatusBuilder {
       extends CommandStatusBuilder {
     private ContainerBlocksDeletionACKProto blocksDeletionAck = null;
     private ContainerBlocksDeletionACKProto blocksDeletionAck = null;

+ 9 - 13
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java

@@ -16,28 +16,24 @@
  */
  */
 package org.apache.hadoop.ozone.container.common;
 package org.apache.hadoop.ozone.container.common;
 
 
-import com.google.protobuf.BlockingService;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.HddsConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos
-    .StorageContainerDatanodeProtocolService;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageContainerDatanodeProtocolService;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
 import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
 import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
 import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
-import org.apache.hadoop.ozone.protocolPB
-    .StorageContainerDatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-
+import com.google.protobuf.BlockingService;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
 
 
 /**
 /**

+ 2 - 2
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java

@@ -94,8 +94,8 @@ public class TestHandler {
     Assert.assertEquals("New ContainerType detected. Not an invalid " +
     Assert.assertEquals("New ContainerType detected. Not an invalid " +
         "containerType", invalidContainerType, null);
         "containerType", invalidContainerType, null);
 
 
-    Handler handler = dispatcher.getHandler(invalidContainerType);
+    Handler dispatcherHandler = dispatcher.getHandler(invalidContainerType);
     Assert.assertEquals("Get Handler for Invalid ContainerType should " +
     Assert.assertEquals("Get Handler for Invalid ContainerType should " +
-        "return null.", handler, null);
+        "return null.", dispatcherHandler, null);
   }
   }
 }
 }

+ 10 - 2
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventWatcher.java

@@ -66,10 +66,10 @@ public abstract class EventWatcher<TIMEOUT_PAYLOAD extends
 
 
   private final String name;
   private final String name;
 
 
-  protected final Map<Long, TIMEOUT_PAYLOAD> trackedEventsByID =
+  private final Map<Long, TIMEOUT_PAYLOAD> trackedEventsByID =
       new ConcurrentHashMap<>();
       new ConcurrentHashMap<>();
 
 
-  protected final Set<TIMEOUT_PAYLOAD> trackedEvents = new HashSet<>();
+  private final Set<TIMEOUT_PAYLOAD> trackedEvents = new HashSet<>();
 
 
   private final Map<Long, Long> startTrackingTimes = new HashedMap();
   private final Map<Long, Long> startTrackingTimes = new HashedMap();
 
 
@@ -206,4 +206,12 @@ public abstract class EventWatcher<TIMEOUT_PAYLOAD extends
   public TIMEOUT_PAYLOAD getTrackedEventbyId(long id) {
   public TIMEOUT_PAYLOAD getTrackedEventbyId(long id) {
     return trackedEventsByID.get(id);
     return trackedEventsByID.get(id);
   }
   }
+
+  public Map<Long, TIMEOUT_PAYLOAD> getTrackedEventsByID() {
+    return trackedEventsByID;
+  }
+
+  public Set<TIMEOUT_PAYLOAD> getTrackedEvents() {
+    return trackedEvents;
+  }
 }
 }

+ 3 - 0
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/PendingDeleteHandler.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.hdds.scm.block;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 
 
+/**
+ * Event handler for PedingDeleteStatuList events.
+ */
 public class PendingDeleteHandler implements
 public class PendingDeleteHandler implements
     EventHandler<PendingDeleteStatusList> {
     EventHandler<PendingDeleteStatusList> {
 
 

+ 1 - 3
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java

@@ -211,9 +211,7 @@ public class ReplicationManager implements Runnable {
                     .add(replica.getDatanodeDetails());
                     .add(replica.getDatanodeDetails());
               });
               });
 
 
-          for(UUID originId : originIdToDnMap.keySet()) {
-            final List<DatanodeDetails> listOfReplica =
-                originIdToDnMap.get(originId);
+          for (List<DatanodeDetails> listOfReplica : originIdToDnMap.values()) {
             if (listOfReplica.size() > 1) {
             if (listOfReplica.size() > 1) {
               final int toDelete = Math.min(listOfReplica.size() - 1,
               final int toDelete = Math.min(listOfReplica.size() - 1,
                   numberOfReplicasToDelete);
                   numberOfReplicasToDelete);

+ 2 - 0
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ObjectsMap.java

@@ -38,6 +38,8 @@ import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.DUP
  * This information is built from the DN container reports.
  * This information is built from the DN container reports.
  */
  */
 public class Node2ObjectsMap<T> {
 public class Node2ObjectsMap<T> {
+
+  @SuppressWarnings("visibilitymodifier")
   protected final Map<UUID, Set<T>> dn2ObjectMap;
   protected final Map<UUID, Set<T>> dn2ObjectMap;
 
 
   /**
   /**

+ 5 - 3
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2PipelineMap.java

@@ -27,10 +27,12 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.UUID;
 
 
 /**
 /**
- * This data structure maintains the list of pipelines which the given datanode is a part of. This
- * information will be added whenever a new pipeline allocation happens.
+ * This data structure maintains the list of pipelines which the given
+ * datanode is a part of. This information will be added whenever a new
+ * pipeline allocation happens.
  *
  *
- * <p>TODO: this information needs to be regenerated from pipeline reports on SCM restart
+ * <p>TODO: this information needs to be regenerated from pipeline reports
+ * on SCM restart
  */
  */
 public class Node2PipelineMap extends Node2ObjectsMap<PipelineID> {
 public class Node2PipelineMap extends Node2ObjectsMap<PipelineID> {
 
 

+ 1 - 0
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java

@@ -400,6 +400,7 @@ public final class TestUtils {
    *
    *
    * @return ContainerInfo
    * @return ContainerInfo
    */
    */
+  @SuppressWarnings("parameternumber")
   public static ContainerReplicaProto createContainerInfo(
   public static ContainerReplicaProto createContainerInfo(
       long containerId, long size, long keyCount, long bytesUsed,
       long containerId, long size, long keyCount, long bytesUsed,
       long readCount, long readBytes, long writeCount, long writeBytes) {
       long readCount, long readBytes, long writeCount, long writeBytes) {

+ 5 - 3
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java

@@ -45,6 +45,9 @@ import java.util.List;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 
 
+/**
+ * Unit test for command status report handler.
+ */
 public class TestCommandStatusReportHandler implements EventPublisher {
 public class TestCommandStatusReportHandler implements EventPublisher {
 
 
   private static final Logger LOG = LoggerFactory
   private static final Logger LOG = LoggerFactory
@@ -67,7 +70,6 @@ public class TestCommandStatusReportHandler implements EventPublisher {
     assertFalse(logCapturer.getOutput().contains("Delete_Block_Status"));
     assertFalse(logCapturer.getOutput().contains("Delete_Block_Status"));
     assertFalse(logCapturer.getOutput().contains("Replicate_Command_Status"));
     assertFalse(logCapturer.getOutput().contains("Replicate_Command_Status"));
 
 
-
     report = this.getStatusReport(this.getCommandStatusList());
     report = this.getStatusReport(this.getCommandStatusList());
     cmdStatusReportHandler.onMessage(report, this);
     cmdStatusReportHandler.onMessage(report, this);
     assertTrue(logCapturer.getOutput().contains("firing event of type " +
     assertTrue(logCapturer.getOutput().contains("firing event of type " +
@@ -92,8 +94,8 @@ public class TestCommandStatusReportHandler implements EventPublisher {
   }
   }
 
 
   @Override
   @Override
-  public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void fireEvent
-      (EVENT_TYPE event, PAYLOAD payload) {
+  public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void
+      fireEvent(EVENT_TYPE event, PAYLOAD payload) {
     LOG.info("firing event of type {}, payload {}", event.getName(), payload
     LOG.info("firing event of type {}, payload {}", event.getName(), payload
         .toString());
         .toString());
   }
   }

+ 2 - 2
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java

@@ -212,8 +212,8 @@ public class TestSCMContainerManager {
         HddsProtos.LifeCycleEvent.FINALIZE);
         HddsProtos.LifeCycleEvent.FINALIZE);
     containerManager.updateContainerState(id,
     containerManager.updateContainerState(id,
         HddsProtos.LifeCycleEvent.CLOSE);
         HddsProtos.LifeCycleEvent.CLOSE);
-   ContainerInfo closedContainer = containerManager.getContainer(id);
-   Assert.assertEquals(LifeCycleState.CLOSED, closedContainer.getState());
+    ContainerInfo closedContainer = containerManager.getContainer(id);
+    Assert.assertEquals(LifeCycleState.CLOSED, closedContainer.getState());
   }
   }
 
 
   /**
   /**

+ 3 - 0
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java

@@ -36,6 +36,9 @@ import static org.mockito.Matchers.anyObject;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
+/**
+ * Test for the scm container placement.
+ */
 public class TestSCMContainerPlacementCapacity {
 public class TestSCMContainerPlacementCapacity {
   @Test
   @Test
   public void chooseDatanodes() throws SCMException {
   public void chooseDatanodes() throws SCMException {

+ 3 - 0
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java

@@ -34,6 +34,9 @@ import static org.mockito.Matchers.anyObject;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
+/**
+ * Test for the random container placement.
+ */
 public class TestSCMContainerPlacementRandom {
 public class TestSCMContainerPlacementRandom {
 
 
   @Test
   @Test

+ 2 - 2
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java

@@ -259,11 +259,11 @@ public class TestDeadNodeHandler {
         "DeadNode event for a unregistered node"));
         "DeadNode event for a unregistered node"));
   }
   }
 
 
-  private void registerReplicas(ContainerManager containerManager,
+  private void registerReplicas(ContainerManager contManager,
       ContainerInfo container, DatanodeDetails... datanodes)
       ContainerInfo container, DatanodeDetails... datanodes)
       throws ContainerNotFoundException {
       throws ContainerNotFoundException {
     for (DatanodeDetails datanode : datanodes) {
     for (DatanodeDetails datanode : datanodes) {
-      containerManager.updateContainerReplica(
+      contManager.updateContainerReplica(
           new ContainerID(container.getContainerID()),
           new ContainerID(container.getContainerID()),
           ContainerReplica.newBuilder()
           ContainerReplica.newBuilder()
               .setContainerState(ContainerReplicaProto.State.OPEN)
               .setContainerState(ContainerReplicaProto.State.OPEN)

+ 3 - 0
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeReportHandler.java

@@ -35,6 +35,9 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+/**
+ * Test for the Node Report Handler.
+ */
 public class TestNodeReportHandler implements EventPublisher {
 public class TestNodeReportHandler implements EventPublisher {
 
 
   private static final Logger LOG = LoggerFactory
   private static final Logger LOG = LoggerFactory

+ 105 - 0
hadoop-ozone/Jenkinsfile

@@ -0,0 +1,105 @@
+/**
+ * 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.
+ */
+node("ubuntu") {
+    docker.image('elek/ozone-build').pull()
+    docker.image('elek/ozone-build').inside {
+
+        stage('Checkout') {
+            checkout scm
+        }
+
+        stage('Clean') {
+            status = sh returnStatus: true, script: 'mvn clean'
+        }
+
+        stageRunner('Author', "author", {})
+
+        stageRunner('Isolation', "isolation", {})
+
+
+        stageRunner('Build', "build", {})
+
+        stageRunner('Licence', "rat", {
+            archiveArtifacts 'target/rat-aggregated.txt'
+        }, 'artifact/target/rat-aggregated.txt/*view*/')
+
+        stageRunner('Unit test', "unit", {
+            junit '**/target/surefire-reports/*.xml'
+        }, 'testReport/')
+
+        stageRunner('Findbugs', "findbugs", {
+            archiveArtifacts 'target/findbugs-all.txt'
+
+        }, 'artifact/target/findbugs-all.txt/*view*/')
+
+        stageRunner('Checkstyle', "checkstyle", {
+            checkstyle canComputeNew: false, canRunOnFailed: true, defaultEncoding: '', healthy: '', pattern: '**/checkstyle-result.xml', unHealthy: ''
+        }, 'checkstyleResult')
+
+    }
+
+}
+
+def stageRunner(name, type, processResult, url = '') {
+    try {
+        stage(name) {
+            prStatusStart(type)
+            status = sh returnStatus: true, script: 'hadoop-ozone/dev-support/checks/' + type + '.sh'
+            processResult()
+            prStatusResult(status, type, url)
+        }
+        return true
+    } catch (RuntimeException ex) {
+        currentBuild.result = "FAILED"
+        return false
+    }
+}
+
+def prStatusStart(name) {
+    if (env.CHANGE_ID) {
+        pullRequest.createStatus(status: "pending",
+                context: 'continuous-integration/jenkins/pr-merge/' + name,
+                description: name + " is started")
+    }
+}
+
+def prStatusResult(responseCode, name, url = '') {
+    status = "error"
+    desc = "failed"
+    if (responseCode == 0) {
+        status = "success"
+        desc = "passed"
+    }
+    message = name + " is " + desc
+    //System.out.println(responseCode)
+    if (env.CHANGE_ID) {
+        if (url) {
+            pullRequest.createStatus(status: status,
+                    context: 'continuous-integration/jenkins/pr-merge/' + name,
+                    description: message,
+                    targetUrl: env.BUILD_URL + url)
+        } else {
+            pullRequest.createStatus(status: status,
+                    context: 'continuous-integration/jenkins/pr-merge/' + name,
+                    description: message)
+        }
+    }
+    if (responseCode != 0) {
+        throw new RuntimeException(message)
+    }
+}

+ 2 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java

@@ -103,6 +103,7 @@ public class OzoneBucket {
    * @param versioning versioning status of the bucket.
    * @param versioning versioning status of the bucket.
    * @param creationTime creation time of the bucket.
    * @param creationTime creation time of the bucket.
    */
    */
+  @SuppressWarnings("parameternumber")
   public OzoneBucket(Configuration conf, ClientProtocol proxy,
   public OzoneBucket(Configuration conf, ClientProtocol proxy,
                      String volumeName, String bucketName,
                      String volumeName, String bucketName,
                      List<OzoneAcl> acls, StorageType storageType,
                      List<OzoneAcl> acls, StorageType storageType,
@@ -125,6 +126,7 @@ public class OzoneBucket {
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
+  @SuppressWarnings("parameternumber")
   OzoneBucket(String volumeName, String name,
   OzoneBucket(String volumeName, String name,
       ReplicationFactor defaultReplication,
       ReplicationFactor defaultReplication,
       ReplicationType defaultReplicationType,
       ReplicationType defaultReplicationType,

+ 1 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java

@@ -35,6 +35,7 @@ public class OzoneKeyDetails extends OzoneKey {
   /**
   /**
    * Constructs OzoneKeyDetails from OmKeyInfo.
    * Constructs OzoneKeyDetails from OmKeyInfo.
    */
    */
+  @SuppressWarnings("parameternumber")
   public OzoneKeyDetails(String volumeName, String bucketName, String keyName,
   public OzoneKeyDetails(String volumeName, String bucketName, String keyName,
                          long size, long creationTime, long modificationTime,
                          long size, long creationTime, long modificationTime,
                          List<OzoneKeyLocation> ozoneKeyLocations,
                          List<OzoneKeyLocation> ozoneKeyLocations,

+ 4 - 2
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java

@@ -82,6 +82,7 @@ public class OzoneVolume {
    * @param creationTime creation time of the volume
    * @param creationTime creation time of the volume
    * @param acls ACLs associated with the volume.
    * @param acls ACLs associated with the volume.
    */
    */
+  @SuppressWarnings("parameternumber")
   public OzoneVolume(Configuration conf, ClientProtocol proxy, String name,
   public OzoneVolume(Configuration conf, ClientProtocol proxy, String name,
                      String admin, String owner, long quotaInBytes,
                      String admin, String owner, long quotaInBytes,
                      long creationTime, List<OzoneAcl> acls) {
                      long creationTime, List<OzoneAcl> acls) {
@@ -265,12 +266,13 @@ public class OzoneVolume {
 
 
 
 
     /**
     /**
-     * Creates an Iterator to iterate over all buckets after prevBucket in the volume.
+     * Creates an Iterator to iterate over all buckets after prevBucket in
+     * the volume.
      * If prevBucket is null it iterates from the first bucket in the volume.
      * If prevBucket is null it iterates from the first bucket in the volume.
      * The returned buckets match bucket prefix.
      * The returned buckets match bucket prefix.
      * @param bucketPrefix
      * @param bucketPrefix
      */
      */
-    public BucketIterator(String bucketPrefix, String prevBucket) {
+    BucketIterator(String bucketPrefix, String prevBucket) {
       this.bucketPrefix = bucketPrefix;
       this.bucketPrefix = bucketPrefix;
       this.currentValue = null;
       this.currentValue = null;
       this.currentIterator = getNextListOfBuckets(prevBucket).iterator();
       this.currentIterator = getNextListOfBuckets(prevBucket).iterator();

+ 1 - 1
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java

@@ -136,7 +136,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
       off += numBytesRead;
       off += numBytesRead;
       len -= numBytesRead;
       len -= numBytesRead;
       if (current.getRemaining() <= 0 &&
       if (current.getRemaining() <= 0 &&
-        ((currentStreamIndex + 1) < streamEntries.size())) {
+          ((currentStreamIndex + 1) < streamEntries.size())) {
         currentStreamIndex += 1;
         currentStreamIndex += 1;
       }
       }
     }
     }

+ 4 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java

@@ -86,6 +86,7 @@ public class KeyOutputStream extends OutputStream {
    * A constructor for testing purpose only.
    * A constructor for testing purpose only.
    */
    */
   @VisibleForTesting
   @VisibleForTesting
+  @SuppressWarnings("parameternumber")
   public KeyOutputStream() {
   public KeyOutputStream() {
     streamEntries = new ArrayList<>();
     streamEntries = new ArrayList<>();
     omClient = null;
     omClient = null;
@@ -143,6 +144,8 @@ public class KeyOutputStream extends OutputStream {
     return locationInfoList;
     return locationInfoList;
   }
   }
 
 
+
+  @SuppressWarnings("parameternumber")
   public KeyOutputStream(OpenKeySession handler,
   public KeyOutputStream(OpenKeySession handler,
       XceiverClientManager xceiverClientManager,
       XceiverClientManager xceiverClientManager,
       StorageContainerLocationProtocolClientSideTranslatorPB scmClient,
       StorageContainerLocationProtocolClientSideTranslatorPB scmClient,
@@ -654,6 +657,7 @@ public class KeyOutputStream extends OutputStream {
     private final long watchTimeout;
     private final long watchTimeout;
     private List<ByteBuffer> bufferList;
     private List<ByteBuffer> bufferList;
 
 
+    @SuppressWarnings("parameternumber")
     BlockOutputStreamEntry(BlockID blockID, String key,
     BlockOutputStreamEntry(BlockID blockID, String key,
         XceiverClientManager xceiverClientManager,
         XceiverClientManager xceiverClientManager,
         XceiverClientSpi xceiverClient, String requestId, int chunkSize,
         XceiverClientSpi xceiverClient, String requestId, int chunkSize,

+ 20 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/hdds/protocol/package-info.java

@@ -0,0 +1,20 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hdds.protocol;
+/**
+ * Helper classes for the hdds protocol.
+ */

+ 2 - 4
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.ozone;
 package org.apache.hadoop.ozone;
 
 
 import java.io.File;
 import java.io.File;
-import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Optional;
 import java.util.Optional;
@@ -27,11 +26,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.server.ServerUtils;
 import org.apache.hadoop.hdds.server.ServerUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
-
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
 import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
 import static org.apache.hadoop.hdds.HddsUtils.getPortNumberFromConfigKeys;
 import static org.apache.hadoop.hdds.HddsUtils.getPortNumberFromConfigKeys;
@@ -40,6 +36,8 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_BIND_HOST_DEFAULT
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_BIND_PORT_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_BIND_PORT_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_PORT_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_PORT_DEFAULT;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
 /**
  * Stateless helper functions for the server and client side of OM
  * Stateless helper functions for the server and client side of OM

+ 2 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/freon/OzoneGetConf.java

@@ -115,6 +115,8 @@ public class OzoneGetConf extends Configured implements Tool {
    * {@link OzoneGetConf.Command}.
    * {@link OzoneGetConf.Command}.
    */
    */
   static class CommandHandler {
   static class CommandHandler {
+
+    @SuppressWarnings("visibilitymodifier")
     protected String key; // Configuration key to lookup
     protected String key; // Configuration key to lookup
 
 
     CommandHandler() {
     CommandHandler() {

+ 2 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmBucketInfoCodec.java

@@ -39,7 +39,8 @@ public class OmBucketInfoCodec implements Codec<OmBucketInfo> {
   @Override
   @Override
   public OmBucketInfo fromPersistedFormat(byte[] rawData) {
   public OmBucketInfo fromPersistedFormat(byte[] rawData) {
     Preconditions
     Preconditions
-        .checkNotNull("Null byte array can't converted to real object.");
+        .checkNotNull(rawData,
+            "Null byte array can't converted to real object.");
     try {
     try {
       return OmBucketInfo.getFromProtobuf(BucketInfo.parseFrom(rawData));
       return OmBucketInfo.getFromProtobuf(BucketInfo.parseFrom(rawData));
     } catch (InvalidProtocolBufferException e) {
     } catch (InvalidProtocolBufferException e) {

+ 2 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmKeyInfoCodec.java

@@ -39,7 +39,8 @@ public class OmKeyInfoCodec implements Codec<OmKeyInfo> {
   @Override
   @Override
   public OmKeyInfo fromPersistedFormat(byte[] rawData) {
   public OmKeyInfo fromPersistedFormat(byte[] rawData) {
     Preconditions
     Preconditions
-        .checkNotNull("Null byte array can't converted to real object.");
+        .checkNotNull(rawData,
+            "Null byte array can't converted to real object.");
     try {
     try {
       return OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(rawData));
       return OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(rawData));
     } catch (InvalidProtocolBufferException e) {
     } catch (InvalidProtocolBufferException e) {

+ 1 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmMultipartKeyInfoCodec.java

@@ -43,7 +43,7 @@ public class OmMultipartKeyInfoCodec implements Codec<OmMultipartKeyInfo> {
    * return null.
    * return null.
    */
    */
   public OmMultipartKeyInfo fromPersistedFormat(byte[] rawData) {
   public OmMultipartKeyInfo fromPersistedFormat(byte[] rawData) {
-    Preconditions.checkNotNull(
+    Preconditions.checkNotNull(rawData,
         "Null byte array can't converted to real object.");
         "Null byte array can't converted to real object.");
     try {
     try {
       return OmMultipartKeyInfo.getFromProto(OzoneManagerProtocolProtos
       return OmMultipartKeyInfo.getFromProto(OzoneManagerProtocolProtos

+ 2 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmVolumeArgsCodec.java

@@ -39,7 +39,8 @@ public class OmVolumeArgsCodec implements Codec<OmVolumeArgs> {
   @Override
   @Override
   public OmVolumeArgs fromPersistedFormat(byte[] rawData) {
   public OmVolumeArgs fromPersistedFormat(byte[] rawData) {
     Preconditions
     Preconditions
-        .checkNotNull("Null byte array can't converted to real object.");
+        .checkNotNull(rawData,
+            "Null byte array can't converted to real object.");
     try {
     try {
       return OmVolumeArgs.getFromProtobuf(VolumeInfo.parseFrom(rawData));
       return OmVolumeArgs.getFromProtobuf(VolumeInfo.parseFrom(rawData));
     } catch (InvalidProtocolBufferException e) {
     } catch (InvalidProtocolBufferException e) {

+ 2 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/VolumeListCodec.java

@@ -38,7 +38,8 @@ public class VolumeListCodec implements Codec<VolumeList> {
   @Override
   @Override
   public VolumeList fromPersistedFormat(byte[] rawData) {
   public VolumeList fromPersistedFormat(byte[] rawData) {
     Preconditions
     Preconditions
-        .checkNotNull("Null byte array can't converted to real object.");
+        .checkNotNull(rawData,
+            "Null byte array can't converted to real object.");
     try {
     try {
       return VolumeList.parseFrom(rawData);
       return VolumeList.parseFrom(rawData);
     } catch (InvalidProtocolBufferException e) {
     } catch (InvalidProtocolBufferException e) {

+ 3 - 4
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java

@@ -22,16 +22,15 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 
 
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.hdds.protocol.StorageType;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.audit.Auditable;
 import org.apache.hadoop.ozone.audit.Auditable;
-import org.apache.hadoop.ozone.protocol.proto
-    .OzoneManagerProtocolProtos.BucketArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketArgs;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 
 
+import com.google.common.base.Preconditions;
+
 /**
 /**
  * A class that encapsulates Bucket Arguments.
  * A class that encapsulates Bucket Arguments.
  */
  */

+ 8 - 9
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java

@@ -17,21 +17,20 @@
  */
  */
 package org.apache.hadoop.ozone.om.helpers;
 package org.apache.hadoop.ozone.om.helpers;
 
 
-import com.google.common.base.Preconditions;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.hdds.protocol.StorageType;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.audit.Auditable;
 import org.apache.hadoop.ozone.audit.Auditable;
-import org.apache.hadoop.ozone.protocol.proto
-    .OzoneManagerProtocolProtos.BucketInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 
 
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
+import com.google.common.base.Preconditions;
 
 
 /**
 /**
  * A class that encapsulates Bucket Info.
  * A class that encapsulates Bucket Info.

+ 1 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java

@@ -43,6 +43,7 @@ public final class OmKeyArgs implements Auditable {
   private final String multipartUploadID;
   private final String multipartUploadID;
   private final int multipartUploadPartNumber;
   private final int multipartUploadPartNumber;
 
 
+  @SuppressWarnings("parameternumber")
   private OmKeyArgs(String volumeName, String bucketName, String keyName,
   private OmKeyArgs(String volumeName, String bucketName, String keyName,
       long dataSize, ReplicationType type, ReplicationFactor factor,
       long dataSize, ReplicationType type, ReplicationFactor factor,
       List<OmKeyLocationInfo> locationInfoList, boolean isMultipart,
       List<OmKeyLocationInfo> locationInfoList, boolean isMultipart,

+ 1 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java

@@ -45,6 +45,7 @@ public final class OmKeyInfo {
   private HddsProtos.ReplicationType type;
   private HddsProtos.ReplicationType type;
   private HddsProtos.ReplicationFactor factor;
   private HddsProtos.ReplicationFactor factor;
 
 
+  @SuppressWarnings("parameternumber")
   private OmKeyInfo(String volumeName, String bucketName, String keyName,
   private OmKeyInfo(String volumeName, String bucketName, String keyName,
                     List<OmKeyLocationInfoGroup> versions, long dataSize,
                     List<OmKeyLocationInfoGroup> versions, long dataSize,
                     long creationTime, long modificationTime,
                     long creationTime, long modificationTime,

+ 33 - 104
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/OzoneVersionInfo.java

@@ -37,7 +37,8 @@ import java.util.Properties;
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class OzoneVersionInfo {
 public class OzoneVersionInfo {
-  private static final Logger LOG = LoggerFactory.getLogger(OzoneVersionInfo.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OzoneVersionInfo.class);
 
 
   private Properties info;
   private Properties info;
 
 
@@ -46,8 +47,9 @@ public class OzoneVersionInfo {
     String versionInfoFile = component + "-version-info.properties";
     String versionInfoFile = component + "-version-info.properties";
     InputStream is = null;
     InputStream is = null;
     try {
     try {
-      is = ThreadUtil.getResourceAsStream(OzoneVersionInfo.class.getClassLoader(),
-          versionInfoFile);
+      is = ThreadUtil
+          .getResourceAsStream(OzoneVersionInfo.class.getClassLoader(),
+              versionInfoFile);
       info.load(is);
       info.load(is);
     } catch (IOException ex) {
     } catch (IOException ex) {
       LoggerFactory.getLogger(getClass()).warn("Could not read '" +
       LoggerFactory.getLogger(getClass()).warn("Could not read '" +
@@ -57,130 +59,51 @@ public class OzoneVersionInfo {
     }
     }
   }
   }
 
 
-  protected String _getVersion() {
+  protected String getVersion() {
     return info.getProperty("version", "Unknown");
     return info.getProperty("version", "Unknown");
   }
   }
 
 
-  protected String _getRelease() {
+  protected String getRelease() {
     return info.getProperty("release", "Unknown");
     return info.getProperty("release", "Unknown");
   }
   }
 
 
-  protected String _getRevision() {
+  protected String getRevision() {
     return info.getProperty("revision", "Unknown");
     return info.getProperty("revision", "Unknown");
   }
   }
 
 
-  protected String _getBranch() {
+  protected String getBranch() {
     return info.getProperty("branch", "Unknown");
     return info.getProperty("branch", "Unknown");
   }
   }
 
 
-  protected String _getDate() {
+  protected String getDate() {
     return info.getProperty("date", "Unknown");
     return info.getProperty("date", "Unknown");
   }
   }
 
 
-  protected String _getUser() {
+  protected String getUser() {
     return info.getProperty("user", "Unknown");
     return info.getProperty("user", "Unknown");
   }
   }
 
 
-  protected String _getUrl() {
+  protected String getUrl() {
     return info.getProperty("url", "Unknown");
     return info.getProperty("url", "Unknown");
   }
   }
 
 
-  protected String _getSrcChecksum() {
+  protected String getSrcChecksum() {
     return info.getProperty("srcChecksum", "Unknown");
     return info.getProperty("srcChecksum", "Unknown");
   }
   }
 
 
-  protected String _getBuildVersion(){
-    return _getVersion() +
-      " from " + _getRevision() +
-      " by " + _getUser() +
-      " source checksum " + _getSrcChecksum();
+  protected String getBuildVersion() {
+    return getVersion() +
+        " from " + getRevision() +
+        " by " + getUser() +
+        " source checksum " + getSrcChecksum();
   }
   }
 
 
-  protected String _getProtocVersion() {
+  protected String getProtocVersion() {
     return info.getProperty("protocVersion", "Unknown");
     return info.getProperty("protocVersion", "Unknown");
   }
   }
 
 
-  private static OzoneVersionInfo OZONE_VERSION_INFO = new OzoneVersionInfo("ozone");
-  /**
-   * Get the Ozone version.
-   * @return the Ozone version string, eg. "0.6.3-dev"
-   */
-  public static String getVersion() {
-    return OZONE_VERSION_INFO._getVersion();
-  }
-
-  /**
-   * Get the Ozone release name.
-   * @return the Ozone release string, eg. "Acadia"
-   */
-  public static String getRelease() {
-    return OZONE_VERSION_INFO._getRelease();
-  }
-
-  /**
-   * Get the Git commit hash of the repository when compiled.
-   * @return the commit hash, eg. "18f64065d5db6208daf50b02c1b5ed4ee3ce547a"
-   */
-  public static String getRevision() {
-    return OZONE_VERSION_INFO._getRevision();
-  }
-
-  /**
-   * Get the branch on which this originated.
-   * @return The branch name, e.g. "trunk" or "branches/branch-0.20"
-   */
-  public static String getBranch() {
-    return OZONE_VERSION_INFO._getBranch();
-  }
-
-  /**
-   * The date that Ozone was compiled.
-   * @return the compilation date in unix date format
-   */
-  public static String getDate() {
-    return OZONE_VERSION_INFO._getDate();
-  }
-
-  /**
-   * The user that compiled Ozone.
-   * @return the username of the user
-   */
-  public static String getUser() {
-    return OZONE_VERSION_INFO._getUser();
-  }
-
-  /**
-   * Get the URL for the Ozone repository.
-   * @return the URL of the Ozone repository
-   */
-  public static String getUrl() {
-    return OZONE_VERSION_INFO._getUrl();
-  }
-
-  /**
-   * Get the checksum of the source files from which Ozone was built.
-   * @return the checksum of the source files
-   */
-  public static String getSrcChecksum() {
-    return OZONE_VERSION_INFO._getSrcChecksum();
-  }
-
-  /**
-   * Returns the buildVersion which includes version,
-   * revision, user and date.
-   * @return the buildVersion
-   */
-  public static String getBuildVersion(){
-    return OZONE_VERSION_INFO._getBuildVersion();
-  }
-
-  /**
-   * Returns the protoc version used for the build.
-   * @return the protoc version
-   */
-  public static String getProtocVersion(){
-    return OZONE_VERSION_INFO._getProtocVersion();
-  }
+  private static final OzoneVersionInfo OZONE_VERSION_INFO =
+      new OzoneVersionInfo("ozone");
 
 
   public static void main(String[] args) {
   public static void main(String[] args) {
     System.out.println(
     System.out.println(
@@ -200,12 +123,18 @@ public class OzoneVersionInfo {
         "             ///////////     ////////            \n" +
         "             ///////////     ////////            \n" +
         "               //////  ////////////              \n" +
         "               //////  ////////////              \n" +
         "               ///   //////////                  \n" +
         "               ///   //////////                  \n" +
-        "              /    "+ getVersion() + "("+ getRelease() +")\n");
-    System.out.println("Source code repository " + getUrl() + " -r " +
-        getRevision());
-    System.out.println("Compiled by " + getUser() + " on " + getDate());
-    System.out.println("Compiled with protoc " + getProtocVersion());
-    System.out.println("From source with checksum " + getSrcChecksum() + "\n");
+            "              /    " + OZONE_VERSION_INFO.getVersion() + "("
+            + OZONE_VERSION_INFO.getRelease() + ")\n");
+    System.out.println(
+        "Source code repository " + OZONE_VERSION_INFO.getUrl() + " -r " +
+            OZONE_VERSION_INFO.getRevision());
+    System.out.println("Compiled by " + OZONE_VERSION_INFO.getUser() + " on "
+        + OZONE_VERSION_INFO.getDate());
+    System.out.println(
+        "Compiled with protoc " + OZONE_VERSION_INFO.getProtocVersion());
+    System.out.println(
+        "From source with checksum " + OZONE_VERSION_INFO.getSrcChecksum()
+            + "\n");
     LOG.debug("This command was run using " +
     LOG.debug("This command was run using " +
         ClassUtil.findContainingJar(OzoneVersionInfo.class));
         ClassUtil.findContainingJar(OzoneVersionInfo.class));
     HddsVersionInfo.main(args);
     HddsVersionInfo.main(args);

+ 22 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/util/package-info.java

@@ -0,0 +1,22 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.util;
+
+/**
+ * Ozone utilities.
+ */

+ 1 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeArgs.java

@@ -66,6 +66,7 @@ public class VolumeArgs extends UserArgs {
    * @param headers - http headers
    * @param headers - http headers
    * @param groups - list of groups allowed to access the volume
    * @param groups - list of groups allowed to access the volume
    */
    */
+  @SuppressWarnings("parameternumber")
   public VolumeArgs(String userName, String volumeName, String requestID,
   public VolumeArgs(String userName, String volumeName, String requestID,
                     String hostName, Request request, UriInfo info,
                     String hostName, Request request, UriInfo info,
                     HttpHeaders headers, String[] groups) {
                     HttpHeaders headers, String[] groups) {

+ 6 - 5
hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/acl/TestOzoneObjInfo.java

@@ -48,14 +48,15 @@ public class TestOzoneObjInfo {
     assertEquals(objInfo.getVolumeName(), volume);
     assertEquals(objInfo.getVolumeName(), volume);
   }
   }
 
 
-  private OzoneObjInfo.Builder getBuilder(String volume, String bucket,
-      String key) {
+  private OzoneObjInfo.Builder getBuilder(String withVolume,
+      String withBucket,
+      String withKey) {
     return OzoneObjInfo.Builder.newBuilder()
     return OzoneObjInfo.Builder.newBuilder()
         .setResType(ResourceType.VOLUME)
         .setResType(ResourceType.VOLUME)
         .setStoreType(STORE)
         .setStoreType(STORE)
-        .setVolumeName(volume)
-        .setBucketName(bucket)
-        .setKeyName(key);
+        .setVolumeName(withVolume)
+        .setBucketName(withBucket)
+        .setKeyName(withKey);
   }
   }
 
 
   @Test
   @Test

+ 18 - 0
hadoop-ozone/dev-support/checks/acceptance.sh

@@ -0,0 +1,18 @@
+#!/usr/bin/env bash
+# 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.
+export HADOOP_VERSION=3
+hadoop-ozone/dist/target/ozone-*-SNAPSHOT/smoketest/test.sh
+exit $?

+ 22 - 0
hadoop-ozone/dev-support/checks/author.sh

@@ -0,0 +1,22 @@
+#!/usr/bin/env bash
+# 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.
+mkdir -p ./target
+grep -r --include="*.java" "@author" .
+if [ $? -gt 0 ]; then
+  exit 0
+else
+  exit -1
+fi

+ 18 - 0
hadoop-ozone/dev-support/checks/build.sh

@@ -0,0 +1,18 @@
+#!/usr/bin/env bash
+# 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.
+export MAVEN_OPTS="-Xmx4096m"
+mvn -am -pl :hadoop-ozone-dist -P hdds -Dmaven.javadoc.skip=true -DskipTests clean install
+exit $?

+ 23 - 0
hadoop-ozone/dev-support/checks/checkstyle.sh

@@ -0,0 +1,23 @@
+#!/usr/bin/env bash
+# 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.
+mvn -fn checkstyle:check -am -pl :hadoop-ozone-dist -Phdds
+
+violations=$(grep -r error --include checkstyle-errors.xml | wc -l)
+if [[ $violations -gt 0 ]]; then
+    echo "There are $violations checkstyle violations"
+    exit -1
+fi
+exit 0

+ 35 - 0
hadoop-ozone/dev-support/checks/findbugs.sh

@@ -0,0 +1,35 @@
+#!/usr/bin/env bash
+# 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.
+
+FINDBUGS_ALL_FILE=./target/findbugs-all.txt
+
+mkdir -p ./target
+rm "$FINDBUGS_ALL_FILE" || true
+touch "$FINDBUGS_ALL_FILE"
+
+mvn -fn findbugs:check -Dfindbugs.failOnError=false  -am -pl :hadoop-ozone-dist -Phdds
+
+find hadoop-ozone -name findbugsXml.xml | xargs -n1 convertXmlToText >> "${FINDBUGS_ALL_FILE}"
+find hadoop-hdds -name findbugsXml.xml | xargs -n1 convertXmlToText >> "${FINDBUGS_ALL_FILE}"
+
+
+bugs=$(cat "$FINDBUGS_ALL_FILE" | wc -l)
+
+if [[ ${bugs} -gt 0 ]]; then
+   exit -1
+else
+   exit 0
+fi

+ 24 - 0
hadoop-ozone/dev-support/checks/isolation.sh

@@ -0,0 +1,24 @@
+#!/usr/bin/env bash
+# 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.
+hadooplines=$(git diff --name-only HEAD~1..HEAD | grep -v hadoop-ozone | grep -v hadoop-hdds | wc -l )
+if [ "$hadooplines" == "0" ]; then
+  echo "Only ozone/hdds subprojects are changed"
+  exit 0
+else
+  echo "Main hadoop projects are changed in an ozone patch."
+  echo "Please do it in a HADOOP/HDFS patch and test it with hadoop precommit tests"
+  exit -1
+fi

+ 24 - 0
hadoop-ozone/dev-support/checks/rat.sh

@@ -0,0 +1,24 @@
+#!/usr/bin/env bash
+# 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.
+
+mkdir -p target
+rm target/rat-aggregated.txt
+mvn -fn org.apache.rat:apache-rat-plugin:0.13:check -am -pl :hadoop-ozone-dist -Phdds
+grep -r --include=rat.txt "!????" | tee ./target/rat-aggregated.txt
+if [ "$(cat target/rat-aggregated.txt)" ]; then
+   exit -1
+fi
+

+ 24 - 0
hadoop-ozone/dev-support/checks/unit.sh

@@ -0,0 +1,24 @@
+#!/usr/bin/env bash
+# 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.
+export MAVEN_OPTS="-Xmx4096m"
+mvn -fn test -am -pl :hadoop-ozone-dist -P hdds
+module_failed_tests=$(find "." -name 'TEST*.xml'\
+    | xargs "grep" -l -E "<failure|<error"\
+    | awk -F/ '{sub("'"TEST-JUNIT_TEST_OUTPUT_DIR"'",""); sub(".xml",""); print $NF}')
+if [[ -n "${module_failed_tests}" ]] ; then
+    exit -1
+fi
+exit 0

+ 66 - 0
hadoop-ozone/dev-support/docker/Dockerfile

@@ -0,0 +1,66 @@
+# 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.
+FROM alpine
+RUN apk add --update --no-cache bash alpine-sdk maven grep openjdk8 py-pip rsync procps autoconf automake libtool findutils
+
+#Install real glibc
+RUN apk --no-cache add ca-certificates wget && \
+    wget -q -O /etc/apk/keys/sgerrand.rsa.pub https://alpine-pkgs.sgerrand.com/sgerrand.rsa.pub && \
+    wget https://github.com/sgerrand/alpine-pkg-glibc/releases/download/2.28-r0/glibc-2.28-r0.apk && \
+    apk add glibc-2.28-r0.apk
+
+#Install protobuf
+RUN mkdir -p /usr/local/src/ && \
+    cd /usr/local/src/ && \
+    wget https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz && \
+    tar xvf protobuf-2.5.0.tar.gz && \
+    cd protobuf-2.5.0 && \
+    ./autogen.sh && \
+    ./configure --prefix=/usr && \
+    make && \
+    make install && \
+    protoc --version
+
+#Findbug install
+RUN mkdir -p /opt && \
+    curl -sL https://sourceforge.net/projects/findbugs/files/findbugs/3.0.1/findbugs-3.0.1.tar.gz/download | tar -xz  && \
+     mv findbugs-* /opt/findbugs
+
+#Install apache-ant
+RUN mkdir -p /opt && \
+    curl -sL 'https://www.apache.org/dyn/mirrors/mirrors.cgi?action=download&filename=/ant/binaries/apache-ant-1.10.5-bin.tar.gz' | tar -xz  && \
+       mv apache-ant* /opt/ant
+
+#Install docker-compose
+RUN pip install docker-compose
+
+ENV PATH=$PATH:/opt/findbugs/bin
+
+RUN addgroup -g 1000 default && \
+   for i in $(seq 1 2000); do adduser jenkins$i -u $i -G default -h /tmp/ -H -D; done
+
+#This is a very huge local maven cache. Usually the mvn repository is not safe to be 
+#shared between builds as concurrent installls are not handled very well
+#A simple workaround is to provide all the required 3rd party lib in the docker image
+#It will be cached by docker, and any additional dependency can be downloaded, artifacts
+#can be installed
+USER jenkins1000
+RUN cd /tmp && \
+   git clone --depth=1 https://gitbox.apache.org/repos/asf/hadoop.git -b trunk && \
+   cd /tmp/hadoop && \
+   mvn package dependency:go-offline -DskipTests -P hdds -pl :hadoop-ozone-dist -am && \
+   rm -rf /tmp/.m2/repository/org/apache/hadoop/*hdds* && \
+   rm -rf /tmp/.m2/repository/org/apache/hadoop/*ozone* && \
+   find /tmp/.m2/repository -exec chmod o+wx {} \;

+ 4 - 3
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestNode2PipelineMap.java

@@ -41,6 +41,9 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
     .ReplicationFactor.THREE;
     .ReplicationFactor.THREE;
 
 
+/**
+ * Test for the Node2Pipeline map.
+ */
 public class TestNode2PipelineMap {
 public class TestNode2PipelineMap {
 
 
   private static MiniOzoneCluster cluster;
   private static MiniOzoneCluster cluster;
@@ -80,7 +83,6 @@ public class TestNode2PipelineMap {
     }
     }
   }
   }
 
 
-
   @Test
   @Test
   public void testPipelineMap() throws IOException {
   public void testPipelineMap() throws IOException {
 
 
@@ -90,7 +92,7 @@ public class TestNode2PipelineMap {
     ContainerID cId = ratisContainer.getContainerInfo().containerID();
     ContainerID cId = ratisContainer.getContainerInfo().containerID();
     Assert.assertEquals(1, set.size());
     Assert.assertEquals(1, set.size());
     set.forEach(containerID ->
     set.forEach(containerID ->
-            Assert.assertEquals(containerID, cId));
+        Assert.assertEquals(containerID, cId));
 
 
     List<DatanodeDetails> dns = ratisContainer.getPipeline().getNodes();
     List<DatanodeDetails> dns = ratisContainer.getPipeline().getNodes();
     Assert.assertEquals(3, dns.size());
     Assert.assertEquals(3, dns.size());
@@ -102,7 +104,6 @@ public class TestNode2PipelineMap {
     pipelines.forEach(p -> Assert.assertEquals(p,
     pipelines.forEach(p -> Assert.assertEquals(p,
         ratisContainer.getPipeline().getId()));
         ratisContainer.getPipeline().getId()));
 
 
-
     // Now close the container and it should not show up while fetching
     // Now close the container and it should not show up while fetching
     // containers by pipeline
     // containers by pipeline
     containerManager
     containerManager

+ 3 - 3
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java

@@ -211,7 +211,7 @@ public interface MiniOzoneCluster {
   /**
   /**
    * Builder class for MiniOzoneCluster.
    * Builder class for MiniOzoneCluster.
    */
    */
-  @SuppressWarnings("CheckStyle")
+  @SuppressWarnings("visibilitymodifier")
   abstract class Builder {
   abstract class Builder {
 
 
     protected static final int DEFAULT_HB_INTERVAL_MS = 1000;
     protected static final int DEFAULT_HB_INTERVAL_MS = 1000;
@@ -259,8 +259,8 @@ public interface MiniOzoneCluster {
       return this;
       return this;
     }
     }
 
 
-    public Builder setStartDataNodes(boolean startDataNodes) {
-      this.startDataNodes = startDataNodes;
+    public Builder setStartDataNodes(boolean nodes) {
+      this.startDataNodes = nodes;
       return this;
       return this;
     }
     }
 
 

+ 6 - 6
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java

@@ -254,15 +254,15 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
     datanodeService.stop();
     datanodeService.stop();
     datanodeService.join();
     datanodeService.join();
     // ensure same ports are used across restarts.
     // ensure same ports are used across restarts.
-    Configuration conf = datanodeService.getConf();
+    Configuration config = datanodeService.getConf();
     int currentPort = datanodeService.getDatanodeDetails()
     int currentPort = datanodeService.getDatanodeDetails()
         .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
         .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
-    conf.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
-    conf.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
+    config.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
+    config.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
     int ratisPort = datanodeService.getDatanodeDetails()
     int ratisPort = datanodeService.getDatanodeDetails()
         .getPort(DatanodeDetails.Port.Name.RATIS).getValue();
         .getPort(DatanodeDetails.Port.Name.RATIS).getValue();
-    conf.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
-    conf.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
+    config.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
+    config.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
     hddsDatanodes.remove(i);
     hddsDatanodes.remove(i);
     if (waitForDatanode) {
     if (waitForDatanode) {
       // wait for node to be removed from SCM healthy node list.
       // wait for node to be removed from SCM healthy node list.
@@ -270,7 +270,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
     }
     }
     String[] args = new String[]{};
     String[] args = new String[]{};
     HddsDatanodeService service =
     HddsDatanodeService service =
-        HddsDatanodeService.createHddsDatanodeService(args, conf);
+        HddsDatanodeService.createHddsDatanodeService(args, config);
     hddsDatanodes.add(i, service);
     hddsDatanodes.add(i, service);
     service.start(null);
     service.start(null);
     if (waitForDatanode) {
     if (waitForDatanode) {

+ 2 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java

@@ -84,7 +84,8 @@ public final class OzoneTestUtils {
       CheckedConsumer<BlockID, Exception> consumer,
       CheckedConsumer<BlockID, Exception> consumer,
       List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups) throws Exception {
       List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups) throws Exception {
 
 
-    for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : omKeyLocationInfoGroups) {
+    for (OmKeyLocationInfoGroup omKeyLocationInfoGroup :
+        omKeyLocationInfoGroups) {
       List<OmKeyLocationInfo> omKeyLocationInfos =
       List<OmKeyLocationInfo> omKeyLocationInfos =
           omKeyLocationInfoGroup.getLocationList();
           omKeyLocationInfoGroup.getLocationList();
       for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) {
       for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) {

+ 7 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java

@@ -47,6 +47,9 @@ import java.io.IOException;
 import java.util.List;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeoutException;
 
 
+/**
+ * Test container closing.
+ */
 public class TestCloseContainerByPipeline {
 public class TestCloseContainerByPipeline {
 
 
   private static MiniOzoneCluster cluster;
   private static MiniOzoneCluster cluster;
@@ -232,10 +235,12 @@ public class TestCloseContainerByPipeline {
     }
     }
   }
   }
 
 
-  private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID,
+  private Boolean isContainerClosed(MiniOzoneCluster ozoneCluster,
+      long containerID,
       DatanodeDetails datanode) {
       DatanodeDetails datanode) {
     ContainerData containerData;
     ContainerData containerData;
-    for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes()) {
+    for (HddsDatanodeService datanodeService : ozoneCluster
+        .getHddsDatanodes()) {
       if (datanode.equals(datanodeService.getDatanodeDetails())) {
       if (datanode.equals(datanodeService.getDatanodeDetails())) {
         containerData =
         containerData =
             datanodeService.getDatanodeStateMachine().getContainer()
             datanodeService.getDatanodeStateMachine().getContainer()

+ 2 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java

@@ -149,7 +149,8 @@ public class TestContainerServer {
     XceiverClientSpi client = null;
     XceiverClientSpi client = null;
     String containerName = OzoneUtils.getRequestID();
     String containerName = OzoneUtils.getRequestID();
     try {
     try {
-      final Pipeline pipeline = ContainerTestHelper.createPipeline(numDatanodes);
+      final Pipeline pipeline =
+          ContainerTestHelper.createPipeline(numDatanodes);
       final OzoneConfiguration conf = new OzoneConfiguration();
       final OzoneConfiguration conf = new OzoneConfiguration();
       initConf.accept(pipeline, conf);
       initConf.accept(pipeline, conf);
 
 

+ 1 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneDatanodeShell.java

@@ -67,7 +67,6 @@ import static org.junit.Assert.fail;
  * This test class specified for testing Ozone datanode shell command.
  * This test class specified for testing Ozone datanode shell command.
  */
  */
 @RunWith(value = Parameterized.class)
 @RunWith(value = Parameterized.class)
-
 public class TestOzoneDatanodeShell {
 public class TestOzoneDatanodeShell {
 
 
   private static final Logger LOG =
   private static final Logger LOG =
@@ -100,6 +99,7 @@ public class TestOzoneDatanodeShell {
   }
   }
 
 
   @Parameterized.Parameter
   @Parameterized.Parameter
+  @SuppressWarnings("visibilitymodifier")
   public Class clientProtocol;
   public Class clientProtocol;
   /**
   /**
    * Create a MiniDFSCluster for testing with using distributed Ozone
    * Create a MiniDFSCluster for testing with using distributed Ozone

+ 1 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java

@@ -125,6 +125,7 @@ public class TestOzoneShell {
   }
   }
 
 
   @Parameterized.Parameter
   @Parameterized.Parameter
+  @SuppressWarnings("visibilitymodifier")
   public Class clientProtocol;
   public Class clientProtocol;
   /**
   /**
    * Create a MiniDFSCluster for testing with using distributed Ozone
    * Create a MiniDFSCluster for testing with using distributed Ozone

+ 5 - 4
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java

@@ -64,10 +64,11 @@ public class TestAllocateContainer {
 
 
   @Test
   @Test
   public void testAllocate() throws Exception {
   public void testAllocate() throws Exception {
-    ContainerWithPipeline container = storageContainerLocationClient.allocateContainer(
-        xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(),
-        containerOwner);
+    ContainerWithPipeline container =
+        storageContainerLocationClient.allocateContainer(
+            xceiverClientManager.getType(),
+            xceiverClientManager.getFactor(),
+            containerOwner);
     Assert.assertNotNull(container);
     Assert.assertNotNull(container);
     Assert.assertNotNull(container.getPipeline().getFirstNode());
     Assert.assertNotNull(container.getPipeline().getFirstNode());
 
 

+ 2 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java

@@ -103,7 +103,8 @@ public class TestOzoneRestWithMiniCluster {
     putKey(bucket, keyName, keyData);
     putKey(bucket, keyName, keyData);
   }
   }
 
 
-  private void putKey(OzoneBucket bucket, String keyName, String keyData) throws IOException {
+  private void putKey(OzoneBucket bucket, String keyName, String keyData)
+      throws IOException {
     try (
     try (
         OzoneOutputStream ozoneOutputStream = bucket
         OzoneOutputStream ozoneOutputStream = bucket
             .createKey(keyName, 0, replicationType, replicationFactor);
             .createKey(keyName, 0, replicationType, replicationFactor);

+ 22 - 21
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java

@@ -78,6 +78,7 @@ public class TestBuckets {
     return Arrays.asList(params);
     return Arrays.asList(params);
   }
   }
 
 
+  @SuppressWarnings("visibilitymodifier")
   @Parameterized.Parameter
   @Parameterized.Parameter
   public static Class clientProtocol;
   public static Class clientProtocol;
 
 
@@ -124,7 +125,7 @@ public class TestBuckets {
     runTestCreateBucket(client);
     runTestCreateBucket(client);
   }
   }
 
 
-  static void runTestCreateBucket(ClientProtocol client)
+  static void runTestCreateBucket(ClientProtocol protocol)
       throws IOException {
       throws IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@@ -132,8 +133,8 @@ public class TestBuckets {
         .setQuota("100TB")
         .setQuota("100TB")
         .setAdmin("hdfs")
         .setAdmin("hdfs")
         .build();
         .build();
-    client.createVolume(volumeName, volumeArgs);
-    OzoneVolume vol = client.getVolumeDetails(volumeName);
+    protocol.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = protocol.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
 
 
     // create 10 buckets under same volume
     // create 10 buckets under same volume
@@ -154,7 +155,7 @@ public class TestBuckets {
       // verify the bucket creation time
       // verify the bucket creation time
       assertTrue((bucket.getCreationTime() / 1000) >= (currentTime / 1000));
       assertTrue((bucket.getCreationTime() / 1000) >= (currentTime / 1000));
     }
     }
-    client.close();
+    protocol.close();
 
 
     assertEquals(vol.getName(), volumeName);
     assertEquals(vol.getName(), volumeName);
     assertEquals(vol.getAdmin(), "hdfs");
     assertEquals(vol.getAdmin(), "hdfs");
@@ -179,7 +180,7 @@ public class TestBuckets {
     runTestAddBucketAcls(client);
     runTestAddBucketAcls(client);
   }
   }
 
 
-  static void runTestAddBucketAcls(ClientProtocol client)
+  static void runTestAddBucketAcls(ClientProtocol protocol)
       throws OzoneException, IOException, ParseException {
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@@ -187,8 +188,8 @@ public class TestBuckets {
         .setQuota("100TB")
         .setQuota("100TB")
         .setAdmin("hdfs")
         .setAdmin("hdfs")
         .build();
         .build();
-    client.createVolume(volumeName, volumeArgs);
-    OzoneVolume vol = client.getVolumeDetails(volumeName);
+    protocol.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = protocol.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
     vol.createBucket(bucketName);
     vol.createBucket(bucketName);
@@ -203,7 +204,7 @@ public class TestBuckets {
     // verify if the creation time is missing after update operation
     // verify if the creation time is missing after update operation
     assertTrue(
     assertTrue(
         (updatedBucket.getCreationTime()) / 1000 >= 0);
         (updatedBucket.getCreationTime()) / 1000 >= 0);
-    client.close();
+    protocol.close();
   }
   }
 
 
   @Test
   @Test
@@ -211,7 +212,7 @@ public class TestBuckets {
     runTestRemoveBucketAcls(client);
     runTestRemoveBucketAcls(client);
   }
   }
 
 
-  static void runTestRemoveBucketAcls(ClientProtocol client)
+  static void runTestRemoveBucketAcls(ClientProtocol protocol)
       throws OzoneException, IOException, ParseException {
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@@ -219,8 +220,8 @@ public class TestBuckets {
         .setQuota("100TB")
         .setQuota("100TB")
         .setAdmin("hdfs")
         .setAdmin("hdfs")
         .build();
         .build();
-    client.createVolume(volumeName, volumeArgs);
-    OzoneVolume vol = client.getVolumeDetails(volumeName);
+    protocol.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = protocol.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
     List<OzoneAcl> aclList =
     List<OzoneAcl> aclList =
@@ -239,7 +240,7 @@ public class TestBuckets {
     // verify if the creation time is missing after update operation
     // verify if the creation time is missing after update operation
     assertTrue(
     assertTrue(
         (updatedBucket.getCreationTime() / 1000) >= 0);
         (updatedBucket.getCreationTime() / 1000) >= 0);
-    client.close();
+    protocol.close();
   }
   }
 
 
   @Test
   @Test
@@ -247,7 +248,7 @@ public class TestBuckets {
     runTestDeleteBucket(client);
     runTestDeleteBucket(client);
   }
   }
 
 
-  static void runTestDeleteBucket(ClientProtocol client)
+  static void runTestDeleteBucket(ClientProtocol protocol)
       throws OzoneException, IOException {
       throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@@ -255,8 +256,8 @@ public class TestBuckets {
         .setQuota("100TB")
         .setQuota("100TB")
         .setAdmin("hdfs")
         .setAdmin("hdfs")
         .build();
         .build();
-    client.createVolume(volumeName, volumeArgs);
-    OzoneVolume vol = client.getVolumeDetails(volumeName);
+    protocol.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = protocol.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
     List<OzoneAcl> aclList =
     List<OzoneAcl> aclList =
@@ -274,7 +275,7 @@ public class TestBuckets {
       // must throw
       // must throw
       assertNotNull(ex);
       assertNotNull(ex);
     }
     }
-    client.close();
+    protocol.close();
   }
   }
 
 
   @Test
   @Test
@@ -282,7 +283,7 @@ public class TestBuckets {
     runTestListBucket(client);
     runTestListBucket(client);
   }
   }
 
 
-  static void runTestListBucket(ClientProtocol client)
+  static void runTestListBucket(ClientProtocol protocol)
       throws OzoneException, IOException, ParseException {
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
@@ -290,11 +291,11 @@ public class TestBuckets {
         .setQuota("100TB")
         .setQuota("100TB")
         .setAdmin("hdfs")
         .setAdmin("hdfs")
         .build();
         .build();
-    client.createVolume(volumeName, volumeArgs);
-    OzoneVolume vol = client.getVolumeDetails(volumeName);
+    protocol.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = protocol.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     List<OzoneAcl> aclList =
     List<OzoneAcl> aclList =
-        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+        Arrays.stream(acls).map(OzoneAcl::parseAcl)
             .collect(Collectors.toList());
             .collect(Collectors.toList());
 
 
     long currentTime = Time.now();
     long currentTime = Time.now();
@@ -321,7 +322,7 @@ public class TestBuckets {
     bucketIterator = vol.listBuckets(null, "listbucket-test-3");
     bucketIterator = vol.listBuckets(null, "listbucket-test-3");
     assertEquals(getSize(bucketIterator), 6);
     assertEquals(getSize(bucketIterator), 6);
 
 
-    client.close();
+    protocol.close();
   }
   }
 
 
   private static int getSize(Iterator<? extends OzoneBucket> bucketIterator) {
   private static int getSize(Iterator<? extends OzoneBucket> bucketIterator) {

+ 1 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java

@@ -57,6 +57,7 @@ public class TestBucketsRatis {
   }
   }
 
 
   @Parameterized.Parameter
   @Parameterized.Parameter
+  @SuppressWarnings("visibilitymodifier")
   public static Class clientProtocol;
   public static Class clientProtocol;
 
 
   @BeforeClass
   @BeforeClass

+ 40 - 38
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java

@@ -72,6 +72,7 @@ public class TestVolume {
     return Arrays.asList(params);
     return Arrays.asList(params);
   }
   }
 
 
+  @SuppressWarnings("visibilitymodifier")
   @Parameterized.Parameter
   @Parameterized.Parameter
   public Class clientProtocol;
   public Class clientProtocol;
 
 
@@ -120,7 +121,7 @@ public class TestVolume {
     runTestCreateVolume(client);
     runTestCreateVolume(client);
   }
   }
 
 
-  static void runTestCreateVolume(ClientProtocol client)
+  static void runTestCreateVolume(ClientProtocol clientProtocol)
       throws OzoneException, IOException, ParseException {
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
 
 
@@ -131,8 +132,8 @@ public class TestVolume {
         .setQuota("100TB")
         .setQuota("100TB")
         .setAdmin("hdfs")
         .setAdmin("hdfs")
         .build();
         .build();
-    client.createVolume(volumeName, volumeArgs);
-    OzoneVolume vol = client.getVolumeDetails(volumeName);
+    clientProtocol.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = clientProtocol.getVolumeDetails(volumeName);
 
 
     assertEquals(vol.getName(), volumeName);
     assertEquals(vol.getName(), volumeName);
     assertEquals(vol.getAdmin(), "hdfs");
     assertEquals(vol.getAdmin(), "hdfs");
@@ -147,7 +148,7 @@ public class TestVolume {
     // not use Rule here because the test method is static.
     // not use Rule here because the test method is static.
     try {
     try {
       String invalidVolumeName = "#" + OzoneUtils.getRequestID().toLowerCase();
       String invalidVolumeName = "#" + OzoneUtils.getRequestID().toLowerCase();
-      client.createVolume(invalidVolumeName);
+      clientProtocol.createVolume(invalidVolumeName);
       /*
       /*
       //TODO: RestClient and RpcClient should use HddsClientUtils to verify name
       //TODO: RestClient and RpcClient should use HddsClientUtils to verify name
       fail("Except the volume creation be failed because the"
       fail("Except the volume creation be failed because the"
@@ -163,11 +164,11 @@ public class TestVolume {
     runTestCreateDuplicateVolume(client);
     runTestCreateDuplicateVolume(client);
   }
   }
 
 
-  static void runTestCreateDuplicateVolume(ClientProtocol client)
+  static void runTestCreateDuplicateVolume(ClientProtocol clientProtocol)
       throws OzoneException, IOException {
       throws OzoneException, IOException {
     try {
     try {
-      client.createVolume("testvol");
-      client.createVolume("testvol");
+      clientProtocol.createVolume("testvol");
+      clientProtocol.createVolume("testvol");
       assertFalse(true);
       assertFalse(true);
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       Assert.assertTrue(ioe.getMessage()
       Assert.assertTrue(ioe.getMessage()
@@ -180,11 +181,11 @@ public class TestVolume {
     runTestDeleteVolume(client);
     runTestDeleteVolume(client);
   }
   }
 
 
-  static void runTestDeleteVolume(ClientProtocol client)
+  static void runTestDeleteVolume(ClientProtocol clientProtocol)
       throws OzoneException, IOException {
       throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.createVolume(volumeName);
-    client.deleteVolume(volumeName);
+    clientProtocol.createVolume(volumeName);
+    clientProtocol.deleteVolume(volumeName);
   }
   }
 
 
   @Test
   @Test
@@ -192,13 +193,13 @@ public class TestVolume {
     runTestChangeOwnerOnVolume(client);
     runTestChangeOwnerOnVolume(client);
   }
   }
 
 
-  static void runTestChangeOwnerOnVolume(ClientProtocol client)
+  static void runTestChangeOwnerOnVolume(ClientProtocol clientProtocol)
       throws OzoneException, ParseException, IOException {
       throws OzoneException, ParseException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.createVolume(volumeName);
-    client.getVolumeDetails(volumeName);
-    client.setVolumeOwner(volumeName, "frodo");
-    OzoneVolume newVol = client.getVolumeDetails(volumeName);
+    clientProtocol.createVolume(volumeName);
+    clientProtocol.getVolumeDetails(volumeName);
+    clientProtocol.setVolumeOwner(volumeName, "frodo");
+    OzoneVolume newVol = clientProtocol.getVolumeDetails(volumeName);
     assertEquals(newVol.getOwner(), "frodo");
     assertEquals(newVol.getOwner(), "frodo");
     // verify if the creation time is missing after setting owner operation
     // verify if the creation time is missing after setting owner operation
     assertTrue(newVol.getCreationTime() > 0);
     assertTrue(newVol.getCreationTime() > 0);
@@ -209,12 +210,12 @@ public class TestVolume {
     runTestChangeQuotaOnVolume(client);
     runTestChangeQuotaOnVolume(client);
   }
   }
 
 
-  static void runTestChangeQuotaOnVolume(ClientProtocol client)
+  static void runTestChangeQuotaOnVolume(ClientProtocol clientProtocol)
       throws OzoneException, IOException, ParseException {
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.createVolume(volumeName);
-    client.setVolumeQuota(volumeName, OzoneQuota.parseQuota("1000MB"));
-    OzoneVolume newVol = client.getVolumeDetails(volumeName);
+    clientProtocol.createVolume(volumeName);
+    clientProtocol.setVolumeQuota(volumeName, OzoneQuota.parseQuota("1000MB"));
+    OzoneVolume newVol = clientProtocol.getVolumeDetails(volumeName);
     assertEquals(newVol.getQuota(),
     assertEquals(newVol.getQuota(),
         OzoneQuota.parseQuota("1000MB").sizeInBytes());
         OzoneQuota.parseQuota("1000MB").sizeInBytes());
     // verify if the creation time is missing after setting quota operation
     // verify if the creation time is missing after setting quota operation
@@ -229,14 +230,14 @@ public class TestVolume {
     runTestListVolume(client);
     runTestListVolume(client);
   }
   }
 
 
-  static void runTestListVolume(ClientProtocol client)
+  static void runTestListVolume(ClientProtocol clientProtocol)
       throws OzoneException, IOException {
       throws OzoneException, IOException {
     for (int x = 0; x < 10; x++) {
     for (int x = 0; x < 10; x++) {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
-      client.createVolume(volumeName);
+      clientProtocol.createVolume(volumeName);
     }
     }
 
 
-    List<OzoneVolume> ovols = client.listVolumes(null, null, 100);
+    List<OzoneVolume> ovols = clientProtocol.listVolumes(null, null, 100);
     assertTrue(ovols.size() >= 10);
     assertTrue(ovols.size() >= 10);
   }
   }
 
 
@@ -247,19 +248,19 @@ public class TestVolume {
     runTestListVolumePagination(client);
     runTestListVolumePagination(client);
   }
   }
 
 
-  static void runTestListVolumePagination(ClientProtocol client)
+  static void runTestListVolumePagination(ClientProtocol clientProtocol)
       throws OzoneException, IOException {
       throws OzoneException, IOException {
     final int volCount = 2000;
     final int volCount = 2000;
     final int step = 100;
     final int step = 100;
     for (int x = 0; x < volCount; x++) {
     for (int x = 0; x < volCount; x++) {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
-      client.createVolume(volumeName);
+      clientProtocol.createVolume(volumeName);
     }
     }
     String prevKey = null;
     String prevKey = null;
     int count = 0;
     int count = 0;
     int pagecount = 0;
     int pagecount = 0;
     while (count < volCount) {
     while (count < volCount) {
-      List<OzoneVolume> ovols = client.listVolumes(null, prevKey, step);
+      List<OzoneVolume> ovols = clientProtocol.listVolumes(null, prevKey, step);
       count += ovols.size();
       count += ovols.size();
       prevKey = ovols.get(ovols.size() - 1).getName();
       prevKey = ovols.get(ovols.size() - 1).getName();
       pagecount++;
       pagecount++;
@@ -274,7 +275,7 @@ public class TestVolume {
     runTestListAllVolumes(client);
     runTestListAllVolumes(client);
   }
   }
 
 
-  static void runTestListAllVolumes(ClientProtocol client)
+  static void runTestListAllVolumes(ClientProtocol clientProtocol)
       throws OzoneException, IOException {
       throws OzoneException, IOException {
     final int volCount = 200;
     final int volCount = 200;
     final int step = 10;
     final int step = 10;
@@ -288,15 +289,15 @@ public class TestVolume {
           .setQuota("100TB")
           .setQuota("100TB")
           .setAdmin("hdfs")
           .setAdmin("hdfs")
           .build();
           .build();
-      client.createVolume(volumeName, volumeArgs);
-      OzoneVolume vol = client.getVolumeDetails(volumeName);
+      clientProtocol.createVolume(volumeName, volumeArgs);
+      OzoneVolume vol = clientProtocol.getVolumeDetails(volumeName);
       assertNotNull(vol);
       assertNotNull(vol);
     }
     }
     String prevKey = null;
     String prevKey = null;
     int count = 0;
     int count = 0;
     int pagecount = 0;
     int pagecount = 0;
     while (count < volCount) {
     while (count < volCount) {
-      List<OzoneVolume> ovols = client.listVolumes(null, prevKey, step);
+      List<OzoneVolume> ovols = clientProtocol.listVolumes(null, prevKey, step);
       count += ovols.size();
       count += ovols.size();
       if (ovols.size() > 0) {
       if (ovols.size() > 0) {
         prevKey = ovols.get(ovols.size() - 1).getName();
         prevKey = ovols.get(ovols.size() - 1).getName();
@@ -316,7 +317,7 @@ public class TestVolume {
     runTestListVolumes(client);
     runTestListVolumes(client);
   }
   }
 
 
-  static void runTestListVolumes(ClientProtocol client)
+  static void runTestListVolumes(ClientProtocol clientProtocol)
       throws OzoneException, IOException, ParseException {
       throws OzoneException, IOException, ParseException {
     final int volCount = 20;
     final int volCount = 20;
     final String user1 = "test-user-a";
     final String user1 = "test-user-a";
@@ -342,13 +343,14 @@ public class TestVolume {
           .setQuota("100TB")
           .setQuota("100TB")
           .setAdmin("hdfs")
           .setAdmin("hdfs")
           .build();
           .build();
-      client.createVolume(volumeName, volumeArgs);
-      OzoneVolume vol = client.getVolumeDetails(volumeName);
+      clientProtocol.createVolume(volumeName, volumeArgs);
+      OzoneVolume vol = clientProtocol.getVolumeDetails(volumeName);
       assertNotNull(vol);
       assertNotNull(vol);
     }
     }
 
 
     // list all the volumes belong to user1
     // list all the volumes belong to user1
-    List<OzoneVolume> volumeList = client.listVolumes(user1, null, null, 100);
+    List<OzoneVolume> volumeList =
+        clientProtocol.listVolumes(user1, null, null, 100);
     assertEquals(10, volumeList.size());
     assertEquals(10, volumeList.size());
     // verify the owner name and creation time of volume
     // verify the owner name and creation time of volume
     for (OzoneVolume vol : volumeList) {
     for (OzoneVolume vol : volumeList) {
@@ -358,25 +360,25 @@ public class TestVolume {
     }
     }
 
 
     // test max key parameter of listing volumes
     // test max key parameter of listing volumes
-    volumeList = client.listVolumes(user1, null, null, 2);
+    volumeList = clientProtocol.listVolumes(user1, null, null, 2);
     assertEquals(2, volumeList.size());
     assertEquals(2, volumeList.size());
 
 
     // test prefix parameter of listing volumes
     // test prefix parameter of listing volumes
-    volumeList = client.listVolumes(user1, "test-vol10", null, 10);
+    volumeList = clientProtocol.listVolumes(user1, "test-vol10", null, 10);
     assertTrue(volumeList.size() == 1
     assertTrue(volumeList.size() == 1
         && volumeList.get(0).getName().equals("test-vol10"));
         && volumeList.get(0).getName().equals("test-vol10"));
 
 
-    volumeList = client.listVolumes(user1, "test-vol1", null, 10);
+    volumeList = clientProtocol.listVolumes(user1, "test-vol1", null, 10);
     assertEquals(5, volumeList.size());
     assertEquals(5, volumeList.size());
 
 
     // test start key parameter of listing volumes
     // test start key parameter of listing volumes
-    volumeList = client.listVolumes(user2, null, "test-vol15", 10);
+    volumeList = clientProtocol.listVolumes(user2, null, "test-vol15", 10);
     assertEquals(2, volumeList.size());
     assertEquals(2, volumeList.size());
 
 
     String volumeName;
     String volumeName;
     for (int x = 0; x < volCount; x++) {
     for (int x = 0; x < volCount; x++) {
       volumeName = "test-vol" + x;
       volumeName = "test-vol" + x;
-      client.deleteVolume(volumeName);
+      clientProtocol.deleteVolume(volumeName);
     }
     }
   }
   }
 }
 }

+ 1 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java

@@ -60,6 +60,7 @@ public class TestVolumeRatis {
   }
   }
 
 
   @Parameterized.Parameter
   @Parameterized.Parameter
+  @SuppressWarnings("visibilitymodifier")
   public Class clientProtocol;
   public Class clientProtocol;
 
 
   @BeforeClass
   @BeforeClass

+ 1 - 0
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/Bucket.java

@@ -164,6 +164,7 @@ public interface Bucket {
           + "04:23:30 GMT", required = true, paramType = "header"),
           + "04:23:30 GMT", required = true, paramType = "header"),
       @ApiImplicitParam(name = "Authorization", example = "OZONE", required =
       @ApiImplicitParam(name = "Authorization", example = "OZONE", required =
           true, paramType = "header")})
           true, paramType = "header")})
+  @SuppressWarnings("parameternumber")
   Response listBucket(@PathParam("volume") String volume,
   Response listBucket(@PathParam("volume") String volume,
                       @PathParam("bucket") String bucket,
                       @PathParam("bucket") String bucket,
                       @DefaultValue(Header.OZONE_INFO_QUERY_KEY)
                       @DefaultValue(Header.OZONE_INFO_QUERY_KEY)

+ 1 - 0
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java

@@ -170,6 +170,7 @@ public interface Volume {
           + "04:23:30 GMT", required = true, paramType = "header"),
           + "04:23:30 GMT", required = true, paramType = "header"),
       @ApiImplicitParam(name = "Authorization", example = "OZONE", required =
       @ApiImplicitParam(name = "Authorization", example = "OZONE", required =
           true, paramType = "header")})
           true, paramType = "header")})
+  @SuppressWarnings("parameternumber")
   Response getVolumeInfo(@PathParam("volume") String volume,
   Response getVolumeInfo(@PathParam("volume") String volume,
       @DefaultValue(Header.OZONE_INFO_QUERY_BUCKET)
       @DefaultValue(Header.OZONE_INFO_QUERY_BUCKET)
       @QueryParam(Header.OZONE_INFO_QUERY_TAG) String info,
       @QueryParam(Header.OZONE_INFO_QUERY_TAG) String info,

+ 1 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManager.java

@@ -74,6 +74,6 @@ public interface BucketManager {
    * @throws IOException
    * @throws IOException
    */
    */
   List<OmBucketInfo> listBuckets(String volumeName,
   List<OmBucketInfo> listBuckets(String volumeName,
-                                 String startBucket, String bucketPrefix, int maxNumOfBuckets)
+      String startBucket, String bucketPrefix, int maxNumOfBuckets)
       throws IOException;
       throws IOException;
 }
 }

+ 1 - 2
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStorage.java

@@ -22,10 +22,9 @@ import java.util.Properties;
 import java.util.UUID;
 import java.util.UUID;
 
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.common.Storage;
 import org.apache.hadoop.ozone.common.Storage;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
 
 
 import static org.apache.hadoop.ozone.OzoneConsts.SCM_ID;
 import static org.apache.hadoop.ozone.OzoneConsts.SCM_ID;
 
 

+ 6 - 7
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java

@@ -28,20 +28,19 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.common.BlockGroup;
+import org.apache.hadoop.ozone.om.codec.OmBucketInfoCodec;
+import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
+import org.apache.hadoop.ozone.om.codec.OmMultipartKeyInfoCodec;
+import org.apache.hadoop.ozone.om.codec.OmVolumeArgsCodec;
+import org.apache.hadoop.ozone.om.codec.VolumeListCodec;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.om.codec.OmBucketInfoCodec;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
-import org.apache.hadoop.ozone.om.codec.OmMultipartKeyInfoCodec;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.om.codec.OmVolumeArgsCodec;
-import org.apache.hadoop.ozone.om.codec.VolumeListCodec;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
-import org.apache.hadoop.util.Time;
 import org.apache.hadoop.utils.db.DBStore;
 import org.apache.hadoop.utils.db.DBStore;
 import org.apache.hadoop.utils.db.DBStoreBuilder;
 import org.apache.hadoop.utils.db.DBStoreBuilder;
 import org.apache.hadoop.utils.db.Table;
 import org.apache.hadoop.utils.db.Table;

+ 1 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManager.java

@@ -96,5 +96,5 @@ public interface VolumeManager {
    * @throws IOException
    * @throws IOException
    */
    */
   List<OmVolumeArgs> listVolumes(String userName, String prefix,
   List<OmVolumeArgs> listVolumes(String userName, String prefix,
-                                 String startKey, int maxKeys) throws IOException;
+      String startKey, int maxKeys) throws IOException;
 }
 }

+ 2 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisHelper.java

@@ -43,7 +43,8 @@ import org.slf4j.LoggerFactory;
 /**
 /**
  * Ratis helper methods for OM Ratis server and client.
  * Ratis helper methods for OM Ratis server and client.
  */
  */
-public class OMRatisHelper {
+public final class OMRatisHelper {
+
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
       OMRatisHelper.class);
       OMRatisHelper.class);
 
 

+ 1 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java

@@ -214,6 +214,7 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
   /**
   /**
    * Submits request directly to OM.
    * Submits request directly to OM.
    */
    */
+  @SuppressWarnings("methodlength")
   private OMResponse submitRequestToOM(OMRequest request)
   private OMResponse submitRequestToOM(OMRequest request)
       throws ServiceException {
       throws ServiceException {
     Type cmdType = request.getCmdType();
     Type cmdType = request.getCmdType();

+ 1 - 1
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/SignedChunksInputStream.java

@@ -66,7 +66,7 @@ public class SignedChunksInputStream extends InputStream {
   }
   }
 
 
   @Override
   @Override
-  public int read(byte b[], int off, int len) throws IOException {
+  public int read(byte[] b, int off, int len) throws IOException {
     if (b == null) {
     if (b == null) {
       throw new NullPointerException();
       throw new NullPointerException();
     } else if (off < 0 || len < 0 || len > b.length - off) {
     } else if (off < 0 || len < 0 || len > b.length - off) {

+ 1 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java

@@ -74,6 +74,7 @@ public class BucketEndpoint extends EndpointBase {
    */
    */
   @GET
   @GET
   @SuppressFBWarnings
   @SuppressFBWarnings
+  @SuppressWarnings("parameternumber")
   public Response list(
   public Response list(
       @PathParam("bucket") String bucketName,
       @PathParam("bucket") String bucketName,
       @QueryParam("delimiter") String delimiter,
       @QueryParam("delimiter") String delimiter,

+ 0 - 1
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectMultiDelete.java

@@ -19,7 +19,6 @@
  */
  */
 package org.apache.hadoop.ozone.s3.endpoint;
 package org.apache.hadoop.ozone.s3.endpoint;
 
 
-import javax.ws.rs.core.Response;
 import javax.xml.bind.JAXBException;
 import javax.xml.bind.JAXBException;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.HashSet;

+ 1 - 1
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/audit/parser/common/DatabaseHelper.java

@@ -145,7 +145,7 @@ public final class DatabaseHelper {
       throws Exception {
       throws Exception {
     ArrayList<AuditEntry> listResult = new ArrayList<AuditEntry>();
     ArrayList<AuditEntry> listResult = new ArrayList<AuditEntry>();
     try(FileInputStream fis = new FileInputStream(filePath);
     try(FileInputStream fis = new FileInputStream(filePath);
-        InputStreamReader isr = new InputStreamReader(fis);
+        InputStreamReader isr = new InputStreamReader(fis, "UTF-8");
         BufferedReader bReader = new BufferedReader(isr)) {
         BufferedReader bReader = new BufferedReader(isr)) {
       String currentLine = null;
       String currentLine = null;
       String[] entry = null;
       String[] entry = null;

+ 0 - 1
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java

@@ -62,7 +62,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
 import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import static java.lang.Math.min;
 import static java.lang.Math.min;
-import org.apache.commons.cli.CommandLine;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.lang3.time.DurationFormatUtils;
 import org.apache.commons.lang3.time.DurationFormatUtils;

+ 3 - 0
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkMetadataStoreReads.java

@@ -34,6 +34,9 @@ import static org.apache.hadoop.ozone.genesis.GenesisUtil.CACHE_1GB_TYPE;
 import static org.apache.hadoop.ozone.genesis.GenesisUtil.CLOSED_TYPE;
 import static org.apache.hadoop.ozone.genesis.GenesisUtil.CLOSED_TYPE;
 import static org.apache.hadoop.ozone.genesis.GenesisUtil.DEFAULT_TYPE;
 import static org.apache.hadoop.ozone.genesis.GenesisUtil.DEFAULT_TYPE;
 
 
+/**
+ * Measure metadatastore read performance.
+ */
 @State(Scope.Thread)
 @State(Scope.Thread)
 public class BenchMarkMetadataStoreReads {
 public class BenchMarkMetadataStoreReads {
 
 

+ 3 - 1
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkMetadataStoreWrites.java

@@ -32,10 +32,12 @@ import static org.apache.hadoop.ozone.genesis.GenesisUtil.CACHE_10MB_TYPE;
 import static org.apache.hadoop.ozone.genesis.GenesisUtil.CACHE_1GB_TYPE;
 import static org.apache.hadoop.ozone.genesis.GenesisUtil.CACHE_1GB_TYPE;
 import static org.apache.hadoop.ozone.genesis.GenesisUtil.DEFAULT_TYPE;
 import static org.apache.hadoop.ozone.genesis.GenesisUtil.DEFAULT_TYPE;
 
 
+/**
+ * Measure default metadatastore put performance.
+ */
 @State(Scope.Thread)
 @State(Scope.Thread)
 public class BenchMarkMetadataStoreWrites {
 public class BenchMarkMetadataStoreWrites {
 
 
-
   private static final int DATA_LEN = 1024;
   private static final int DATA_LEN = 1024;
   private static final long MAX_KEYS = 1024 * 10;
   private static final long MAX_KEYS = 1024 * 10;
 
 

+ 5 - 1
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkRocksDbStore.java

@@ -31,6 +31,9 @@ import java.io.IOException;
 import java.nio.charset.Charset;
 import java.nio.charset.Charset;
 import java.nio.file.Paths;
 import java.nio.file.Paths;
 
 
+/**
+ * Benchmark rocksdb store.
+ */
 @State(Scope.Thread)
 @State(Scope.Thread)
 public class BenchMarkRocksDbStore {
 public class BenchMarkRocksDbStore {
   private static final int DATA_LEN = 1024;
   private static final int DATA_LEN = 1024;
@@ -86,7 +89,8 @@ public class BenchMarkRocksDbStore {
     opts.setLevel0SlowdownWritesTrigger(20);
     opts.setLevel0SlowdownWritesTrigger(20);
     opts.setLevel0StopWritesTrigger(40);
     opts.setLevel0StopWritesTrigger(40);
     opts.setTargetFileSizeBase(
     opts.setTargetFileSizeBase(
-        (long) StorageUnit.MB.toBytes(Long.parseLong(maxBytesForLevelBase)) / 10);
+        (long) StorageUnit.MB.toBytes(Long.parseLong(maxBytesForLevelBase))
+            / 10);
     opts.setMaxBackgroundCompactions(8);
     opts.setMaxBackgroundCompactions(8);
     opts.setUseFsync(false);
     opts.setUseFsync(false);
     opts.setBytesPerSync(8388608);
     opts.setBytesPerSync(8388608);