Bläddra i källkod

Merge branch 'trunk' into HDDS-1535

supratimdeka 6 år sedan
förälder
incheckning
9da62f33be
100 ändrade filer med 2718 tillägg och 988 borttagningar
  1. 10 0
      hadoop-common-project/hadoop-common/pom.xml
  2. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableQuantiles.java
  4. 32 30
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/OpenSSLSocketFactory.java
  5. 9 9
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  6. 57 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestOpenSSLSocketFactory.java
  7. 1 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
  8. 39 14
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolClientSideTranslatorPB.java
  9. 5 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolServerSideTranslatorPB.java
  10. 2 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
  11. 13 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java
  12. 23 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java
  13. 1 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
  14. 1 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
  15. 1 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
  16. 4 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
  17. 7 3
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java
  18. 25 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java
  19. 74 4
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TypedTable.java
  20. 56 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/CacheKey.java
  21. 47 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/CacheValue.java
  22. 74 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/EpochEntry.java
  23. 97 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/PartialTableCache.java
  24. 63 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/TableCache.java
  25. 18 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/package-info.java
  26. 1 0
      hadoop-hdds/common/src/main/proto/SCMSecurityProtocol.proto
  27. 4 20
      hadoop-hdds/common/src/main/resources/ozone-default.xml
  28. 81 1
      hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestTypedRDBTableStore.java
  29. 142 0
      hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/cache/TestPartialTableCache.java
  30. 22 0
      hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/cache/package-info.java
  31. 10 26
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java
  32. 19 9
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
  33. 16 3
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
  34. 46 44
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
  35. 74 13
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
  36. 3 17
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfo.java
  37. 23 11
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java
  38. 11 4
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueBlockIterator.java
  39. 7 6
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
  40. 31 30
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
  41. 2 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java
  42. 20 16
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
  43. 104 101
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
  44. 59 57
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
  45. 29 26
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
  46. 3 12
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java
  47. 3 12
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
  48. 109 103
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java
  49. 24 20
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
  50. 45 43
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerCheck.java
  51. 20 9
      hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
  52. 14 5
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
  53. 7 3
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
  54. 7 8
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/SCMContainerManager.java
  55. 4 0
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
  56. 1 0
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
  57. 116 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
  58. 3 70
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java
  59. 4 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
  60. 5 0
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
  61. 72 0
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestHddsServerUtils.java
  62. 43 0
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
  63. 45 2
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java
  64. 0 32
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java
  65. 5 0
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockRatisPipelineProvider.java
  66. 7 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  67. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
  68. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java
  69. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
  70. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  71. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
  72. 3 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
  73. 12 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  74. 42 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
  75. 22 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  76. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  77. 17 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
  78. 18 1
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
  79. 11 3
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
  80. 27 11
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
  81. 1 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
  82. 43 69
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
  83. 2 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
  84. 0 12
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
  85. 42 18
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmOzoneAclMap.java
  86. 89 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java
  87. 6 5
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFileStatus.java
  88. 14 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
  89. 40 7
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
  90. 32 33
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
  91. 16 4
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/IAccessAuthorizer.java
  92. 65 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/OzoneAclConfig.java
  93. 32 5
      hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
  94. 55 24
      hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java
  95. 21 0
      hadoop-ozone/dev-support/intellij/install-runconfigs.sh
  96. 18 0
      hadoop-ozone/dev-support/intellij/log4j.properties
  97. 66 0
      hadoop-ozone/dev-support/intellij/ozone-site.xml
  98. 33 0
      hadoop-ozone/dev-support/intellij/runConfigurations/Datanode.xml
  99. 33 0
      hadoop-ozone/dev-support/intellij/runConfigurations/FreonStandalone.xml
  100. 33 0
      hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManager.xml

+ 10 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -334,6 +334,16 @@
       <artifactId>dnsjava</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.wildfly.openssl</groupId>
+      <artifactId>wildfly-openssl</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.assertj</groupId>
+      <artifactId>assertj-core</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.protocolPB;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.StringInterner;
 
 import java.io.IOException;
 
@@ -91,8 +92,8 @@ public final class PBHelper {
     mtime = proto.getModificationTime();
     atime = proto.getAccessTime();
     permission = convert(proto.getPermission());
-    owner = proto.getOwner();
-    group = proto.getGroup();
+    owner = StringInterner.weakIntern(proto.getOwner());
+    group = StringInterner.weakIntern(proto.getGroup());
     int flags = proto.getFlags();
     FileStatus fileStatus = new FileStatus(length, isdir, blockReplication,
         blocksize, mtime, atime, permission, owner, group, symlink, path,

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableQuantiles.java

@@ -107,7 +107,7 @@ public class MutableQuantiles extends MutableMetric {
     estimator = new SampleQuantiles(quantiles);
 
     this.interval = interval;
-    scheduledTask = scheduler.scheduleAtFixedRate(new RolloverSample(this),
+    scheduledTask = scheduler.scheduleWithFixedDelay(new RolloverSample(this),
         interval, interval, TimeUnit.SECONDS);
   }
 

+ 32 - 30
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/OpenSSLSocketFactory.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.azurebfs.utils;
+package org.apache.hadoop.security.ssl;
 
 import java.io.IOException;
 import java.net.InetAddress;
@@ -42,11 +42,11 @@ import org.wildfly.openssl.SSL;
  * performance.
  *
  */
-public final class SSLSocketFactoryEx extends SSLSocketFactory {
+public final class OpenSSLSocketFactory extends SSLSocketFactory {
 
   /**
    * Default indicates Ordered, preferred OpenSSL, if failed to load then fall
-   * back to Default_JSSE
+   * back to Default_JSSE.
    */
   public enum SSLChannelMode {
     OpenSSL,
@@ -54,9 +54,9 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
     Default_JSSE
   }
 
-  private static SSLSocketFactoryEx instance = null;
+  private static OpenSSLSocketFactory instance = null;
   private static final Logger LOG = LoggerFactory.getLogger(
-      SSLSocketFactoryEx.class);
+      OpenSSLSocketFactory.class);
   private String providerName;
   private SSLContext ctx;
   private String[] ciphers;
@@ -71,7 +71,7 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
   public static synchronized void initializeDefaultFactory(
       SSLChannelMode preferredMode) throws IOException {
     if (instance == null) {
-      instance = new SSLSocketFactoryEx(preferredMode);
+      instance = new OpenSSLSocketFactory(preferredMode);
     }
   }
 
@@ -84,7 +84,7 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
    * @return instance of the SSLSocketFactory, instance must be initialized by
    * initializeDefaultFactory.
    */
-  public static SSLSocketFactoryEx getDefaultFactory() {
+  public static OpenSSLSocketFactory getDefaultFactory() {
     return instance;
   }
 
@@ -92,7 +92,7 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
     OpenSSLProvider.register();
   }
 
-  private SSLSocketFactoryEx(SSLChannelMode preferredChannelMode)
+  private OpenSSLSocketFactory(SSLChannelMode preferredChannelMode)
       throws IOException {
     try {
       initializeSSLContext(preferredChannelMode);
@@ -118,33 +118,35 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
   private void initializeSSLContext(SSLChannelMode preferredChannelMode)
       throws NoSuchAlgorithmException, KeyManagementException {
     switch (preferredChannelMode) {
-      case Default:
-        try {
-          java.util.logging.Logger logger = java.util.logging.Logger.getLogger(SSL.class.getName());
-          logger.setLevel(Level.WARNING);
-          ctx = SSLContext.getInstance("openssl.TLS");
-          ctx.init(null, null, null);
-          // Strong reference needs to be kept to logger until initialization of SSLContext finished (see HADOOP-16174):
-          logger.setLevel(Level.INFO);
-          channelMode = SSLChannelMode.OpenSSL;
-        } catch (NoSuchAlgorithmException e) {
-          LOG.warn("Failed to load OpenSSL. Falling back to the JSSE default.");
-          ctx = SSLContext.getDefault();
-          channelMode = SSLChannelMode.Default_JSSE;
-        }
-        break;
-      case OpenSSL:
+    case Default:
+      try {
+        java.util.logging.Logger logger = java.util.logging.Logger.getLogger(
+                SSL.class.getName());
+        logger.setLevel(Level.WARNING);
         ctx = SSLContext.getInstance("openssl.TLS");
         ctx.init(null, null, null);
+        // Strong reference needs to be kept to logger until initialization of
+        // SSLContext finished (see HADOOP-16174):
+        logger.setLevel(Level.INFO);
         channelMode = SSLChannelMode.OpenSSL;
-        break;
-      case Default_JSSE:
+      } catch (NoSuchAlgorithmException e) {
+        LOG.warn("Failed to load OpenSSL. Falling back to the JSSE default.");
         ctx = SSLContext.getDefault();
         channelMode = SSLChannelMode.Default_JSSE;
-        break;
-      default:
-        throw new AssertionError("Unknown channel mode: "
-            + preferredChannelMode);
+      }
+      break;
+    case OpenSSL:
+      ctx = SSLContext.getInstance("openssl.TLS");
+      ctx.init(null, null, null);
+      channelMode = SSLChannelMode.OpenSSL;
+      break;
+    case Default_JSSE:
+      ctx = SSLContext.getDefault();
+      channelMode = SSLChannelMode.Default_JSSE;
+      break;
+    default:
+      throw new AssertionError("Unknown channel mode: "
+          + preferredChannelMode);
     }
   }
 

+ 9 - 9
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -1904,15 +1904,15 @@
   <name>fs.s3a.change.detection.mode</name>
   <value>server</value>
   <description>
-    Determines how change detection is applied to alert to S3 objects
-    rewritten while being read. Value 'server' indicates to apply the attribute
-    constraint directly on GetObject requests to S3. Value 'client' means to do a
-    client-side comparison of the attribute value returned in the response.  Value
-    'server' would not work with third-party S3 implementations that do not
-    support these constraints on GetObject. Values 'server' and 'client' generate
-    RemoteObjectChangedException when a mismatch is detected.  Value 'warn' works
-    like 'client' but generates only a warning.  Value 'none' will ignore change
-    detection completely.
+    Determines how change detection is applied to alert to inconsistent S3
+    objects read during or after an overwrite. Value 'server' indicates to apply
+    the attribute constraint directly on GetObject requests to S3. Value 'client'
+    means to do a client-side comparison of the attribute value returned in the
+    response.  Value 'server' would not work with third-party S3 implementations
+    that do not support these constraints on GetObject. Values 'server' and
+    'client' generate RemoteObjectChangedException when a mismatch is detected.
+    Value 'warn' works like 'client' but generates only a warning.  Value 'none'
+    will ignore change detection completely.
   </description>
 </property>
 

+ 57 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestOpenSSLSocketFactory.java

@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.security.ssl;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import org.apache.hadoop.util.NativeCodeLoader;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests for {@link OpenSSLSocketFactory}.
+ */
+public class TestOpenSSLSocketFactory {
+
+  @Test
+  public void testOpenSSL() throws IOException {
+    assumeTrue("Unable to load native libraries",
+            NativeCodeLoader.isNativeCodeLoaded());
+    assumeTrue("Build was not compiled with support for OpenSSL",
+            NativeCodeLoader.buildSupportsOpenssl());
+    OpenSSLSocketFactory.initializeDefaultFactory(
+            OpenSSLSocketFactory.SSLChannelMode.OpenSSL);
+    assertThat(OpenSSLSocketFactory.getDefaultFactory()
+            .getProviderName()).contains("openssl");
+  }
+
+  @Test
+  public void testJSEEJava8() throws IOException {
+    assumeTrue("Not running on Java 8",
+            System.getProperty("java.version").startsWith("1.8"));
+    OpenSSLSocketFactory.initializeDefaultFactory(
+            OpenSSLSocketFactory.SSLChannelMode.Default_JSSE);
+    assertThat(Arrays.stream(OpenSSLSocketFactory.getDefaultFactory()
+            .getSupportedCipherSuites())).noneMatch("GCM"::contains);
+  }
+}

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

@@ -178,7 +178,7 @@ public final class HddsUtils {
    * @return {@link SCMSecurityProtocol}
    * @throws IOException
    */
-  public static SCMSecurityProtocol getScmSecurityClient(
+  public static SCMSecurityProtocolClientSideTranslatorPB getScmSecurityClient(
       OzoneConfiguration conf, InetSocketAddress address) throws IOException {
     RPC.setProtocolEngine(conf, SCMSecurityProtocolPB.class,
         ProtobufRpcEngine.class);

+ 39 - 14
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolClientSideTranslatorPB.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.OzoneManagerDetailsProto;
 import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCACertificateRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertificateRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertificateRequestProto.Builder;
 import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetDataNodeCertRequestProto;
@@ -79,18 +80,8 @@ public class SCMSecurityProtocolClientSideTranslatorPB implements
   @Override
   public String getDataNodeCertificate(DatanodeDetailsProto dataNodeDetails,
       String certSignReq) throws IOException {
-    SCMGetDataNodeCertRequestProto.Builder builder =
-        SCMGetDataNodeCertRequestProto
-            .newBuilder()
-            .setCSR(certSignReq)
-            .setDatanodeDetails(dataNodeDetails);
-    try {
-      return rpcProxy
-          .getDataNodeCertificate(NULL_RPC_CONTROLLER, builder.build())
-          .getX509Certificate();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
+    return getDataNodeCertificateChain(dataNodeDetails, certSignReq)
+        .getX509Certificate();
   }
 
   /**
@@ -103,13 +94,25 @@ public class SCMSecurityProtocolClientSideTranslatorPB implements
   @Override
   public String getOMCertificate(OzoneManagerDetailsProto omDetails,
       String certSignReq) throws IOException {
+    return getOMCertChain(omDetails, certSignReq).getX509Certificate();
+  }
+
+  /**
+   * Get SCM signed certificate for OM.
+   *
+   * @param omDetails       - OzoneManager Details.
+   * @param certSignReq     - Certificate signing request.
+   * @return byte[]         - SCM signed certificate.
+   */
+  public SCMGetCertResponseProto getOMCertChain(
+      OzoneManagerDetailsProto omDetails, String certSignReq)
+      throws IOException {
     SCMGetOMCertRequestProto.Builder builder = SCMGetOMCertRequestProto
         .newBuilder()
         .setCSR(certSignReq)
         .setOmDetails(omDetails);
     try {
-      return rpcProxy.getOMCertificate(NULL_RPC_CONTROLLER, builder.build())
-          .getX509Certificate();
+      return rpcProxy.getOMCertificate(NULL_RPC_CONTROLLER, builder.build());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -135,6 +138,28 @@ public class SCMSecurityProtocolClientSideTranslatorPB implements
     }
   }
 
+  /**
+   * Get SCM signed certificate for Datanode.
+   *
+   * @param dnDetails       - Datanode Details.
+   * @param certSignReq     - Certificate signing request.
+   * @return byte[]         - SCM signed certificate.
+   */
+  public SCMGetCertResponseProto getDataNodeCertificateChain(
+      DatanodeDetailsProto dnDetails, String certSignReq)
+      throws IOException {
+    SCMGetDataNodeCertRequestProto.Builder builder =
+        SCMGetDataNodeCertRequestProto.newBuilder()
+            .setCSR(certSignReq)
+            .setDatanodeDetails(dnDetails);
+    try {
+      return rpcProxy.getDataNodeCertificate(NULL_RPC_CONTROLLER,
+          builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   /**
    * Get CA certificate.
    *

+ 5 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolServerSideTranslatorPB.java

@@ -61,7 +61,9 @@ public class SCMSecurityProtocolServerSideTranslatorPB implements
           SCMGetCertResponseProto
               .newBuilder()
               .setResponseCode(ResponseCode.success)
-              .setX509Certificate(certificate);
+              .setX509Certificate(certificate)
+              .setX509CACertificate(impl.getCACertificate());
+
       return builder.build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -87,7 +89,8 @@ public class SCMSecurityProtocolServerSideTranslatorPB implements
           SCMGetCertResponseProto
               .newBuilder()
               .setResponseCode(ResponseCode.success)
-              .setX509Certificate(certificate);
+              .setX509Certificate(certificate)
+              .setX509CACertificate(impl.getCACertificate());
       return builder.build();
     } catch (IOException e) {
       throw new ServiceException(e);

+ 2 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java

@@ -260,7 +260,7 @@ public final class ScmConfigKeys {
   public static final String OZONE_SCM_STALENODE_INTERVAL =
       "ozone.scm.stale.node.interval";
   public static final String OZONE_SCM_STALENODE_INTERVAL_DEFAULT =
-      "90s";
+      "5m";
 
   public static final String OZONE_SCM_HEARTBEAT_RPC_TIMEOUT =
       "ozone.scm.heartbeat.rpc-timeout";
@@ -330,7 +330,7 @@ public final class ScmConfigKeys {
       "ozone.scm.pipeline.destroy.timeout";
 
   public static final String OZONE_SCM_PIPELINE_DESTROY_TIMEOUT_DEFAULT =
-      "300s";
+      "66s";
 
   public static final String OZONE_SCM_PIPELINE_CREATION_INTERVAL =
       "ozone.scm.pipeline.creation.interval";

+ 13 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java

@@ -141,6 +141,19 @@ public interface CertificateClient {
   void storeCertificate(String pemEncodedCert, boolean force)
       throws CertificateException;
 
+  /**
+   * Stores the Certificate  for this client. Don't use this api to add
+   * trusted certificates of others.
+   *
+   * @param pemEncodedCert        - pem encoded X509 Certificate
+   * @param force                 - override any existing file
+   * @param caCert                - Is CA certificate.
+   * @throws CertificateException - on Error.
+   *
+   */
+  void storeCertificate(String pemEncodedCert, boolean force, boolean caCert)
+      throws CertificateException;
+
   /**
    * Stores the trusted chain of certificates.
    *

+ 23 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java

@@ -80,6 +80,7 @@ import static org.apache.hadoop.hdds.security.x509.exceptions.CertificateExcepti
 public abstract class DefaultCertificateClient implements CertificateClient {
 
   private static final String CERT_FILE_NAME_FORMAT = "%s.crt";
+  private static final String CA_CERT_PREFIX = "CA-";
   private final Logger logger;
   private final SecurityConfig securityConfig;
   private final KeyCodec keyCodec;
@@ -452,14 +453,30 @@ public abstract class DefaultCertificateClient implements CertificateClient {
    * Stores the Certificate  for this client. Don't use this api to add trusted
    * certificates of others.
    *
-   * @param pemEncodedCert - pem encoded X509 Certificate
-   * @param force - override any existing file
+   * @param pemEncodedCert        - pem encoded X509 Certificate
+   * @param force                 - override any existing file
    * @throws CertificateException - on Error.
    *
    */
   @Override
   public void storeCertificate(String pemEncodedCert, boolean force)
       throws CertificateException {
+    this.storeCertificate(pemEncodedCert, force, false);
+  }
+
+  /**
+   * Stores the Certificate  for this client. Don't use this api to add trusted
+   * certificates of others.
+   *
+   * @param pemEncodedCert        - pem encoded X509 Certificate
+   * @param force                 - override any existing file
+   * @param caCert                - Is CA certificate.
+   * @throws CertificateException - on Error.
+   *
+   */
+  @Override
+  public void storeCertificate(String pemEncodedCert, boolean force,
+      boolean caCert) throws CertificateException {
     CertificateCodec certificateCodec = new CertificateCodec(securityConfig);
     try {
       Path basePath = securityConfig.getCertificateLocation();
@@ -469,6 +486,10 @@ public abstract class DefaultCertificateClient implements CertificateClient {
       String certName = String.format(CERT_FILE_NAME_FORMAT,
           cert.getSerialNumber().toString());
 
+      if(caCert) {
+        certName = CA_CERT_PREFIX + certName;
+      }
+
       certificateCodec.writeCertificate(basePath, certName,
           pemEncodedCert, force);
       certificateMap.putIfAbsent(cert.getSerialNumber().toString(), cert);

+ 1 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java

@@ -56,7 +56,7 @@ public final class OzoneConfigKeys {
 
   public static final String DFS_CONTAINER_CHUNK_WRITE_SYNC_KEY =
       "dfs.container.chunk.write.sync";
-  public static final boolean DFS_CONTAINER_CHUNK_WRITE_SYNC_DEFAULT = true;
+  public static final boolean DFS_CONTAINER_CHUNK_WRITE_SYNC_DEFAULT = false;
   /**
    * Ratis Port where containers listen to.
    */

+ 1 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java

@@ -52,12 +52,11 @@ public final class OzoneConsts {
   public static final String OZONE_ACL_USER_TYPE = "user";
   public static final String OZONE_ACL_GROUP_TYPE = "group";
   public static final String OZONE_ACL_WORLD_TYPE = "world";
+  public static final String OZONE_ACL_ANONYMOUS_TYPE = "anonymous";
   public static final String OZONE_ACL_IP_TYPE = "ip";
 
   public static final String OZONE_ACL_READ = "r";
   public static final String OZONE_ACL_WRITE = "w";
-  public static final String OZONE_ACL_READ_WRITE = "rw";
-  public static final String OZONE_ACL_WRITE_READ = "wr";
   public static final String OZONE_ACL_DELETE = "d";
   public static final String OZONE_ACL_LIST = "l";
   public static final String OZONE_ACL_ALL = "a";

+ 1 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java

@@ -44,6 +44,7 @@ public interface DBStore extends AutoCloseable {
    */
   Table<byte[], byte[]> getTable(String name) throws IOException;
 
+
   /**
    * Gets an existing TableStore with implicit key/value conversion.
    *

+ 4 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java

@@ -298,4 +298,8 @@ public class RDBStore implements DBStore {
   public File getDbLocation() {
     return dbLocation;
   }
+
+  public CodecRegistry getCodecRegistry() {
+    return codecRegistry;
+  }
 }

+ 7 - 3
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java

@@ -22,6 +22,7 @@ package org.apache.hadoop.utils.db;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSUtil;
 
 import org.rocksdb.ColumnFamilyHandle;
@@ -33,9 +34,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * RocksDB implementation of ozone metadata store.
+ * RocksDB implementation of ozone metadata store. This class should be only
+ * used as part of TypedTable as it's underlying implementation to access the
+ * metadata store content. All other user's using Table should use TypedTable.
  */
-public class RDBTable implements Table<byte[], byte[]> {
+@InterfaceAudience.Private
+class RDBTable implements Table<byte[], byte[]> {
 
 
   private static final Logger LOG =
@@ -52,7 +56,7 @@ public class RDBTable implements Table<byte[], byte[]> {
    * @param handle - ColumnFamily Handle.
    * @param writeOptions - RocksDB write Options.
    */
-  public RDBTable(RocksDB db, ColumnFamilyHandle handle,
+  RDBTable(RocksDB db, ColumnFamilyHandle handle,
       WriteOptions writeOptions) {
     this.db = db;
     this.handle = handle;

+ 25 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java

@@ -21,8 +21,10 @@ package org.apache.hadoop.utils.db;
 
 import java.io.IOException;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.classification.InterfaceStability;
-
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
 /**
  * Interface for key-value store that stores ozone metadata. Ozone metadata is
  * stored as key value pairs, both key and value are arbitrary byte arrays. Each
@@ -97,6 +99,28 @@ public interface Table<KEY, VALUE> extends AutoCloseable {
    */
   String getName() throws IOException;
 
+  /**
+   * Add entry to the table cache.
+   *
+   * If the cacheKey already exists, it will override the entry.
+   * @param cacheKey
+   * @param cacheValue
+   */
+  default void addCacheEntry(CacheKey<KEY> cacheKey,
+      CacheValue<VALUE> cacheValue) {
+    throw new NotImplementedException("addCacheEntry is not implemented");
+  }
+
+  /**
+   * Removes all the entries from the table cache which are having epoch value
+   * less
+   * than or equal to specified epoch value.
+   * @param epoch
+   */
+  default void cleanupCache(long epoch) {
+    throw new NotImplementedException("cleanupCache is not implemented");
+  }
+
   /**
    * Class used to represent the key and value pair of a db entry.
    */

+ 74 - 4
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TypedTable.java

@@ -20,6 +20,12 @@ package org.apache.hadoop.utils.db;
 
 import java.io.IOException;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+import org.apache.hadoop.utils.db.cache.PartialTableCache;
+import org.apache.hadoop.utils.db.cache.TableCache;
+
 /**
  * Strongly typed table implementation.
  * <p>
@@ -31,13 +37,16 @@ import java.io.IOException;
  */
 public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
 
-  private Table<byte[], byte[]> rawTable;
+  private final Table<byte[], byte[]> rawTable;
+
+  private final CodecRegistry codecRegistry;
 
-  private CodecRegistry codecRegistry;
+  private final Class<KEY> keyType;
 
-  private Class<KEY> keyType;
+  private final Class<VALUE> valueType;
+
+  private final TableCache<CacheKey<KEY>, CacheValue<VALUE>> cache;
 
-  private Class<VALUE> valueType;
 
   public TypedTable(
       Table<byte[], byte[]> rawTable,
@@ -47,6 +56,7 @@ public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
     this.codecRegistry = codecRegistry;
     this.keyType = keyType;
     this.valueType = valueType;
+    cache = new PartialTableCache<>();
   }
 
   @Override
@@ -69,8 +79,34 @@ public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
     return rawTable.isEmpty();
   }
 
+  /**
+   * Returns the value mapped to the given key in byte array or returns null
+   * if the key is not found.
+   *
+   * Caller's of this method should use synchronization mechanism, when
+   * accessing. First it will check from cache, if it has entry return the
+   * value, otherwise get from the RocksDB table.
+   *
+   * @param key metadata key
+   * @return VALUE
+   * @throws IOException
+   */
   @Override
   public VALUE get(KEY key) throws IOException {
+    // Here the metadata lock will guarantee that cache is not updated for same
+    // key during get key.
+    CacheValue< VALUE > cacheValue = cache.get(new CacheKey<>(key));
+    if (cacheValue == null) {
+      // If no cache for the table or if it does not exist in cache get from
+      // RocksDB table.
+      return getFromTable(key);
+    } else {
+      // We have a value in cache, return the value.
+      return cacheValue.getValue();
+    }
+  }
+
+  private VALUE getFromTable(KEY key) throws IOException {
     byte[] keyBytes = codecRegistry.asRawData(key);
     byte[] valueBytes = rawTable.get(keyBytes);
     return codecRegistry.asObject(valueBytes, valueType);
@@ -106,6 +142,40 @@ public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
 
   }
 
+  @Override
+  public void addCacheEntry(CacheKey<KEY> cacheKey,
+      CacheValue<VALUE> cacheValue) {
+    // This will override the entry if there is already entry for this key.
+    cache.put(cacheKey, cacheValue);
+  }
+
+
+  @Override
+  public void cleanupCache(long epoch) {
+    cache.cleanup(epoch);
+  }
+
+  @VisibleForTesting
+  TableCache<CacheKey<KEY>, CacheValue<VALUE>> getCache() {
+    return cache;
+  }
+
+  public Table<byte[], byte[]> getRawTable() {
+    return rawTable;
+  }
+
+  public CodecRegistry getCodecRegistry() {
+    return codecRegistry;
+  }
+
+  public Class<KEY> getKeyType() {
+    return keyType;
+  }
+
+  public Class<VALUE> getValueType() {
+    return valueType;
+  }
+
   /**
    * Key value implementation for strongly typed tables.
    */

+ 56 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/CacheKey.java

@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.utils.db.cache;
+
+import java.util.Objects;
+
+/**
+ * CacheKey for the RocksDB table.
+ * @param <KEY>
+ */
+public class CacheKey<KEY> {
+
+  private final KEY key;
+
+  public CacheKey(KEY key) {
+    Objects.requireNonNull(key, "Key Should not be null in CacheKey");
+    this.key = key;
+  }
+
+  public KEY getKey() {
+    return key;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    CacheKey<?> cacheKey = (CacheKey<?>) o;
+    return Objects.equals(key, cacheKey.key);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(key);
+  }
+}

+ 47 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/CacheValue.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.utils.db.cache;
+
+import com.google.common.base.Optional;
+
+/**
+ * CacheValue for the RocksDB Table.
+ * @param <VALUE>
+ */
+public class CacheValue<VALUE> {
+
+  private Optional<VALUE> value;
+  // This value is used for evict entries from cache.
+  // This value is set with ratis transaction context log entry index.
+  private long epoch;
+
+  public CacheValue(Optional<VALUE> value, long epoch) {
+    this.value = value;
+    this.epoch = epoch;
+  }
+
+  public VALUE getValue() {
+    return value.orNull();
+  }
+
+  public long getEpoch() {
+    return epoch;
+  }
+
+}

+ 74 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/EpochEntry.java

@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.utils.db.cache;
+
+import java.util.Objects;
+
+/**
+ * Class used which describes epoch entry. This will be used during deletion
+ * entries from cache for partial table cache.
+ * @param <CACHEKEY>
+ */
+public class EpochEntry<CACHEKEY> implements Comparable<CACHEKEY> {
+
+  private long epoch;
+  private CACHEKEY cachekey;
+
+  EpochEntry(long epoch, CACHEKEY cachekey) {
+    this.epoch = epoch;
+    this.cachekey = cachekey;
+  }
+
+  public long getEpoch() {
+    return epoch;
+  }
+
+  public CACHEKEY getCachekey() {
+    return cachekey;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    EpochEntry<?> that = (EpochEntry<?>) o;
+    return epoch == that.epoch && cachekey == that.cachekey;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(epoch, cachekey);
+  }
+
+  public int compareTo(Object o) {
+    if(this.epoch == ((EpochEntry<?>)o).epoch) {
+      return 0;
+    } else if (this.epoch < ((EpochEntry<?>)o).epoch) {
+      return -1;
+    } else {
+      return 1;
+    }
+  }
+
+}

+ 97 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/PartialTableCache.java

@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.utils.db.cache;
+
+import java.util.Iterator;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Cache implementation for the table, this cache is partial cache, this will
+ * be cleaned up, after entries are flushed to DB.
+ */
+@Private
+@Evolving
+public class PartialTableCache<CACHEKEY extends CacheKey,
+    CACHEVALUE extends CacheValue> implements TableCache<CACHEKEY, CACHEVALUE> {
+
+  private final ConcurrentHashMap<CACHEKEY, CACHEVALUE> cache;
+  private final TreeSet<EpochEntry<CACHEKEY>> epochEntries;
+  private ExecutorService executorService;
+
+
+
+  public PartialTableCache() {
+    cache = new ConcurrentHashMap<>();
+    epochEntries = new TreeSet<>();
+    // Created a singleThreadExecutor, so one cleanup will be running at a
+    // time.
+    ThreadFactory build = new ThreadFactoryBuilder().setDaemon(true)
+        .setNameFormat("PartialTableCache Cleanup Thread - %d").build();
+    executorService = Executors.newSingleThreadExecutor(build);
+
+  }
+
+  @Override
+  public CACHEVALUE get(CACHEKEY cachekey) {
+    return cache.get(cachekey);
+  }
+
+  @Override
+  public void put(CACHEKEY cacheKey, CACHEVALUE value) {
+    cache.put(cacheKey, value);
+    epochEntries.add(new EpochEntry<>(value.getEpoch(), cacheKey));
+  }
+
+  @Override
+  public void cleanup(long epoch) {
+    executorService.submit(() -> evictCache(epoch));
+  }
+
+  @Override
+  public int size() {
+    return cache.size();
+  }
+
+  private void evictCache(long epoch) {
+    EpochEntry<CACHEKEY> currentEntry = null;
+    for (Iterator<EpochEntry<CACHEKEY>> iterator = epochEntries.iterator();
+         iterator.hasNext();) {
+      currentEntry = iterator.next();
+      CACHEKEY cachekey = currentEntry.getCachekey();
+      CacheValue cacheValue = cache.get(cachekey);
+      if (cacheValue.getEpoch() <= epoch) {
+        cache.remove(cachekey);
+        iterator.remove();
+      } else {
+        // If currentEntry epoch is greater than epoch, we have deleted all
+        // entries less than specified epoch. So, we can break.
+        break;
+      }
+    }
+  }
+}

+ 63 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/TableCache.java

@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.utils.db.cache;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Cache used for RocksDB tables.
+ * @param <CACHEKEY>
+ * @param <CACHEVALUE>
+ */
+
+@Private
+@Evolving
+public interface TableCache<CACHEKEY extends CacheKey,
+    CACHEVALUE extends CacheValue> {
+
+  /**
+   * Return the value for the key if it is present, otherwise return null.
+   * @param cacheKey
+   * @return CACHEVALUE
+   */
+  CACHEVALUE get(CACHEKEY cacheKey);
+
+  /**
+   * Add an entry to the cache, if the key already exists it overrides.
+   * @param cacheKey
+   * @param value
+   */
+  void put(CACHEKEY cacheKey, CACHEVALUE value);
+
+  /**
+   * Removes all the entries from the cache which are having epoch value less
+   * than or equal to specified epoch value. For FullTable Cache this is a
+   * do-nothing operation.
+   * @param epoch
+   */
+  void cleanup(long epoch);
+
+  /**
+   * Return the size of the cache.
+   * @return size
+   */
+  int size();
+}

+ 18 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/package-info.java

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

+ 1 - 0
hadoop-hdds/common/src/main/proto/SCMSecurityProtocol.proto

@@ -76,6 +76,7 @@ message SCMGetCertResponseProto {
   }
   required ResponseCode responseCode = 1;
   required string x509Certificate = 2; // Base64 encoded X509 certificate.
+  optional string x509CACertificate = 3; // Base64 encoded CA X509 certificate.
 }
 
 

+ 4 - 20
hadoop-hdds/common/src/main/resources/ozone-default.xml

@@ -54,7 +54,7 @@
   </property>
   <property>
     <name>dfs.container.chunk.write.sync</name>
-    <value>true</value>
+    <value>false</value>
     <tag>OZONE, CONTAINER, MANAGEMENT</tag>
     <description>Determines whether the chunk writes in the container happen as
       sync I/0 or buffered I/O operation.
@@ -540,14 +540,6 @@
       the address of the OM.
     </description>
   </property>
-  <property>
-    <name>ozone.om.group.rights</name>
-    <value>READ_WRITE</value>
-    <tag>OM, SECURITY</tag>
-    <description>
-      Default group permissions in Ozone OM.
-    </description>
-  </property>
   <property>
     <name>ozone.om.handler.count.key</name>
     <value>20</value>
@@ -640,14 +632,6 @@
       of buckets or keys inside each bucket a user can create.
     </description>
   </property>
-  <property>
-    <name>ozone.om.user.rights</name>
-    <value>READ_WRITE</value>
-    <tag>OM, SECURITY</tag>
-    <description>
-      Default user permissions used in OM.
-    </description>
-  </property>
   <property>
     <name>ozone.om.db.dirs</name>
     <value/>
@@ -1052,7 +1036,7 @@
   </property>
   <property>
     <name>ozone.scm.stale.node.interval</name>
-    <value>90s</value>
+    <value>5m</value>
     <tag>OZONE, MANAGEMENT</tag>
     <description>
       The interval for stale node flagging. Please
@@ -1291,7 +1275,7 @@
   </property>
   <property>
     <name>ozone.scm.pipeline.destroy.timeout</name>
-    <value>300s</value>
+    <value>66s</value>
     <tag>OZONE, SCM, PIPELINE</tag>
     <description>
       Once a pipeline is closed, SCM should wait for the above configured time
@@ -1809,7 +1793,7 @@
       assumed.
     </description>
   </property>
-  
+
   <property>
     <name>hdds.block.token.enabled</name>
     <value>false</value>

+ 81 - 1
hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestTypedRDBTableStore.java

@@ -26,10 +26,14 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 
+import com.google.common.base.Optional;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.utils.db.Table.KeyValue;
 
 import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -51,7 +55,7 @@ public class TestTypedRDBTableStore {
       Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
           "First", "Second", "Third",
           "Fourth", "Fifth",
-          "Sixth");
+          "Sixth", "Seven");
   @Rule
   public TemporaryFolder folder = new TemporaryFolder();
   private RDBStore rdbStore = null;
@@ -236,4 +240,80 @@ public class TestTypedRDBTableStore {
       }
     }
   }
+
+  @Test
+  public void testTypedTableWithCache() throws Exception {
+    int iterCount = 10;
+    try (Table<String, String> testTable = createTypedTable(
+        "Seven")) {
+
+      for (int x = 0; x < iterCount; x++) {
+        String key = Integer.toString(x);
+        String value = Integer.toString(x);
+        testTable.addCacheEntry(new CacheKey<>(key),
+            new CacheValue<>(Optional.of(value),
+            x));
+      }
+
+      // As we have added to cache, so get should return value even if it
+      // does not exist in DB.
+      for (int x = 0; x < iterCount; x++) {
+        Assert.assertEquals(Integer.toString(1),
+            testTable.get(Integer.toString(1)));
+      }
+
+    }
+  }
+
+  @Test
+  public void testTypedTableWithCacheWithFewDeletedOperationType()
+      throws Exception {
+    int iterCount = 10;
+    try (Table<String, String> testTable = createTypedTable(
+        "Seven")) {
+
+      for (int x = 0; x < iterCount; x++) {
+        String key = Integer.toString(x);
+        String value = Integer.toString(x);
+        if (x % 2 == 0) {
+          testTable.addCacheEntry(new CacheKey<>(key),
+              new CacheValue<>(Optional.of(value), x));
+        } else {
+          testTable.addCacheEntry(new CacheKey<>(key),
+              new CacheValue<>(Optional.absent(),
+              x));
+        }
+      }
+
+      // As we have added to cache, so get should return value even if it
+      // does not exist in DB.
+      for (int x = 0; x < iterCount; x++) {
+        if (x % 2 == 0) {
+          Assert.assertEquals(Integer.toString(x),
+              testTable.get(Integer.toString(x)));
+        } else {
+          Assert.assertNull(testTable.get(Integer.toString(x)));
+        }
+      }
+
+      testTable.cleanupCache(5);
+
+      GenericTestUtils.waitFor(() ->
+          ((TypedTable<String, String>) testTable).getCache().size() == 4,
+          100, 5000);
+
+
+      //Check remaining values
+      for (int x = 6; x < iterCount; x++) {
+        if (x % 2 == 0) {
+          Assert.assertEquals(Integer.toString(x),
+              testTable.get(Integer.toString(x)));
+        } else {
+          Assert.assertNull(testTable.get(Integer.toString(x)));
+        }
+      }
+
+
+    }
+  }
 }

+ 142 - 0
hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/cache/TestPartialTableCache.java

@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.utils.db.cache;
+
+import java.util.concurrent.CompletableFuture;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Class tests partial table cache.
+ */
+public class TestPartialTableCache {
+  private TableCache<CacheKey<String>, CacheValue<String>> tableCache;
+
+  @Before
+  public void create() {
+    tableCache = new PartialTableCache<>();
+  }
+  @Test
+  public void testPartialTableCache() {
+
+
+    for (int i = 0; i< 10; i++) {
+      tableCache.put(new CacheKey<>(Integer.toString(i)),
+          new CacheValue<>(Optional.of(Integer.toString(i)), i));
+    }
+
+
+    for (int i=0; i < 10; i++) {
+      Assert.assertEquals(Integer.toString(i),
+          tableCache.get(new CacheKey<>(Integer.toString(i))).getValue());
+    }
+
+    // On a full table cache if some one calls cleanup it is a no-op.
+    tableCache.cleanup(4);
+
+    for (int i=5; i < 10; i++) {
+      Assert.assertEquals(Integer.toString(i),
+          tableCache.get(new CacheKey<>(Integer.toString(i))).getValue());
+    }
+  }
+
+
+  @Test
+  public void testPartialTableCacheParallel() throws Exception {
+
+    int totalCount = 0;
+    CompletableFuture<Integer> future =
+        CompletableFuture.supplyAsync(() -> {
+          try {
+            return writeToCache(10, 1, 0);
+          } catch (InterruptedException ex) {
+            fail("writeToCache got interrupt exception");
+          }
+          return 0;
+        });
+    int value = future.get();
+    Assert.assertEquals(10, value);
+
+    totalCount += value;
+
+    future =
+        CompletableFuture.supplyAsync(() -> {
+          try {
+            return writeToCache(10, 11, 100);
+          } catch (InterruptedException ex) {
+            fail("writeToCache got interrupt exception");
+          }
+          return 0;
+        });
+
+    // Check we have first 10 entries in cache.
+    for (int i=1; i <= 10; i++) {
+      Assert.assertEquals(Integer.toString(i),
+          tableCache.get(new CacheKey<>(Integer.toString(i))).getValue());
+    }
+
+    int deleted = 5;
+    // cleanup first 5 entires
+    tableCache.cleanup(deleted);
+
+    value = future.get();
+    Assert.assertEquals(10, value);
+
+    totalCount += value;
+
+    // We should totalCount - deleted entries in cache.
+    final int tc = totalCount;
+    GenericTestUtils.waitFor(() -> (tc - deleted == tableCache.size()), 100,
+        5000);
+
+    // Check if we have remaining entries.
+    for (int i=6; i <= totalCount; i++) {
+      Assert.assertEquals(Integer.toString(i),
+          tableCache.get(new CacheKey<>(Integer.toString(i))).getValue());
+    }
+
+    tableCache.cleanup(10);
+
+    tableCache.cleanup(totalCount);
+
+    // Cleaned up all entries, so cache size should be zero.
+    GenericTestUtils.waitFor(() -> (0 == tableCache.size()), 100,
+        5000);
+  }
+
+  private int writeToCache(int count, int startVal, long sleep)
+      throws InterruptedException {
+    int counter = 1;
+    while (counter <= count){
+      tableCache.put(new CacheKey<>(Integer.toString(startVal)),
+          new CacheValue<>(Optional.of(Integer.toString(startVal)), startVal));
+      startVal++;
+      counter++;
+      Thread.sleep(sleep);
+    }
+    return count;
+  }
+}

+ 22 - 0
hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/cache/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
+ *
+ *      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.
+ *
+ */
+/**
+ * Tests for the DB Cache Utilities.
+ */
+package org.apache.hadoop.utils.db.cache;

+ 10 - 26
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java

@@ -253,25 +253,15 @@ public final class HddsServerUtil {
     //
     // Here we check that staleNodeInterval is at least five times more than the
     // frequency at which the accounting thread is going to run.
-    try {
-      sanitizeUserArgs(staleNodeIntervalMs, heartbeatThreadFrequencyMs,
-          5, 1000);
-    } catch (IllegalArgumentException ex) {
-      LOG.error("Stale Node Interval is cannot be honored due to " +
-              "mis-configured {}. ex:  {}",
-          OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, ex);
-      throw ex;
-    }
+    staleNodeIntervalMs = sanitizeUserArgs(OZONE_SCM_STALENODE_INTERVAL,
+        staleNodeIntervalMs, OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
+        heartbeatThreadFrequencyMs, 5, 1000);
 
     // Make sure that stale node value is greater than configured value that
     // datanodes are going to send HBs.
-    try {
-      sanitizeUserArgs(staleNodeIntervalMs, heartbeatIntervalMs, 3, 1000);
-    } catch (IllegalArgumentException ex) {
-      LOG.error("Stale Node Interval MS is cannot be honored due to " +
-          "mis-configured {}. ex:  {}", HDDS_HEARTBEAT_INTERVAL, ex);
-      throw ex;
-    }
+    staleNodeIntervalMs = sanitizeUserArgs(OZONE_SCM_STALENODE_INTERVAL,
+        staleNodeIntervalMs, HDDS_HEARTBEAT_INTERVAL, heartbeatIntervalMs, 3,
+        1000);
     return staleNodeIntervalMs;
   }
 
@@ -290,16 +280,10 @@ public final class HddsServerUtil {
         OZONE_SCM_DEADNODE_INTERVAL_DEFAULT,
         TimeUnit.MILLISECONDS);
 
-    try {
-      // Make sure that dead nodes Ms is at least twice the time for staleNodes
-      // with a max of 1000 times the staleNodes.
-      sanitizeUserArgs(deadNodeIntervalMs, staleNodeIntervalMs, 2, 1000);
-    } catch (IllegalArgumentException ex) {
-      LOG.error("Dead Node Interval MS is cannot be honored due to " +
-          "mis-configured {}. ex:  {}", OZONE_SCM_STALENODE_INTERVAL, ex);
-      throw ex;
-    }
-    return deadNodeIntervalMs;
+    // Make sure that dead nodes Ms is at least twice the time for staleNodes
+    // with a max of 1000 times the staleNodes.
+    return sanitizeUserArgs(OZONE_SCM_DEADNODE_INTERVAL, deadNodeIntervalMs,
+        OZONE_SCM_STALENODE_INTERVAL, staleNodeIntervalMs, 2, 1000);
   }
 
   /**

+ 19 - 9
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java

@@ -26,7 +26,8 @@ import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.HddsServerUtil;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
@@ -271,16 +272,25 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
     try {
       PKCS10CertificationRequest csr = getCSR(config);
       // TODO: For SCM CA we should fetch certificate from multiple SCMs.
-      SCMSecurityProtocol secureScmClient =
+      SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
           HddsUtils.getScmSecurityClient(config,
               HddsUtils.getScmAddressForSecurityProtocol(config));
-
-      String pemEncodedCert = secureScmClient.getDataNodeCertificate(
-          datanodeDetails.getProtoBufMessage(), getEncodedString(csr));
-      dnCertClient.storeCertificate(pemEncodedCert, true);
-      datanodeDetails.setCertSerialId(getX509Certificate(pemEncodedCert).
-          getSerialNumber().toString());
-      persistDatanodeDetails(datanodeDetails);
+      SCMGetCertResponseProto response = secureScmClient.
+          getDataNodeCertificateChain(datanodeDetails.getProtoBufMessage(),
+              getEncodedString(csr));
+      // Persist certificates.
+      if(response.hasX509CACertificate()) {
+        String pemEncodedCert = response.getX509Certificate();
+        dnCertClient.storeCertificate(pemEncodedCert, true);
+        dnCertClient.storeCertificate(response.getX509CACertificate(), true,
+            true);
+        datanodeDetails.setCertSerialId(getX509Certificate(pemEncodedCert).
+            getSerialNumber().toString());
+        persistDatanodeDetails(datanodeDetails);
+      } else {
+        throw new RuntimeException("Unable to retrieve datanode certificate " +
+            "chain");
+      }
     } catch (IOException | CertificateException e) {
       LOG.error("Error while storing SCM signed certificate.", e);
       throw new RuntimeException(e);

+ 16 - 3
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java

@@ -24,6 +24,7 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Res
 import static org.apache.hadoop.ozone.container.common.impl.ContainerData.CHARSET_ENCODING;
 
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.nio.file.Paths;
 import java.security.MessageDigest;
@@ -35,6 +36,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@@ -51,6 +53,9 @@ import com.google.common.base.Preconditions;
  */
 public final class ContainerUtils {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerUtils.class);
+
   private ContainerUtils() {
     //never constructed.
   }
@@ -198,7 +203,7 @@ public final class ContainerUtils {
         throw new IOException("Unable to overwrite the datanode ID file.");
       }
     } else {
-      if(!path.getParentFile().exists() &&
+      if (!path.getParentFile().exists() &&
           !path.getParentFile().mkdirs()) {
         throw new IOException("Unable to create datanode ID directories.");
       }
@@ -221,8 +226,16 @@ public final class ContainerUtils {
     try {
       return DatanodeIdYaml.readDatanodeIdFile(path);
     } catch (IOException e) {
-      throw new IOException("Failed to parse DatanodeDetails from "
-          + path.getAbsolutePath(), e);
+      LOG.warn("Error loading DatanodeDetails yaml from " +
+          path.getAbsolutePath(), e);
+      // Try to load as protobuf before giving up
+      try (FileInputStream in = new FileInputStream(path)) {
+        return DatanodeDetails.getFromProtoBuf(
+            HddsProtos.DatanodeDetailsProto.parseFrom(in));
+      } catch (IOException io) {
+        throw new IOException("Failed to parse DatanodeDetails from "
+            + path.getAbsolutePath(), io);
+      }
     }
   }
 

+ 46 - 44
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java

@@ -48,7 +48,7 @@ import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -198,52 +198,54 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
     }
 
     int newDeletionBlocks = 0;
-    MetadataStore containerDB = BlockUtils.getDB(containerData, conf);
-    for (Long blk : delTX.getLocalIDList()) {
-      BatchOperation batch = new BatchOperation();
-      byte[] blkBytes = Longs.toByteArray(blk);
-      byte[] blkInfo = containerDB.get(blkBytes);
-      if (blkInfo != null) {
-        byte[] deletingKeyBytes =
-            DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk);
-        byte[] deletedKeyBytes =
-            DFSUtil.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blk);
-        if (containerDB.get(deletingKeyBytes) != null
-            || containerDB.get(deletedKeyBytes) != null) {
-          LOG.debug(String.format(
-              "Ignoring delete for block %d in container %d."
-                  + " Entry already added.", blk, containerId));
-          continue;
-        }
-        // Found the block in container db,
-        // use an atomic update to change its state to deleting.
-        batch.put(deletingKeyBytes, blkInfo);
-        batch.delete(blkBytes);
-        try {
-          containerDB.writeBatch(batch);
-          newDeletionBlocks++;
-          LOG.debug("Transited Block {} to DELETING state in container {}",
-              blk, containerId);
-        } catch (IOException e) {
-          // if some blocks failed to delete, we fail this TX,
-          // without sending this ACK to SCM, SCM will resend the TX
-          // with a certain number of retries.
-          throw new IOException(
-              "Failed to delete blocks for TXID = " + delTX.getTxID(), e);
+    try(ReferenceCountedDB containerDB =
+            BlockUtils.getDB(containerData, conf)) {
+      for (Long blk : delTX.getLocalIDList()) {
+        BatchOperation batch = new BatchOperation();
+        byte[] blkBytes = Longs.toByteArray(blk);
+        byte[] blkInfo = containerDB.getStore().get(blkBytes);
+        if (blkInfo != null) {
+          byte[] deletingKeyBytes =
+              DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk);
+          byte[] deletedKeyBytes =
+              DFSUtil.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blk);
+          if (containerDB.getStore().get(deletingKeyBytes) != null
+              || containerDB.getStore().get(deletedKeyBytes) != null) {
+            LOG.debug(String.format(
+                "Ignoring delete for block %d in container %d."
+                    + " Entry already added.", blk, containerId));
+            continue;
+          }
+          // Found the block in container db,
+          // use an atomic update to change its state to deleting.
+          batch.put(deletingKeyBytes, blkInfo);
+          batch.delete(blkBytes);
+          try {
+            containerDB.getStore().writeBatch(batch);
+            newDeletionBlocks++;
+            LOG.debug("Transited Block {} to DELETING state in container {}",
+                blk, containerId);
+          } catch (IOException e) {
+            // if some blocks failed to delete, we fail this TX,
+            // without sending this ACK to SCM, SCM will resend the TX
+            // with a certain number of retries.
+            throw new IOException(
+                "Failed to delete blocks for TXID = " + delTX.getTxID(), e);
+          }
+        } else {
+          LOG.debug("Block {} not found or already under deletion in"
+              + " container {}, skip deleting it.", blk, containerId);
         }
-      } else {
-        LOG.debug("Block {} not found or already under deletion in"
-            + " container {}, skip deleting it.", blk, containerId);
       }
-    }
 
-    containerDB
-        .put(DFSUtil.string2Bytes(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX),
-            Longs.toByteArray(delTX.getTxID()));
-    containerData
-        .updateDeleteTransactionId(delTX.getTxID());
-    // update pending deletion blocks count in in-memory container status
-    containerData.incrPendingDeletionBlocks(newDeletionBlocks);
+      containerDB.getStore()
+          .put(DFSUtil.string2Bytes(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX),
+              Longs.toByteArray(delTX.getTxID()));
+      containerData
+          .updateDeleteTransactionId(delTX.getTxID());
+      // update pending deletion blocks count in in-memory container status
+      containerData.incrPendingDeletionBlocks(newDeletionBlocks);
+    }
   }
 
   @Override

+ 74 - 13
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java

@@ -28,8 +28,11 @@ import org.apache.hadoop.utils.MetadataStoreBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -92,8 +95,8 @@ public final class ContainerCache extends LRUMap {
       MapIterator iterator = cache.mapIterator();
       while (iterator.hasNext()) {
         iterator.next();
-        MetadataStore db = (MetadataStore) iterator.getValue();
-        closeDB((String)iterator.getKey(), db);
+        ReferenceCountedDB db = (ReferenceCountedDB) iterator.getValue();
+        db.setEvicted(true);
       }
       // reset the cache
       cache.clear();
@@ -107,11 +110,11 @@ public final class ContainerCache extends LRUMap {
    */
   @Override
   protected boolean removeLRU(LinkEntry entry) {
-    MetadataStore db = (MetadataStore) entry.getValue();
+    ReferenceCountedDB db = (ReferenceCountedDB) entry.getValue();
     String dbFile = (String)entry.getKey();
     lock.lock();
     try {
-      closeDB(dbFile, db);
+      db.setEvicted(false);
       return true;
     } catch (Exception e) {
       LOG.error("Eviction for db:{} failed", dbFile, e);
@@ -128,26 +131,30 @@ public final class ContainerCache extends LRUMap {
    * @param containerDBType - DB type of the container.
    * @param containerDBPath - DB path of the container.
    * @param conf - Hadoop Configuration.
-   * @return MetadataStore.
+   * @return ReferenceCountedDB.
    */
-  public MetadataStore getDB(long containerID, String containerDBType,
+  public ReferenceCountedDB getDB(long containerID, String containerDBType,
                              String containerDBPath, Configuration conf)
       throws IOException {
     Preconditions.checkState(containerID >= 0,
         "Container ID cannot be negative.");
     lock.lock();
     try {
-      MetadataStore db = (MetadataStore) this.get(containerDBPath);
+      ReferenceCountedDB db = (ReferenceCountedDB) this.get(containerDBPath);
 
       if (db == null) {
-        db = MetadataStoreBuilder.newBuilder()
+        MetadataStore metadataStore =
+            MetadataStoreBuilder.newBuilder()
             .setDbFile(new File(containerDBPath))
             .setCreateIfMissing(false)
             .setConf(conf)
             .setDBType(containerDBType)
             .build();
+        db = new ReferenceCountedDB(metadataStore, containerDBPath);
         this.put(containerDBPath, db);
       }
+      // increment the reference before returning the object
+      db.incrementReference();
       return db;
     } catch (Exception e) {
       LOG.error("Error opening DB. Container:{} ContainerPath:{}",
@@ -161,16 +168,70 @@ public final class ContainerCache extends LRUMap {
   /**
    * Remove a DB handler from cache.
    *
-   * @param containerPath - path of the container db file.
+   * @param containerDBPath - path of the container db file.
    */
-  public void removeDB(String containerPath) {
+  public void removeDB(String containerDBPath) {
     lock.lock();
     try {
-      MetadataStore db = (MetadataStore)this.get(containerPath);
-      closeDB(containerPath, db);
-      this.remove(containerPath);
+      ReferenceCountedDB db = (ReferenceCountedDB)this.get(containerDBPath);
+      if (db != null) {
+        // marking it as evicted will close the db as well.
+        db.setEvicted(true);
+      }
+      this.remove(containerDBPath);
     } finally {
       lock.unlock();
     }
   }
+
+
+  /**
+   * Class to implement reference counting over instances handed by Container
+   * Cache.
+   */
+  public class ReferenceCountedDB implements Closeable {
+    private final AtomicInteger referenceCount;
+    private final AtomicBoolean isEvicted;
+    private final MetadataStore store;
+    private final String containerDBPath;
+
+    public ReferenceCountedDB(MetadataStore store, String containerDBPath) {
+      this.referenceCount = new AtomicInteger(0);
+      this.isEvicted = new AtomicBoolean(false);
+      this.store = store;
+      this.containerDBPath = containerDBPath;
+    }
+
+    private void incrementReference() {
+      this.referenceCount.incrementAndGet();
+    }
+
+    private void decrementReference() {
+      this.referenceCount.decrementAndGet();
+      cleanup();
+    }
+
+    private void setEvicted(boolean checkNoReferences) {
+      Preconditions.checkState(!checkNoReferences ||
+              (referenceCount.get() == 0),
+          "checkNoReferences:%b, referencount:%d",
+          checkNoReferences, referenceCount.get());
+      isEvicted.set(true);
+      cleanup();
+    }
+
+    private void cleanup() {
+      if (referenceCount.get() == 0 && isEvicted.get() && store != null) {
+        closeDB(containerDBPath, store);
+      }
+    }
+
+    public MetadataStore getStore() {
+      return store;
+    }
+
+    public void close() {
+      decrementReference();
+    }
+  }
 }

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

@@ -39,7 +39,8 @@ public final class VolumeInfo {
   private final StorageType storageType;
 
   // Space usage calculator
-  private VolumeUsage usage;
+  private final VolumeUsage usage;
+
   // Capacity configured. This is useful when we want to
   // limit the visible capacity for tests. If negative, then we just
   // query from the filesystem.
@@ -97,36 +98,21 @@ public final class VolumeInfo {
 
   public long getCapacity() throws IOException {
     if (configuredCapacity < 0) {
-      if (usage == null) {
-        throw new IOException("Volume Usage thread is not running. This error" +
-            " is usually seen during DataNode shutdown.");
-      }
       return usage.getCapacity();
     }
     return configuredCapacity;
   }
 
   public long getAvailable() throws IOException {
-    if (usage == null) {
-      throw new IOException("Volume Usage thread is not running. This error " +
-          "is usually seen during DataNode shutdown.");
-    }
     return usage.getAvailable();
   }
 
   public long getScmUsed() throws IOException {
-    if (usage == null) {
-      throw new IOException("Volume Usage thread is not running. This error " +
-          "is usually seen during DataNode shutdown.");
-    }
     return usage.getScmUsed();
   }
 
   protected void shutdownUsageThread() {
-    if (usage != null) {
-      usage.shutdown();
-    }
-    usage = null;
+    usage.shutdown();
   }
 
   public String getRootDir() {

+ 23 - 11
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.container.common.volume;
 
 import com.google.common.annotations.VisibleForTesting;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CachingGetSpaceUsed;
 import org.apache.hadoop.fs.DF;
@@ -35,6 +36,7 @@ import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.nio.charset.StandardCharsets;
 import java.util.Scanner;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * Class that wraps the space df of the Datanode Volumes used by SCM
@@ -46,7 +48,8 @@ public class VolumeUsage {
   private final File rootDir;
   private final DF df;
   private final File scmUsedFile;
-  private GetSpaceUsed scmUsage;
+  private AtomicReference<GetSpaceUsed> scmUsage;
+  private boolean shutdownComplete;
 
   private static final String DU_CACHE_FILE = "scmUsed";
   private volatile boolean scmUsedSaved = false;
@@ -65,10 +68,11 @@ public class VolumeUsage {
 
   void startScmUsageThread(Configuration conf) throws IOException {
     // get SCM specific df
-    this.scmUsage = new CachingGetSpaceUsed.Builder().setPath(rootDir)
-        .setConf(conf)
-        .setInitialUsed(loadScmUsed())
-        .build();
+    scmUsage = new AtomicReference<>(
+        new CachingGetSpaceUsed.Builder().setPath(rootDir)
+            .setConf(conf)
+            .setInitialUsed(loadScmUsed())
+            .build());
   }
 
   long getCapacity() {
@@ -89,14 +93,18 @@ public class VolumeUsage {
   }
 
   long getScmUsed() throws IOException{
-    return scmUsage.getUsed();
+    return scmUsage.get().getUsed();
   }
 
-  public void shutdown() {
-    saveScmUsed();
+  public synchronized void shutdown() {
+    if (!shutdownComplete) {
+      saveScmUsed();
 
-    if (scmUsage instanceof CachingGetSpaceUsed) {
-      IOUtils.cleanupWithLogger(null, ((CachingGetSpaceUsed) scmUsage));
+      if (scmUsage.get() instanceof CachingGetSpaceUsed) {
+        IOUtils.cleanupWithLogger(
+            null, ((CachingGetSpaceUsed) scmUsage.get()));
+      }
+      shutdownComplete = true;
     }
   }
 
@@ -175,7 +183,11 @@ public class VolumeUsage {
    * Only for testing. Do not use otherwise.
    */
   @VisibleForTesting
+  @SuppressFBWarnings(
+      value = "IS2_INCONSISTENT_SYNC",
+      justification = "scmUsage is an AtomicReference. No additional " +
+          "synchronization is needed.")
   public void setScmUsageForTesting(GetSpaceUsed scmUsageForTest) {
-    this.scmUsage = scmUsageForTest;
+    scmUsage.set(scmUsageForTest);
   }
 }

+ 11 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueBlockIterator.java

@@ -31,11 +31,12 @@ import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocat
 import org.apache.hadoop.utils.MetaStoreIterator;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.apache.hadoop.utils.MetadataStore.KeyValue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 import java.util.NoSuchElementException;
@@ -48,12 +49,14 @@ import java.util.NoSuchElementException;
  * {@link MetadataKeyFilters#getNormalKeyFilter()}
  */
 @InterfaceAudience.Public
-public class KeyValueBlockIterator implements BlockIterator<BlockData> {
+public class KeyValueBlockIterator implements BlockIterator<BlockData>,
+    Closeable {
 
   private static final Logger LOG = LoggerFactory.getLogger(
       KeyValueBlockIterator.class);
 
   private MetaStoreIterator<KeyValue> blockIterator;
+  private final ReferenceCountedDB db;
   private static KeyPrefixFilter defaultBlockFilter = MetadataKeyFilters
       .getNormalKeyFilter();
   private KeyPrefixFilter blockFilter;
@@ -91,9 +94,9 @@ public class KeyValueBlockIterator implements BlockIterator<BlockData> {
         containerData;
     keyValueContainerData.setDbFile(KeyValueContainerLocationUtil
         .getContainerDBFile(metdataPath, containerId));
-    MetadataStore metadataStore = BlockUtils.getDB(keyValueContainerData, new
+    db = BlockUtils.getDB(keyValueContainerData, new
         OzoneConfiguration());
-    blockIterator = metadataStore.iterator();
+    blockIterator = db.getStore().iterator();
     blockFilter = filter;
   }
 
@@ -145,4 +148,8 @@ public class KeyValueBlockIterator implements BlockIterator<BlockData> {
     nextBlock = null;
     blockIterator.seekToLast();
   }
+
+  public void close() {
+    db.close();
+  }
 }

+ 7 - 6
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java

@@ -54,7 +54,6 @@ import org.apache.hadoop.ozone.container.keyvalue.helpers
     .KeyValueContainerLocationUtil;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
-import org.apache.hadoop.utils.MetadataStore;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
@@ -74,6 +73,7 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.UNSUPPORTED_REQUEST;
 
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -349,11 +349,12 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
 
   void compactDB() throws StorageContainerException {
     try {
-      MetadataStore db = BlockUtils.getDB(containerData, config);
-      db.compactDB();
-      LOG.info("Container {} is closed with bcsId {}.",
-          containerData.getContainerID(),
-          containerData.getBlockCommitSequenceId());
+      try(ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
+        db.getStore().compactDB();
+        LOG.info("Container {} is closed with bcsId {}.",
+            containerData.getContainerID(),
+            containerData.getBlockCommitSequenceId());
+      }
     } catch (StorageContainerException ex) {
       throw ex;
     } catch (IOException ex) {

+ 31 - 30
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java

@@ -30,12 +30,12 @@ import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
-import org.apache.hadoop.utils.MetadataStore;
 
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -236,41 +236,42 @@ public class KeyValueContainerCheck {
 
 
     onDiskContainerData.setDbFile(dbFile);
-    MetadataStore db = BlockUtils
-        .getDB(onDiskContainerData, checkConfig);
-
-    iterateBlockDB(db);
+    try(ReferenceCountedDB db =
+            BlockUtils.getDB(onDiskContainerData, checkConfig)) {
+      iterateBlockDB(db);
+    }
   }
 
-  private void iterateBlockDB(MetadataStore db)
+  private void iterateBlockDB(ReferenceCountedDB db)
       throws IOException {
     Preconditions.checkState(db != null);
 
     // get "normal" keys from the Block DB
-    KeyValueBlockIterator kvIter = new KeyValueBlockIterator(containerID,
-        new File(onDiskContainerData.getContainerPath()));
-
-    // ensure there is a chunk file for each key in the DB
-    while (kvIter.hasNext()) {
-      BlockData block = kvIter.nextBlock();
-
-      List<ContainerProtos.ChunkInfo> chunkInfoList = block.getChunks();
-      for (ContainerProtos.ChunkInfo chunk : chunkInfoList) {
-        File chunkFile;
-        chunkFile = ChunkUtils.getChunkFile(onDiskContainerData,
-            ChunkInfo.getFromProtoBuf(chunk));
-
-        if (!chunkFile.exists()) {
-          // concurrent mutation in Block DB? lookup the block again.
-          byte[] bdata = db.get(
-              Longs.toByteArray(block.getBlockID().getLocalID()));
-          if (bdata == null) {
-            LOG.trace("concurrency with delete, ignoring deleted block");
-            break; // skip to next block from kvIter
-          } else {
-            String errorStr = "Missing chunk file "
-                + chunkFile.getAbsolutePath();
-            throw new IOException(errorStr);
+    try(KeyValueBlockIterator kvIter = new KeyValueBlockIterator(containerID,
+        new File(onDiskContainerData.getContainerPath()))) {
+
+      // ensure there is a chunk file for each key in the DB
+      while (kvIter.hasNext()) {
+        BlockData block = kvIter.nextBlock();
+
+        List<ContainerProtos.ChunkInfo> chunkInfoList = block.getChunks();
+        for (ContainerProtos.ChunkInfo chunk : chunkInfoList) {
+          File chunkFile;
+          chunkFile = ChunkUtils.getChunkFile(onDiskContainerData,
+              ChunkInfo.getFromProtoBuf(chunk));
+
+          if (!chunkFile.exists()) {
+            // concurrent mutation in Block DB? lookup the block again.
+            byte[] bdata = db.getStore().get(
+                Longs.toByteArray(block.getBlockID().getLocalID()));
+            if (bdata == null) {
+              LOG.trace("concurrency with delete, ignoring deleted block");
+              break; // skip to next block from kvIter
+            } else {
+              String errorStr = "Missing chunk file "
+                  + chunkFile.getAbsolutePath();
+              throw new IOException(errorStr);
+            }
           }
         }
       }

+ 2 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 
 import java.io.IOException;
 
@@ -66,7 +66,7 @@ public final class BlockUtils {
    * @return MetadataStore handle.
    * @throws StorageContainerException
    */
-  public static MetadataStore getDB(KeyValueContainerData containerData,
+  public static ReferenceCountedDB getDB(KeyValueContainerData containerData,
                                     Configuration conf) throws
       StorageContainerException {
     Preconditions.checkNotNull(containerData);

+ 20 - 16
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.utils.MetadataStoreBuilder;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -174,22 +175,25 @@ public final class KeyValueContainerUtil {
     }
     kvContainerData.setDbFile(dbFile);
 
-    MetadataStore metadata = BlockUtils.getDB(kvContainerData, config);
-    long bytesUsed = 0;
-    List<Map.Entry<byte[], byte[]>> liveKeys = metadata
-        .getRangeKVs(null, Integer.MAX_VALUE,
-            MetadataKeyFilters.getNormalKeyFilter());
-    bytesUsed = liveKeys.parallelStream().mapToLong(e-> {
-      BlockData blockData;
-      try {
-        blockData = BlockUtils.getBlockData(e.getValue());
-        return blockData.getSize();
-      } catch (IOException ex) {
-        return 0L;
-      }
-    }).sum();
-    kvContainerData.setBytesUsed(bytesUsed);
-    kvContainerData.setKeyCount(liveKeys.size());
+    try(ReferenceCountedDB metadata =
+            BlockUtils.getDB(kvContainerData, config)) {
+      long bytesUsed = 0;
+      List<Map.Entry<byte[], byte[]>> liveKeys = metadata.getStore()
+          .getRangeKVs(null, Integer.MAX_VALUE,
+              MetadataKeyFilters.getNormalKeyFilter());
+
+      bytesUsed = liveKeys.parallelStream().mapToLong(e-> {
+        BlockData blockData;
+        try {
+          blockData = BlockUtils.getBlockData(e.getValue());
+          return blockData.getSize();
+        } catch (IOException ex) {
+          return 0L;
+        }
+      }).sum();
+      kvContainerData.setBytesUsed(bytesUsed);
+      kvContainerData.setKeyCount(liveKeys.size());
+    }
   }
 
   /**

+ 104 - 101
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
 import org.apache.hadoop.utils.BatchOperation;
 import org.apache.hadoop.utils.MetadataKeyFilters;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -84,47 +84,47 @@ public class BlockManagerImpl implements BlockManager {
         "cannot be negative");
     // We are not locking the key manager since LevelDb serializes all actions
     // against a single DB. We rely on DB level locking to avoid conflicts.
-    MetadataStore db = BlockUtils.getDB((KeyValueContainerData) container
-        .getContainerData(), config);
+    try(ReferenceCountedDB db = BlockUtils.
+        getDB((KeyValueContainerData) container.getContainerData(), config)) {
+      // This is a post condition that acts as a hint to the user.
+      // Should never fail.
+      Preconditions.checkNotNull(db, "DB cannot be null here");
 
-    // This is a post condition that acts as a hint to the user.
-    // Should never fail.
-    Preconditions.checkNotNull(db, "DB cannot be null here");
+      long bcsId = data.getBlockCommitSequenceId();
+      long containerBCSId = ((KeyValueContainerData) container.
+          getContainerData()).getBlockCommitSequenceId();
 
-    long bcsId = data.getBlockCommitSequenceId();
-    long containerBCSId = ((KeyValueContainerData) container.getContainerData())
-        .getBlockCommitSequenceId();
-
-    // default blockCommitSequenceId for any block is 0. It the putBlock
-    // request is not coming via Ratis(for test scenarios), it will be 0.
-    // In such cases, we should overwrite the block as well
-    if (bcsId != 0) {
-      if (bcsId <= containerBCSId) {
-        // Since the blockCommitSequenceId stored in the db is greater than
-        // equal to blockCommitSequenceId to be updated, it means the putBlock
-        // transaction is reapplied in the ContainerStateMachine on restart.
-        // It also implies that the given block must already exist in the db.
-        // just log and return
-        LOG.warn("blockCommitSequenceId " + containerBCSId
-            + " in the Container Db is greater than" + " the supplied value "
-            + bcsId + " .Ignoring it");
-        return data.getSize();
+      // default blockCommitSequenceId for any block is 0. It the putBlock
+      // request is not coming via Ratis(for test scenarios), it will be 0.
+      // In such cases, we should overwrite the block as well
+      if (bcsId != 0) {
+        if (bcsId <= containerBCSId) {
+          // Since the blockCommitSequenceId stored in the db is greater than
+          // equal to blockCommitSequenceId to be updated, it means the putBlock
+          // transaction is reapplied in the ContainerStateMachine on restart.
+          // It also implies that the given block must already exist in the db.
+          // just log and return
+          LOG.warn("blockCommitSequenceId " + containerBCSId
+              + " in the Container Db is greater than" + " the supplied value "
+              + bcsId + " .Ignoring it");
+          return data.getSize();
+        }
       }
+      // update the blockData as well as BlockCommitSequenceId here
+      BatchOperation batch = new BatchOperation();
+      batch.put(Longs.toByteArray(data.getLocalID()),
+          data.getProtoBufMessage().toByteArray());
+      batch.put(blockCommitSequenceIdKey,
+          Longs.toByteArray(bcsId));
+      db.getStore().writeBatch(batch);
+      container.updateBlockCommitSequenceId(bcsId);
+      // Increment keycount here
+      container.getContainerData().incrKeyCount();
+      LOG.debug(
+          "Block " + data.getBlockID() + " successfully committed with bcsId "
+              + bcsId + " chunk size " + data.getChunks().size());
+      return data.getSize();
     }
-    // update the blockData as well as BlockCommitSequenceId here
-    BatchOperation batch = new BatchOperation();
-    batch.put(Longs.toByteArray(data.getLocalID()),
-        data.getProtoBufMessage().toByteArray());
-    batch.put(blockCommitSequenceIdKey,
-        Longs.toByteArray(bcsId));
-    db.writeBatch(batch);
-    container.updateBlockCommitSequenceId(bcsId);
-    // Increment keycount here
-    container.getContainerData().incrKeyCount();
-    LOG.debug(
-        "Block " + data.getBlockID() + " successfully committed with bcsId "
-            + bcsId + " chunk size " + data.getChunks().size());
-    return data.getSize();
   }
 
   /**
@@ -146,32 +146,33 @@ public class BlockManagerImpl implements BlockManager {
 
     KeyValueContainerData containerData = (KeyValueContainerData) container
         .getContainerData();
-    MetadataStore db = BlockUtils.getDB(containerData, config);
-    // This is a post condition that acts as a hint to the user.
-    // Should never fail.
-    Preconditions.checkNotNull(db, "DB cannot be null here");
+    try(ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
+      // This is a post condition that acts as a hint to the user.
+      // Should never fail.
+      Preconditions.checkNotNull(db, "DB cannot be null here");
 
-    long containerBCSId = containerData.getBlockCommitSequenceId();
-    if (containerBCSId < bcsId) {
-      throw new StorageContainerException(
-          "Unable to find the block with bcsID " + bcsId + " .Container "
-              + container.getContainerData().getContainerID() + " bcsId is "
-              + containerBCSId + ".", UNKNOWN_BCSID);
-    }
-    byte[] kData = db.get(Longs.toByteArray(blockID.getLocalID()));
-    if (kData == null) {
-      throw new StorageContainerException("Unable to find the block." + blockID,
-          NO_SUCH_BLOCK);
-    }
-    ContainerProtos.BlockData blockData =
-        ContainerProtos.BlockData.parseFrom(kData);
-    long id = blockData.getBlockID().getBlockCommitSequenceId();
-    if (id < bcsId) {
-      throw new StorageContainerException(
-          "bcsId " + bcsId + " mismatches with existing block Id "
-              + id + " for block " + blockID + ".", BCSID_MISMATCH);
+      long containerBCSId = containerData.getBlockCommitSequenceId();
+      if (containerBCSId < bcsId) {
+        throw new StorageContainerException(
+            "Unable to find the block with bcsID " + bcsId + " .Container "
+                + container.getContainerData().getContainerID() + " bcsId is "
+                + containerBCSId + ".", UNKNOWN_BCSID);
+      }
+      byte[] kData = db.getStore().get(Longs.toByteArray(blockID.getLocalID()));
+      if (kData == null) {
+        throw new StorageContainerException("Unable to find the block." +
+            blockID, NO_SUCH_BLOCK);
+      }
+      ContainerProtos.BlockData blockData =
+          ContainerProtos.BlockData.parseFrom(kData);
+      long id = blockData.getBlockID().getBlockCommitSequenceId();
+      if (id < bcsId) {
+        throw new StorageContainerException(
+            "bcsId " + bcsId + " mismatches with existing block Id "
+                + id + " for block " + blockID + ".", BCSID_MISMATCH);
+      }
+      return BlockData.getFromProtoBuf(blockData);
     }
-    return BlockData.getFromProtoBuf(blockData);
   }
 
   /**
@@ -187,18 +188,19 @@ public class BlockManagerImpl implements BlockManager {
       throws IOException {
     KeyValueContainerData containerData = (KeyValueContainerData) container
         .getContainerData();
-    MetadataStore db = BlockUtils.getDB(containerData, config);
-    // This is a post condition that acts as a hint to the user.
-    // Should never fail.
-    Preconditions.checkNotNull(db, "DB cannot be null here");
-    byte[] kData = db.get(Longs.toByteArray(blockID.getLocalID()));
-    if (kData == null) {
-      throw new StorageContainerException("Unable to find the block.",
-          NO_SUCH_BLOCK);
+    try(ReferenceCountedDB db = BlockUtils.getDB(containerData, config)) {
+      // This is a post condition that acts as a hint to the user.
+      // Should never fail.
+      Preconditions.checkNotNull(db, "DB cannot be null here");
+      byte[] kData = db.getStore().get(Longs.toByteArray(blockID.getLocalID()));
+      if (kData == null) {
+        throw new StorageContainerException("Unable to find the block.",
+            NO_SUCH_BLOCK);
+      }
+      ContainerProtos.BlockData blockData =
+          ContainerProtos.BlockData.parseFrom(kData);
+      return blockData.getSize();
     }
-    ContainerProtos.BlockData blockData =
-        ContainerProtos.BlockData.parseFrom(kData);
-    return blockData.getSize();
   }
 
   /**
@@ -218,24 +220,24 @@ public class BlockManagerImpl implements BlockManager {
 
     KeyValueContainerData cData = (KeyValueContainerData) container
         .getContainerData();
-    MetadataStore db = BlockUtils.getDB(cData, config);
-    // This is a post condition that acts as a hint to the user.
-    // Should never fail.
-    Preconditions.checkNotNull(db, "DB cannot be null here");
-    // Note : There is a race condition here, since get and delete
-    // are not atomic. Leaving it here since the impact is refusing
-    // to delete a Block which might have just gotten inserted after
-    // the get check.
-    byte[] kKey = Longs.toByteArray(blockID.getLocalID());
-    byte[] kData = db.get(kKey);
-    if (kData == null) {
-      throw new StorageContainerException("Unable to find the block.",
-          NO_SUCH_BLOCK);
+    try(ReferenceCountedDB db = BlockUtils.getDB(cData, config)) {
+      // This is a post condition that acts as a hint to the user.
+      // Should never fail.
+      Preconditions.checkNotNull(db, "DB cannot be null here");
+      // Note : There is a race condition here, since get and delete
+      // are not atomic. Leaving it here since the impact is refusing
+      // to delete a Block which might have just gotten inserted after
+      // the get check.
+      byte[] kKey = Longs.toByteArray(blockID.getLocalID());
+      try {
+        db.getStore().delete(kKey);
+      } catch (IOException e) {
+        throw new StorageContainerException("Unable to find the block.",
+            NO_SUCH_BLOCK);
+      }
+      // Decrement blockcount here
+      container.getContainerData().decrKeyCount();
     }
-    db.delete(kKey);
-
-    // Decrement blockcount here
-    container.getContainerData().decrKeyCount();
   }
 
   /**
@@ -258,18 +260,19 @@ public class BlockManagerImpl implements BlockManager {
     List<BlockData> result = null;
     KeyValueContainerData cData = (KeyValueContainerData) container
         .getContainerData();
-    MetadataStore db = BlockUtils.getDB(cData, config);
-    result = new ArrayList<>();
-    byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
-    List<Map.Entry<byte[], byte[]>> range =
-        db.getSequentialRangeKVs(startKeyInBytes, count,
-            MetadataKeyFilters.getNormalKeyFilter());
-    for (Map.Entry<byte[], byte[]> entry : range) {
-      BlockData value = BlockUtils.getBlockData(entry.getValue());
-      BlockData data = new BlockData(value.getBlockID());
-      result.add(data);
+    try(ReferenceCountedDB db = BlockUtils.getDB(cData, config)) {
+      result = new ArrayList<>();
+      byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
+      List<Map.Entry<byte[], byte[]>> range =
+          db.getStore().getSequentialRangeKVs(startKeyInBytes, count,
+              MetadataKeyFilters.getNormalKeyFilter());
+      for (Map.Entry<byte[], byte[]> entry : range) {
+        BlockData value = BlockUtils.getBlockData(entry.getValue());
+        BlockData data = new BlockData(value.getBlockID());
+        result.add(data);
+      }
+      return result;
     }
-    return result;
   }
 
   /**

+ 59 - 57
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.utils.BackgroundTaskQueue;
 import org.apache.hadoop.utils.BackgroundTaskResult;
 import org.apache.hadoop.utils.BatchOperation;
 import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -185,69 +185,71 @@ public class BlockDeletingService extends BackgroundService{
       ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
       long startTime = Time.monotonicNow();
       // Scan container's db and get list of under deletion blocks
-      MetadataStore meta = BlockUtils.getDB(
-          (KeyValueContainerData) containerData, conf);
-      // # of blocks to delete is throttled
-      KeyPrefixFilter filter =
-          new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
-      List<Map.Entry<byte[], byte[]>> toDeleteBlocks =
-          meta.getSequentialRangeKVs(null, blockLimitPerTask, filter);
-      if (toDeleteBlocks.isEmpty()) {
-        LOG.debug("No under deletion block found in container : {}",
-            containerData.getContainerID());
-      }
+      try (ReferenceCountedDB meta = BlockUtils.getDB(containerData, conf)) {
+        // # of blocks to delete is throttled
+        KeyPrefixFilter filter =
+            new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
+        List<Map.Entry<byte[], byte[]>> toDeleteBlocks =
+            meta.getStore().getSequentialRangeKVs(null, blockLimitPerTask,
+                filter);
+        if (toDeleteBlocks.isEmpty()) {
+          LOG.debug("No under deletion block found in container : {}",
+              containerData.getContainerID());
+        }
 
-      List<String> succeedBlocks = new LinkedList<>();
-      LOG.debug("Container : {}, To-Delete blocks : {}",
-          containerData.getContainerID(), toDeleteBlocks.size());
-      File dataDir = new File(containerData.getChunksPath());
-      if (!dataDir.exists() || !dataDir.isDirectory()) {
-        LOG.error("Invalid container data dir {} : "
-            + "does not exist or not a directory", dataDir.getAbsolutePath());
-        return crr;
-      }
+        List<String> succeedBlocks = new LinkedList<>();
+        LOG.debug("Container : {}, To-Delete blocks : {}",
+            containerData.getContainerID(), toDeleteBlocks.size());
+        File dataDir = new File(containerData.getChunksPath());
+        if (!dataDir.exists() || !dataDir.isDirectory()) {
+          LOG.error("Invalid container data dir {} : "
+              + "does not exist or not a directory", dataDir.getAbsolutePath());
+          return crr;
+        }
 
-      toDeleteBlocks.forEach(entry -> {
-        String blockName = DFSUtil.bytes2String(entry.getKey());
-        LOG.debug("Deleting block {}", blockName);
-        try {
-          ContainerProtos.BlockData data =
-              ContainerProtos.BlockData.parseFrom(entry.getValue());
-          for (ContainerProtos.ChunkInfo chunkInfo : data.getChunksList()) {
-            File chunkFile = dataDir.toPath()
-                .resolve(chunkInfo.getChunkName()).toFile();
-            if (FileUtils.deleteQuietly(chunkFile)) {
-              LOG.debug("block {} chunk {} deleted", blockName,
-                  chunkFile.getAbsolutePath());
+        toDeleteBlocks.forEach(entry -> {
+          String blockName = DFSUtil.bytes2String(entry.getKey());
+          LOG.debug("Deleting block {}", blockName);
+          try {
+            ContainerProtos.BlockData data =
+                ContainerProtos.BlockData.parseFrom(entry.getValue());
+            for (ContainerProtos.ChunkInfo chunkInfo : data.getChunksList()) {
+              File chunkFile = dataDir.toPath()
+                  .resolve(chunkInfo.getChunkName()).toFile();
+              if (FileUtils.deleteQuietly(chunkFile)) {
+                LOG.debug("block {} chunk {} deleted", blockName,
+                    chunkFile.getAbsolutePath());
+              }
             }
+            succeedBlocks.add(blockName);
+          } catch (InvalidProtocolBufferException e) {
+            LOG.error("Failed to parse block info for block {}", blockName, e);
           }
-          succeedBlocks.add(blockName);
-        } catch (InvalidProtocolBufferException e) {
-          LOG.error("Failed to parse block info for block {}", blockName, e);
-        }
-      });
+        });
 
-      // Once files are deleted... replace deleting entries with deleted entries
-      BatchOperation batch = new BatchOperation();
-      succeedBlocks.forEach(entry -> {
-        String blockId =
-            entry.substring(OzoneConsts.DELETING_KEY_PREFIX.length());
-        String deletedEntry = OzoneConsts.DELETED_KEY_PREFIX + blockId;
-        batch.put(DFSUtil.string2Bytes(deletedEntry),
-            DFSUtil.string2Bytes(blockId));
-        batch.delete(DFSUtil.string2Bytes(entry));
-      });
-      meta.writeBatch(batch);
-      // update count of pending deletion blocks in in-memory container status
-      containerData.decrPendingDeletionBlocks(succeedBlocks.size());
+        // Once files are deleted... replace deleting entries with deleted
+        // entries
+        BatchOperation batch = new BatchOperation();
+        succeedBlocks.forEach(entry -> {
+          String blockId =
+              entry.substring(OzoneConsts.DELETING_KEY_PREFIX.length());
+          String deletedEntry = OzoneConsts.DELETED_KEY_PREFIX + blockId;
+          batch.put(DFSUtil.string2Bytes(deletedEntry),
+              DFSUtil.string2Bytes(blockId));
+          batch.delete(DFSUtil.string2Bytes(entry));
+        });
+        meta.getStore().writeBatch(batch);
+        // update count of pending deletion blocks in in-memory container status
+        containerData.decrPendingDeletionBlocks(succeedBlocks.size());
 
-      if (!succeedBlocks.isEmpty()) {
-        LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms",
-            containerData.getContainerID(), succeedBlocks.size(),
-            Time.monotonicNow() - startTime);
+        if (!succeedBlocks.isEmpty()) {
+          LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms",
+              containerData.getContainerID(), succeedBlocks.size(),
+              Time.monotonicNow() - startTime);
+        }
+        crr.addAll(succeedBlocks);
+        return crr;
       }
-      crr.addAll(succeedBlocks);
-      return crr;
     }
 
     @Override

+ 29 - 26
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java

@@ -41,7 +41,7 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.hadoop.utils.MetadataKeyFilters;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -191,33 +191,36 @@ public class ContainerReader implements Runnable {
         KeyValueContainerUtil.parseKVContainerData(kvContainerData, config);
         KeyValueContainer kvContainer = new KeyValueContainer(
             kvContainerData, config);
-        MetadataStore containerDB = BlockUtils.getDB(kvContainerData, config);
-        MetadataKeyFilters.KeyPrefixFilter filter =
-            new MetadataKeyFilters.KeyPrefixFilter()
-                .addFilter(OzoneConsts.DELETING_KEY_PREFIX);
-        int numPendingDeletionBlocks =
-            containerDB.getSequentialRangeKVs(null, Integer.MAX_VALUE, filter)
-                .size();
-        kvContainerData.incrPendingDeletionBlocks(numPendingDeletionBlocks);
-        byte[] delTxnId = containerDB.get(
-            DFSUtil.string2Bytes(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX));
-        if (delTxnId != null) {
-          kvContainerData
-              .updateDeleteTransactionId(Longs.fromByteArray(delTxnId));
-        }
-        // sets the BlockCommitSequenceId.
-        byte[] bcsId = containerDB.get(
-            DFSUtil.string2Bytes(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX));
-        if (bcsId != null) {
-          kvContainerData
-              .updateBlockCommitSequenceId(Longs.fromByteArray(bcsId));
-        }
-        if (kvContainer.getContainerState()
+        try(ReferenceCountedDB containerDB = BlockUtils.getDB(kvContainerData,
+            config)) {
+          MetadataKeyFilters.KeyPrefixFilter filter =
+              new MetadataKeyFilters.KeyPrefixFilter()
+                  .addFilter(OzoneConsts.DELETING_KEY_PREFIX);
+          int numPendingDeletionBlocks =
+              containerDB.getStore().getSequentialRangeKVs(null,
+                  Integer.MAX_VALUE, filter)
+                  .size();
+          kvContainerData.incrPendingDeletionBlocks(numPendingDeletionBlocks);
+          byte[] delTxnId = containerDB.getStore().get(
+              DFSUtil.string2Bytes(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX));
+          if (delTxnId != null) {
+            kvContainerData
+                .updateDeleteTransactionId(Longs.fromByteArray(delTxnId));
+          }
+          // sets the BlockCommitSequenceId.
+          byte[] bcsId = containerDB.getStore().get(DFSUtil.string2Bytes(
+              OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX));
+          if (bcsId != null) {
+            kvContainerData
+                .updateBlockCommitSequenceId(Longs.fromByteArray(bcsId));
+          }
+          if (kvContainer.getContainerState()
             == ContainerProtos.ContainerDataProto.State.OPEN) {
-          // commitSpace for Open Containers relies on usedBytes
-          initializeUsedBytes(kvContainer);
+            // commitSpace for Open Containers relies on usedBytes
+            initializeUsedBytes(kvContainer);
+          }
+          containerSet.addContainer(kvContainer);
         }
-        containerSet.addContainer(kvContainer);
       } else {
         throw new StorageContainerException("Container File is corrupted. " +
             "ContainerType is KeyValueContainer but cast to " +

+ 3 - 12
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java

@@ -29,14 +29,12 @@ import org.junit.rules.TemporaryFolder;
 import org.mockito.Mockito;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.Properties;
 import java.util.UUID;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 /**
  * Unit tests for {@link HddsVolume}.
@@ -134,15 +132,8 @@ public class TestHddsVolume {
     assertTrue("scmUsed cache file should be saved on shutdown",
         scmUsedFile.exists());
 
-    try {
-      // Volume.getAvailable() should fail with IOException
-      // as usage thread is shutdown.
-      volume.getAvailable();
-      fail("HddsVolume#shutdown test failed");
-    } catch (Exception ex) {
-      assertTrue(ex instanceof IOException);
-      assertTrue(ex.getMessage().contains(
-          "Volume Usage thread is not running."));
-    }
+    // Volume.getAvailable() should succeed even when usage thread
+    // is shutdown.
+    volume.getAvailable();
   }
 }

+ 3 - 12
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java

@@ -35,7 +35,6 @@ import static org.apache.hadoop.ozone.container.common.volume.HddsVolume
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import org.junit.After;
 import org.junit.Assert;
@@ -213,18 +212,10 @@ public class TestVolumeSet {
 
     volumeSet.shutdown();
 
-    // Verify that the volumes are shutdown and the volumeUsage is set to null.
+    // Verify that volume usage can be queried during shutdown.
     for (HddsVolume volume : volumesList) {
-      Assert.assertNull(volume.getVolumeInfo().getUsageForTesting());
-      try {
-        // getAvailable() should throw null pointer exception as usage is null.
-        volume.getAvailable();
-        fail("Volume shutdown failed.");
-      } catch (IOException ex) {
-        // Do Nothing. Exception is expected.
-        assertTrue(ex.getMessage().contains(
-            "Volume Usage thread is not running."));
-      }
+      Assert.assertNotNull(volume.getVolumeInfo().getUsageForTesting());
+      volume.getAvailable();
     }
   }
 

+ 109 - 103
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.utils.MetadataKeyFilters;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -109,30 +109,31 @@ public class TestKeyValueBlockIterator {
     createContainerWithBlocks(containerID, normalBlocks, deletedBlocks);
     String containerPath = new File(containerData.getMetadataPath())
         .getParent();
-    KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
-        containerID, new File(containerPath));
-
-    int counter = 0;
-    while(keyValueBlockIterator.hasNext()) {
-      BlockData blockData = keyValueBlockIterator.nextBlock();
-      assertEquals(blockData.getLocalID(), counter++);
-    }
-
-    assertFalse(keyValueBlockIterator.hasNext());
-
-    keyValueBlockIterator.seekToFirst();
-    counter = 0;
-    while(keyValueBlockIterator.hasNext()) {
-      BlockData blockData = keyValueBlockIterator.nextBlock();
-      assertEquals(blockData.getLocalID(), counter++);
-    }
-    assertFalse(keyValueBlockIterator.hasNext());
-
-    try {
-      keyValueBlockIterator.nextBlock();
-    } catch (NoSuchElementException ex) {
-      GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
-          "for ContainerID " + containerID, ex);
+    try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
+        containerID, new File(containerPath))) {
+
+      int counter = 0;
+      while (keyValueBlockIterator.hasNext()) {
+        BlockData blockData = keyValueBlockIterator.nextBlock();
+        assertEquals(blockData.getLocalID(), counter++);
+      }
+
+      assertFalse(keyValueBlockIterator.hasNext());
+
+      keyValueBlockIterator.seekToFirst();
+      counter = 0;
+      while (keyValueBlockIterator.hasNext()) {
+        BlockData blockData = keyValueBlockIterator.nextBlock();
+        assertEquals(blockData.getLocalID(), counter++);
+      }
+      assertFalse(keyValueBlockIterator.hasNext());
+
+      try {
+        keyValueBlockIterator.nextBlock();
+      } catch (NoSuchElementException ex) {
+        GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
+            "for ContainerID " + containerID, ex);
+      }
     }
   }
 
@@ -142,17 +143,18 @@ public class TestKeyValueBlockIterator {
     createContainerWithBlocks(containerID, 2, 0);
     String containerPath = new File(containerData.getMetadataPath())
         .getParent();
-    KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
-        containerID, new File(containerPath));
-    long blockID = 0L;
-    assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
-    assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
-
-    try {
-      keyValueBlockIterator.nextBlock();
-    } catch (NoSuchElementException ex) {
-      GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
-          "for ContainerID " + containerID, ex);
+    try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
+        containerID, new File(containerPath))) {
+      long blockID = 0L;
+      assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
+      assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
+
+      try {
+        keyValueBlockIterator.nextBlock();
+      } catch (NoSuchElementException ex) {
+        GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
+            "for ContainerID " + containerID, ex);
+      }
     }
   }
 
@@ -162,42 +164,41 @@ public class TestKeyValueBlockIterator {
     createContainerWithBlocks(containerID, 2, 0);
     String containerPath = new File(containerData.getMetadataPath())
         .getParent();
-    KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
-        containerID, new File(containerPath));
-    long blockID = 0L;
-
-    // Even calling multiple times hasNext() should not move entry forward.
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
-
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
-
-    keyValueBlockIterator.seekToLast();
-    assertTrue(keyValueBlockIterator.hasNext());
-    assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
-
-    keyValueBlockIterator.seekToFirst();
-    blockID = 0L;
-    assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
-    assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
-
-    try {
-      keyValueBlockIterator.nextBlock();
-    } catch (NoSuchElementException ex) {
-      GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
-          "for ContainerID " + containerID, ex);
+    try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
+        containerID, new File(containerPath))) {
+      long blockID = 0L;
+
+      // Even calling multiple times hasNext() should not move entry forward.
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
+
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
+
+      keyValueBlockIterator.seekToLast();
+      assertTrue(keyValueBlockIterator.hasNext());
+      assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
+
+      keyValueBlockIterator.seekToFirst();
+      blockID = 0L;
+      assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
+      assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
+
+      try {
+        keyValueBlockIterator.nextBlock();
+      } catch (NoSuchElementException ex) {
+        GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
+            "for ContainerID " + containerID, ex);
+      }
     }
-
-
   }
 
   @Test
@@ -208,14 +209,15 @@ public class TestKeyValueBlockIterator {
     createContainerWithBlocks(containerId, normalBlocks, deletedBlocks);
     String containerPath = new File(containerData.getMetadataPath())
         .getParent();
-    KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
+    try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
         containerId, new File(containerPath), MetadataKeyFilters
-        .getDeletingKeyFilter());
+        .getDeletingKeyFilter())) {
 
-    int counter = 5;
-    while(keyValueBlockIterator.hasNext()) {
-      BlockData blockData = keyValueBlockIterator.nextBlock();
-      assertEquals(blockData.getLocalID(), counter++);
+      int counter = 5;
+      while (keyValueBlockIterator.hasNext()) {
+        BlockData blockData = keyValueBlockIterator.nextBlock();
+        assertEquals(blockData.getLocalID(), counter++);
+      }
     }
   }
 
@@ -226,11 +228,12 @@ public class TestKeyValueBlockIterator {
     createContainerWithBlocks(containerId, 0, 5);
     String containerPath = new File(containerData.getMetadataPath())
         .getParent();
-    KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
-        containerId, new File(containerPath));
-    //As all blocks are deleted blocks, blocks does not match with normal key
-    // filter.
-    assertFalse(keyValueBlockIterator.hasNext());
+    try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
+        containerId, new File(containerPath))) {
+      //As all blocks are deleted blocks, blocks does not match with normal key
+      // filter.
+      assertFalse(keyValueBlockIterator.hasNext());
+    }
   }
 
   /**
@@ -251,27 +254,30 @@ public class TestKeyValueBlockIterator {
     container = new KeyValueContainer(containerData, conf);
     container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), UUID
         .randomUUID().toString());
-    MetadataStore metadataStore = BlockUtils.getDB(containerData, conf);
-
-    List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
-    ChunkInfo info = new ChunkInfo("chunkfile", 0, 1024);
-    chunkList.add(info.getProtoBufMessage());
-
-    for (int i=0; i<normalBlocks; i++) {
-      BlockID blockID = new BlockID(containerId, i);
-      BlockData blockData = new BlockData(blockID);
-      blockData.setChunks(chunkList);
-      metadataStore.put(Longs.toByteArray(blockID.getLocalID()), blockData
-          .getProtoBufMessage().toByteArray());
-    }
-
-    for (int i=normalBlocks; i<deletedBlocks; i++) {
-      BlockID blockID = new BlockID(containerId, i);
-      BlockData blockData = new BlockData(blockID);
-      blockData.setChunks(chunkList);
-      metadataStore.put(DFSUtil.string2Bytes(OzoneConsts
-          .DELETING_KEY_PREFIX + blockID.getLocalID()), blockData
-          .getProtoBufMessage().toByteArray());
+    try(ReferenceCountedDB metadataStore = BlockUtils.getDB(containerData,
+        conf)) {
+
+      List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
+      ChunkInfo info = new ChunkInfo("chunkfile", 0, 1024);
+      chunkList.add(info.getProtoBufMessage());
+
+      for (int i = 0; i < normalBlocks; i++) {
+        BlockID blockID = new BlockID(containerId, i);
+        BlockData blockData = new BlockData(blockID);
+        blockData.setChunks(chunkList);
+        metadataStore.getStore().put(Longs.toByteArray(blockID.getLocalID()),
+            blockData
+            .getProtoBufMessage().toByteArray());
+      }
+
+      for (int i = normalBlocks; i < deletedBlocks; i++) {
+        BlockID blockID = new BlockID(containerId, i);
+        BlockData blockData = new BlockData(blockID);
+        blockData.setChunks(chunkList);
+        metadataStore.getStore().put(DFSUtil.string2Bytes(OzoneConsts
+            .DELETING_KEY_PREFIX + blockID.getLocalID()), blockData
+            .getProtoBufMessage().toByteArray());
+      }
     }
   }
 

+ 24 - 20
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 
 import org.junit.Assert;
 import org.junit.Before;
@@ -132,23 +132,24 @@ public class TestKeyValueContainer {
   private void addBlocks(int count) throws Exception {
     long containerId = keyValueContainerData.getContainerID();
 
-    MetadataStore metadataStore = BlockUtils.getDB(keyValueContainer
-        .getContainerData(), conf);
-    for (int i=0; i < count; i++) {
-      // Creating BlockData
-      BlockID blockID = new BlockID(containerId, i);
-      BlockData blockData = new BlockData(blockID);
-      blockData.addMetadata("VOLUME", "ozone");
-      blockData.addMetadata("OWNER", "hdfs");
-      List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
-      ChunkInfo info = new ChunkInfo(String.format("%d.data.%d", blockID
-          .getLocalID(), 0), 0, 1024);
-      chunkList.add(info.getProtoBufMessage());
-      blockData.setChunks(chunkList);
-      metadataStore.put(Longs.toByteArray(blockID.getLocalID()), blockData
-          .getProtoBufMessage().toByteArray());
+    try(ReferenceCountedDB metadataStore = BlockUtils.getDB(keyValueContainer
+        .getContainerData(), conf)) {
+      for (int i = 0; i < count; i++) {
+        // Creating BlockData
+        BlockID blockID = new BlockID(containerId, i);
+        BlockData blockData = new BlockData(blockID);
+        blockData.addMetadata("VOLUME", "ozone");
+        blockData.addMetadata("OWNER", "hdfs");
+        List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
+        ChunkInfo info = new ChunkInfo(String.format("%d.data.%d", blockID
+            .getLocalID(), 0), 0, 1024);
+        chunkList.add(info.getProtoBufMessage());
+        blockData.setChunks(chunkList);
+        metadataStore.getStore().put(Longs.toByteArray(blockID.getLocalID()),
+            blockData
+            .getProtoBufMessage().toByteArray());
+      }
     }
-
   }
 
   @SuppressWarnings("RedundantCast")
@@ -191,9 +192,12 @@ public class TestKeyValueContainer {
 
     int numberOfKeysToWrite = 12;
     //write one few keys to check the key count after import
-    MetadataStore metadataStore = BlockUtils.getDB(keyValueContainerData, conf);
-    for (int i = 0; i < numberOfKeysToWrite; i++) {
-      metadataStore.put(("test" + i).getBytes(UTF_8), "test".getBytes(UTF_8));
+    try(ReferenceCountedDB metadataStore =
+        BlockUtils.getDB(keyValueContainerData, conf)) {
+      for (int i = 0; i < numberOfKeysToWrite; i++) {
+        metadataStore.getStore().put(("test" + i).getBytes(UTF_8),
+            "test".getBytes(UTF_8));
+      }
     }
     BlockUtils.removeDB(keyValueContainerData, conf);
 

+ 45 - 43
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainerCheck.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingP
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -149,48 +149,50 @@ import static org.junit.Assert.assertTrue;
     container = new KeyValueContainer(containerData, conf);
     container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
         UUID.randomUUID().toString());
-    MetadataStore metadataStore = BlockUtils.getDB(containerData, conf);
-    chunkManager = new ChunkManagerImpl(true);
-
-    assertTrue(containerData.getChunksPath() != null);
-    File chunksPath = new File(containerData.getChunksPath());
-    assertTrue(chunksPath.exists());
-    // Initially chunks folder should be empty.
-    assertTrue(chunksPath.listFiles().length == 0);
-
-    List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
-    for (int i = 0; i < (totalBlks); i++) {
-      BlockID blockID = new BlockID(containerId, i);
-      BlockData blockData = new BlockData(blockID);
-
-      chunkList.clear();
-      for (chunkCount = 0; chunkCount < chunksPerBlock; chunkCount++) {
-        String chunkName = strBlock + i + strChunk + chunkCount;
-        long offset = chunkCount * chunkLen;
-        ChunkInfo info = new ChunkInfo(chunkName, offset, chunkLen);
-        chunkList.add(info.getProtoBufMessage());
-        chunkManager
-            .writeChunk(container, blockID, info, ByteBuffer.wrap(chunkData),
-                new DispatcherContext.Builder()
-                    .setStage(DispatcherContext.WriteChunkStage.WRITE_DATA)
-                    .build());
-        chunkManager
-            .writeChunk(container, blockID, info, ByteBuffer.wrap(chunkData),
-                new DispatcherContext.Builder()
-                    .setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA)
-                    .build());
-      }
-      blockData.setChunks(chunkList);
-
-      if (i >= normalBlocks) {
-        // deleted key
-        metadataStore.put(DFSUtil.string2Bytes(
-            OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID()),
-            blockData.getProtoBufMessage().toByteArray());
-      } else {
-        // normal key
-        metadataStore.put(Longs.toByteArray(blockID.getLocalID()),
-            blockData.getProtoBufMessage().toByteArray());
+    try (ReferenceCountedDB metadataStore = BlockUtils.getDB(containerData,
+        conf)) {
+      chunkManager = new ChunkManagerImpl(true);
+
+      assertTrue(containerData.getChunksPath() != null);
+      File chunksPath = new File(containerData.getChunksPath());
+      assertTrue(chunksPath.exists());
+      // Initially chunks folder should be empty.
+      assertTrue(chunksPath.listFiles().length == 0);
+
+      List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
+      for (int i = 0; i < (totalBlks); i++) {
+        BlockID blockID = new BlockID(containerId, i);
+        BlockData blockData = new BlockData(blockID);
+
+        chunkList.clear();
+        for (chunkCount = 0; chunkCount < chunksPerBlock; chunkCount++) {
+          String chunkName = strBlock + i + strChunk + chunkCount;
+          long offset = chunkCount * chunkLen;
+          ChunkInfo info = new ChunkInfo(chunkName, offset, chunkLen);
+          chunkList.add(info.getProtoBufMessage());
+          chunkManager
+              .writeChunk(container, blockID, info, ByteBuffer.wrap(chunkData),
+                  new DispatcherContext.Builder()
+                      .setStage(DispatcherContext.WriteChunkStage.WRITE_DATA)
+                      .build());
+          chunkManager
+              .writeChunk(container, blockID, info, ByteBuffer.wrap(chunkData),
+                  new DispatcherContext.Builder()
+                      .setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA)
+                      .build());
+        }
+        blockData.setChunks(chunkList);
+
+        if (i >= normalBlocks) {
+          // deleted key
+          metadataStore.getStore().put(DFSUtil.string2Bytes(
+              OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID()),
+              blockData.getProtoBufMessage().toByteArray());
+        } else {
+          // normal key
+          metadataStore.getStore().put(Longs.toByteArray(blockID.getLocalID()),
+              blockData.getProtoBufMessage().toByteArray());
+        }
       }
     }
   }

+ 20 - 9
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java

@@ -47,23 +47,34 @@ public final class ServerUtils {
    * For example, sanitizeUserArgs(17, 3, 5, 10)
    * ensures that 17 is greater/equal than 3 * 5 and less/equal to 3 * 10.
    *
+   * @param key           - config key of the value
    * @param valueTocheck  - value to check
+   * @param baseKey       - config key of the baseValue
    * @param baseValue     - the base value that is being used.
    * @param minFactor     - range min - a 2 here makes us ensure that value
    *                        valueTocheck is at least twice the baseValue.
    * @param maxFactor     - range max
    * @return long
    */
-  public static long sanitizeUserArgs(long valueTocheck, long baseValue,
-      long minFactor, long maxFactor)
-      throws IllegalArgumentException {
-    if ((valueTocheck >= (baseValue * minFactor)) &&
-        (valueTocheck <= (baseValue * maxFactor))) {
-      return valueTocheck;
+  public static long sanitizeUserArgs(String key, long valueTocheck,
+      String baseKey, long baseValue, long minFactor, long maxFactor) {
+    long minLimit = baseValue * minFactor;
+    long maxLimit = baseValue * maxFactor;
+    if (valueTocheck < minLimit) {
+      LOG.warn(
+          "{} value = {} is smaller than min = {} based on"
+          + " the key value of {}, reset to the min value {}.",
+          key, valueTocheck, minLimit, baseKey, minLimit);
+      valueTocheck = minLimit;
+    } else if (valueTocheck > maxLimit) {
+      LOG.warn(
+          "{} value = {} is larger than max = {} based on"
+          + " the key value of {}, reset to the max value {}.",
+          key, valueTocheck, maxLimit, baseKey, maxLimit);
+      valueTocheck = maxLimit;
     }
-    String errMsg = String.format("%d is not within min = %d or max = " +
-        "%d", valueTocheck, baseValue * minFactor, baseValue * maxFactor);
-    throw new IllegalArgumentException(errMsg);
+
+    return valueTocheck;
   }
 
 

+ 14 - 5
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java

@@ -182,18 +182,27 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
           pipelineManager
               .getPipelines(type, factor, Pipeline.PipelineState.OPEN,
                   excludeList.getDatanodes(), excludeList.getPipelineIds());
-      Pipeline pipeline;
+      Pipeline pipeline = null;
       if (availablePipelines.size() == 0) {
         try {
           // TODO: #CLUTIL Remove creation logic when all replication types and
           // factors are handled by pipeline creator
           pipeline = pipelineManager.createPipeline(type, factor);
         } catch (IOException e) {
-          LOG.error("Pipeline creation failed for type:{} factor:{}",
-              type, factor, e);
-          break;
+          LOG.warn("Pipeline creation failed for type:{} factor:{}. Retrying " +
+                  "get pipelines call once.", type, factor, e);
+          availablePipelines = pipelineManager
+              .getPipelines(type, factor, Pipeline.PipelineState.OPEN,
+                  excludeList.getDatanodes(), excludeList.getPipelineIds());
+          if (availablePipelines.size() == 0) {
+            LOG.info("Could not find available pipeline of type:{} and " +
+                "factor:{} even after retrying", type, factor);
+            break;
+          }
         }
-      } else {
+      }
+
+      if (null == pipeline) {
         // TODO: #CLUTIL Make the selection policy driven.
         pipeline = availablePipelines
             .get((int) (Math.random() * availablePipelines.size()));

+ 7 - 3
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java

@@ -263,11 +263,15 @@ public class ContainerStateManager {
       }
       pipeline = pipelines.get((int) containerCount.get() % pipelines.size());
     }
-    return allocateContainer(pipelineManager, owner, pipeline);
+    synchronized (pipeline) {
+      return allocateContainer(pipelineManager, owner, pipeline);
+    }
   }
 
   /**
    * Allocates a new container based on the type, replication etc.
+   * This method should be called only after the lock on the pipeline is held
+   * on which the container will be allocated.
    *
    * @param pipelineManager   - Pipeline Manager class.
    * @param owner             - Owner of the container.
@@ -296,10 +300,10 @@ public class ContainerStateManager {
         .setReplicationFactor(pipeline.getFactor())
         .setReplicationType(pipeline.getType())
         .build();
-    pipelineManager.addContainerToPipeline(pipeline.getId(),
-        ContainerID.valueof(containerID));
     Preconditions.checkNotNull(containerInfo);
     containers.addContainer(containerInfo);
+    pipelineManager.addContainerToPipeline(pipeline.getId(),
+        ContainerID.valueof(containerID));
     containerStateCount.incrementAndGet(containerInfo.getState());
     LOG.trace("New container allocated: {}", containerInfo);
     return containerInfo;

+ 7 - 8
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/SCMContainerManager.java

@@ -386,18 +386,17 @@ public class SCMContainerManager implements ContainerManager {
 
   public ContainerInfo getMatchingContainer(final long sizeRequired,
       String owner, Pipeline pipeline, List<ContainerID> excludedContainers) {
+    NavigableSet<ContainerID> containerIDs;
     try {
-      //TODO: #CLUTIL See if lock is required here
-      NavigableSet<ContainerID> containerIDs =
-          pipelineManager.getContainersInPipeline(pipeline.getId());
+      synchronized (pipeline) {
+        //TODO: #CLUTIL See if lock is required here
+        containerIDs =
+            pipelineManager.getContainersInPipeline(pipeline.getId());
 
-      containerIDs = getContainersForOwner(containerIDs, owner);
-      if (containerIDs.size() < numContainerPerOwnerInPipeline) {
-        synchronized (pipeline) {
+        containerIDs = getContainersForOwner(containerIDs, owner);
+        if (containerIDs.size() < numContainerPerOwnerInPipeline) {
           // TODO: #CLUTIL Maybe we can add selection logic inside synchronized
           // as well
-          containerIDs = getContainersForOwner(
-              pipelineManager.getContainersInPipeline(pipeline.getId()), owner);
           if (containerIDs.size() < numContainerPerOwnerInPipeline) {
             ContainerInfo containerInfo =
                 containerStateManager.allocateContainer(pipelineManager, owner,

+ 4 - 0
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java

@@ -61,4 +61,8 @@ public final class PipelineFactory {
       List<DatanodeDetails> nodes) {
     return providers.get(type).create(factor, nodes);
   }
+
+  public void shutdown() {
+    providers.values().forEach(provider -> provider.shutdown());
+  }
 }

+ 1 - 0
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java

@@ -33,4 +33,5 @@ public interface PipelineProvider {
 
   Pipeline create(ReplicationFactor factor, List<DatanodeDetails> nodes);
 
+  void shutdown();
 }

+ 116 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java

@@ -24,17 +24,39 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.ratis.RatisHelper;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.grpc.GrpcTlsConfig;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftGroup;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.retry.RetryPolicy;
+import org.apache.ratis.rpc.SupportedRpcType;
+import org.apache.ratis.util.TimeDuration;
+import org.apache.ratis.util.function.CheckedBiConsumer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.ForkJoinWorkerThread;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 /**
@@ -42,10 +64,28 @@ import java.util.stream.Collectors;
  */
 public class RatisPipelineProvider implements PipelineProvider {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RatisPipelineProvider.class);
+
   private final NodeManager nodeManager;
   private final PipelineStateManager stateManager;
   private final Configuration conf;
 
+  // Set parallelism at 3, as now in Ratis we create 1 and 3 node pipelines.
+  private final int parallelismForPool = 3;
+
+  private final ForkJoinPool.ForkJoinWorkerThreadFactory factory =
+      (pool -> {
+        final ForkJoinWorkerThread worker = ForkJoinPool.
+            defaultForkJoinWorkerThreadFactory.newThread(pool);
+        worker.setName("RATISCREATEPIPELINE" + worker.getPoolIndex());
+        return worker;
+      });
+
+  private final ForkJoinPool forkJoinPool = new ForkJoinPool(
+      parallelismForPool, factory, null, false);
+
+
   RatisPipelineProvider(NodeManager nodeManager,
       PipelineStateManager stateManager, Configuration conf) {
     this.nodeManager = nodeManager;
@@ -53,6 +93,7 @@ public class RatisPipelineProvider implements PipelineProvider {
     this.conf = conf;
   }
 
+
   /**
    * Create pluggable container placement policy implementation instance.
    *
@@ -133,7 +174,81 @@ public class RatisPipelineProvider implements PipelineProvider {
         .build();
   }
 
+
+  @Override
+  public void shutdown() {
+    forkJoinPool.shutdownNow();
+    try {
+      forkJoinPool.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (Exception e) {
+      LOG.error("Unexpected exception occurred during shutdown of " +
+              "RatisPipelineProvider", e);
+    }
+  }
+
   protected void initializePipeline(Pipeline pipeline) throws IOException {
-    RatisPipelineUtils.createPipeline(pipeline, conf);
+    final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
+    LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
+    callRatisRpc(pipeline.getNodes(),
+        (raftClient, peer) -> {
+          RaftClientReply reply = raftClient.groupAdd(group, peer.getId());
+          if (reply == null || !reply.isSuccess()) {
+            String msg = "Pipeline initialization failed for pipeline:"
+                + pipeline.getId() + " node:" + peer.getId();
+            LOG.error(msg);
+            throw new IOException(msg);
+          }
+        });
+  }
+
+  private void callRatisRpc(List<DatanodeDetails> datanodes,
+      CheckedBiConsumer< RaftClient, RaftPeer, IOException> rpc)
+      throws IOException {
+    if (datanodes.isEmpty()) {
+      return;
+    }
+
+    final String rpcType = conf
+        .get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
+            ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
+    final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(conf);
+    final List< IOException > exceptions =
+        Collections.synchronizedList(new ArrayList<>());
+    final int maxOutstandingRequests =
+        HddsClientUtils.getMaxOutstandingRequests(conf);
+    final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(new
+        SecurityConfig(conf));
+    final TimeDuration requestTimeout =
+        RatisHelper.getClientRequestTimeout(conf);
+    try {
+      forkJoinPool.submit(() -> {
+        datanodes.parallelStream().forEach(d -> {
+          final RaftPeer p = RatisHelper.toRaftPeer(d);
+          try (RaftClient client = RatisHelper
+              .newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p,
+                  retryPolicy, maxOutstandingRequests, tlsConfig,
+                  requestTimeout)) {
+            rpc.accept(client, p);
+          } catch (IOException ioe) {
+            String errMsg =
+                "Failed invoke Ratis rpc " + rpc + " for " + d.getUuid();
+            LOG.error(errMsg, ioe);
+            exceptions.add(new IOException(errMsg, ioe));
+          }
+        });
+      }).get();
+    } catch (ExecutionException | RejectedExecutionException ex) {
+      LOG.error(ex.getClass().getName() + " exception occurred during " +
+          "createPipeline", ex);
+      throw new IOException(ex.getClass().getName() + " exception occurred " +
+          "during createPipeline", ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IOException("Interrupt exception occurred during " +
+          "createPipeline", ex);
+    }
+    if (!exceptions.isEmpty()) {
+      throw MultipleIOException.createIOException(exceptions);
+    }
   }
 }

+ 3 - 70
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java

@@ -17,66 +17,37 @@
  */
 package org.apache.hadoop.hdds.scm.pipeline;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
-import org.apache.hadoop.io.MultipleIOException;
 import org.apache.ratis.RatisHelper;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.grpc.GrpcTlsConfig;
-import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.retry.RetryPolicy;
 import org.apache.ratis.rpc.SupportedRpcType;
 import org.apache.ratis.util.TimeDuration;
-import org.apache.ratis.util.function.CheckedBiConsumer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
 
 /**
  * Utility class for Ratis pipelines. Contains methods to create and destroy
  * ratis pipelines.
  */
-final class RatisPipelineUtils {
+public final class RatisPipelineUtils {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(RatisPipelineUtils.class);
 
   private RatisPipelineUtils() {
   }
-
-  /**
-   * Sends ratis command to create pipeline on all the datanodes.
-   *
-   * @param pipeline  - Pipeline to be created
-   * @param ozoneConf - Ozone Confinuration
-   * @throws IOException if creation fails
-   */
-  public static void createPipeline(Pipeline pipeline, Configuration ozoneConf)
-      throws IOException {
-    final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
-    LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
-    callRatisRpc(pipeline.getNodes(), ozoneConf,
-        (raftClient, peer) -> {
-          RaftClientReply reply = raftClient.groupAdd(group, peer.getId());
-          if (reply == null || !reply.isSuccess()) {
-            String msg = "Pipeline initialization failed for pipeline:"
-                + pipeline.getId() + " node:" + peer.getId();
-            LOG.error(msg);
-            throw new IOException(msg);
-          }
-        });
-  }
-
   /**
    * Removes pipeline from SCM. Sends ratis command to destroy pipeline on all
    * the datanodes.
@@ -125,42 +96,4 @@ final class RatisPipelineUtils {
     client
         .groupRemove(RaftGroupId.valueOf(pipelineID.getId()), true, p.getId());
   }
-
-  private static void callRatisRpc(List<DatanodeDetails> datanodes,
-      Configuration ozoneConf,
-      CheckedBiConsumer<RaftClient, RaftPeer, IOException> rpc)
-      throws IOException {
-    if (datanodes.isEmpty()) {
-      return;
-    }
-
-    final String rpcType = ozoneConf
-        .get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
-            ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
-    final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf);
-    final List<IOException> exceptions =
-        Collections.synchronizedList(new ArrayList<>());
-    final int maxOutstandingRequests =
-        HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
-    final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(new
-        SecurityConfig(ozoneConf));
-    final TimeDuration requestTimeout =
-        RatisHelper.getClientRequestTimeout(ozoneConf);
-    datanodes.parallelStream().forEach(d -> {
-      final RaftPeer p = RatisHelper.toRaftPeer(d);
-      try (RaftClient client = RatisHelper
-          .newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p,
-              retryPolicy, maxOutstandingRequests, tlsConfig, requestTimeout)) {
-        rpc.accept(client, p);
-      } catch (IOException ioe) {
-        String errMsg =
-            "Failed invoke Ratis rpc " + rpc + " for " + d.getUuid();
-        LOG.error(errMsg, ioe);
-        exceptions.add(new IOException(errMsg, ioe));
-      }
-    });
-    if (!exceptions.isEmpty()) {
-      throw MultipleIOException.createIOException(exceptions);
-    }
-  }
 }

+ 4 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java

@@ -87,7 +87,8 @@ public class SCMPipelineManager implements PipelineManager {
     this.lock = new ReentrantReadWriteLock();
     this.conf = conf;
     this.stateManager = new PipelineStateManager(conf);
-    this.pipelineFactory = new PipelineFactory(nodeManager, stateManager, conf);
+    this.pipelineFactory = new PipelineFactory(nodeManager, stateManager,
+        conf);
     // TODO: See if thread priority needs to be set for these threads
     scheduler = new Scheduler("RatisPipelineUtilsThread", false, 1);
     this.backgroundPipelineCreator =
@@ -419,5 +420,7 @@ public class SCMPipelineManager implements PipelineManager {
     if(metrics != null) {
       metrics.unRegister();
     }
+    // shutdown pipeline provider.
+    pipelineFactory.shutdown();
   }
 }

+ 5 - 0
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java

@@ -72,4 +72,9 @@ public class SimplePipelineProvider implements PipelineProvider {
         .setNodes(nodes)
         .build();
   }
+
+  @Override
+  public void shutdown() {
+    // Do nothing.
+  }
 }

+ 72 - 0
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestHddsServerUtils.java

@@ -19,7 +19,13 @@
 package org.apache.hadoop.hdds.scm;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.test.PathUtils;
+
+import org.apache.commons.io.FileUtils;
+import static org.junit.Assert.assertTrue;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -27,12 +33,15 @@ import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.net.InetSocketAddress;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_NAMES;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -154,4 +163,67 @@ public class TestHddsServerUtils {
     HddsServerUtil.getScmAddressForDataNodes(conf);
   }
 
+  /**
+   * Test {@link ServerUtils#getScmDbDir}.
+   */
+  @Test
+  public void testGetScmDbDir() {
+    final File testDir = PathUtils.getTestDir(TestHddsServerUtils.class);
+    final File dbDir = new File(testDir, "scmDbDir");
+    final File metaDir = new File(testDir, "metaDir");   // should be ignored.
+    final Configuration conf = new OzoneConfiguration();
+    conf.set(ScmConfigKeys.OZONE_SCM_DB_DIRS, dbDir.getPath());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());
+
+    try {
+      assertEquals(dbDir, ServerUtils.getScmDbDir(conf));
+      assertTrue(dbDir.exists());          // should have been created.
+    } finally {
+      FileUtils.deleteQuietly(dbDir);
+    }
+  }
+
+  /**
+   * Test {@link ServerUtils#getScmDbDir} with fallback to OZONE_METADATA_DIRS
+   * when OZONE_SCM_DB_DIRS is undefined.
+   */
+  @Test
+  public void testGetScmDbDirWithFallback() {
+    final File testDir = PathUtils.getTestDir(TestHddsServerUtils.class);
+    final File metaDir = new File(testDir, "metaDir");
+    final Configuration conf = new OzoneConfiguration();
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());
+    try {
+      assertEquals(metaDir, ServerUtils.getScmDbDir(conf));
+      assertTrue(metaDir.exists());        // should have been created.
+    } finally {
+      FileUtils.deleteQuietly(metaDir);
+    }
+  }
+
+  @Test
+  public void testNoScmDbDirConfigured() {
+    thrown.expect(IllegalArgumentException.class);
+    ServerUtils.getScmDbDir(new OzoneConfiguration());
+  }
+
+  @Test
+  public void testGetStaleNodeInterval() {
+    final Configuration conf = new OzoneConfiguration();
+
+    // Reset OZONE_SCM_STALENODE_INTERVAL to 300s that
+    // larger than max limit value.
+    conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 300, TimeUnit.SECONDS);
+    conf.setInt(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 100);
+    // the max limit value will be returned
+    assertEquals(100000, HddsServerUtil.getStaleNodeInterval(conf));
+
+    // Reset OZONE_SCM_STALENODE_INTERVAL to 10ms that
+    // smaller than min limit value.
+    conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 10,
+        TimeUnit.MILLISECONDS);
+    conf.setInt(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 100);
+    // the min limit value will be returned
+    assertEquals(90000, HddsServerUtil.getStaleNodeInterval(conf));
+  }
 }

+ 43 - 0
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java

@@ -19,7 +19,13 @@ package org.apache.hadoop.hdds.scm.block;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.TimeoutException;
+
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -132,6 +138,43 @@ public class TestBlockManager {
     Assert.assertNotNull(block);
   }
 
+  @Test
+  public void testAllocateBlockInParallel() throws Exception {
+    eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
+    GenericTestUtils.waitFor(() -> {
+      return !blockManager.isScmInSafeMode();
+    }, 10, 1000 * 5);
+    int threadCount = 20;
+    List<ExecutorService> executors = new ArrayList<>(threadCount);
+    for (int i = 0; i < threadCount; i++) {
+      executors.add(Executors.newSingleThreadExecutor());
+    }
+    List<CompletableFuture<AllocatedBlock>> futureList =
+        new ArrayList<>(threadCount);
+    for (int i = 0; i < threadCount; i++) {
+      final CompletableFuture<AllocatedBlock> future =
+          new CompletableFuture<>();
+      CompletableFuture.supplyAsync(() -> {
+        try {
+          future.complete(blockManager
+              .allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner,
+                  new ExcludeList()));
+        } catch (IOException e) {
+          future.completeExceptionally(e);
+        }
+        return future;
+      }, executors.get(i));
+      futureList.add(future);
+    }
+    try {
+      CompletableFuture
+          .allOf(futureList.toArray(new CompletableFuture[futureList.size()]))
+          .get();
+    } catch (Exception e) {
+      Assert.fail("testAllocateBlockInParallel failed");
+    }
+  }
+
   @Test
   public void testAllocateOversizedBlock() throws Exception {
     eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);

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

@@ -43,14 +43,20 @@ import org.junit.rules.ExpectedException;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.Optional;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.UUID;
+import java.util.Iterator;
+import java.util.Optional;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
@@ -144,6 +150,43 @@ public class TestSCMContainerManager {
     Assert.assertTrue(pipelineList.size() > 5);
   }
 
+  @Test
+  public void testAllocateContainerInParallel() throws Exception {
+    int threadCount = 20;
+    List<ExecutorService> executors = new ArrayList<>(threadCount);
+    for (int i = 0; i < threadCount; i++) {
+      executors.add(Executors.newSingleThreadExecutor());
+    }
+    List<CompletableFuture<ContainerInfo>> futureList =
+        new ArrayList<>(threadCount);
+    for (int i = 0; i < threadCount; i++) {
+      final CompletableFuture<ContainerInfo> future = new CompletableFuture<>();
+      CompletableFuture.supplyAsync(() -> {
+        try {
+          ContainerInfo containerInfo = containerManager
+              .allocateContainer(xceiverClientManager.getType(),
+                  xceiverClientManager.getFactor(), containerOwner);
+
+          Assert.assertNotNull(containerInfo);
+          Assert.assertNotNull(containerInfo.getPipelineID());
+          future.complete(containerInfo);
+          return containerInfo;
+        } catch (IOException e) {
+          future.completeExceptionally(e);
+        }
+        return future;
+      }, executors.get(i));
+      futureList.add(future);
+    }
+    try {
+      CompletableFuture
+          .allOf(futureList.toArray(new CompletableFuture[futureList.size()]))
+          .get();
+    } catch (Exception e) {
+      Assert.fail("testAllocateBlockInParallel failed");
+    }
+  }
+
   @Test
   public void testGetContainer() throws IOException {
     ContainerInfo containerInfo = containerManager.allocateContainer(

+ 0 - 32
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java

@@ -68,7 +68,6 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
     .HEALTHY;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
 import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
-import static org.hamcrest.core.StringStartsWith.startsWith;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -234,37 +233,6 @@ public class TestSCMNodeManager {
     }
   }
 
-  /**
-   * Asserts that if user provides a value less than 5 times the heartbeat
-   * interval as the StaleNode Value, we throw since that is a QoS that we
-   * cannot maintain.
-   *
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws TimeoutException
-   */
-
-  @Test
-  public void testScmSanityOfUserConfig1()
-      throws IOException, AuthenticationException {
-    OzoneConfiguration conf = getConf();
-    final int interval = 100;
-    conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, interval,
-        MILLISECONDS);
-    conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, 1, SECONDS);
-
-    // This should be 5 times more than  OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL
-    // and 3 times more than OZONE_SCM_HEARTBEAT_INTERVAL
-    conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, interval, MILLISECONDS);
-
-    thrown.expect(IllegalArgumentException.class);
-
-    // This string is a multiple of the interval value
-    thrown.expectMessage(
-        startsWith("100 is not within min = 500 or max = 100000"));
-    createNodeManager(conf);
-  }
-
   /**
    * Asserts that if Stale Interval value is more than 5 times the value of HB
    * processing thread it is a sane value.

+ 5 - 0
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockRatisPipelineProvider.java

@@ -37,4 +37,9 @@ public class MockRatisPipelineProvider extends RatisPipelineProvider {
   protected void initializePipeline(Pipeline pipeline) throws IOException {
     // do nothing as the datanodes do not exists
   }
+
+  @Override
+  public void shutdown() {
+    // Do nothing.
+  }
 }

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -609,7 +609,13 @@ public class WebHdfsFileSystem extends FileSystem
       boolean pathAlreadyEncoded = false;
       try {
         fspathUriDecoded = URLDecoder.decode(fspathUri.getPath(), "UTF-8");
-        pathAlreadyEncoded = true;
+        //below condition check added as part of fixing HDFS-14323 to make
+        //sure pathAlreadyEncoded is not set in the case the input url does
+        //not have any encoded sequence already.This will help pulling data
+        //from 2.x hadoop cluster to 3.x using 3.x distcp client operation
+        if(!fspathUri.getPath().equals(fspathUriDecoded)) {
+          pathAlreadyEncoded = true;
+        }
       } catch (IllegalArgumentException ex) {
         LOG.trace("Cannot decode URL encoded file", ex);
       }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java

@@ -170,4 +170,8 @@ public final class ErasureCodingWorker {
     stripedReconstructionPool.shutdown();
     stripedReadPool.shutdown();
   }
+
+  public float getXmitWeight() {
+    return xmitWeight;
+  }
 }

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java

@@ -67,7 +67,11 @@ class StripedBlockReconstructor extends StripedReconstructor
       LOG.warn("Failed to reconstruct striped block: {}", getBlockGroup(), e);
       getDatanode().getMetrics().incrECFailedReconstructionTasks();
     } finally {
-      getDatanode().decrementXmitsInProgress(getXmits());
+      float xmitWeight = getErasureCodingWorker().getXmitWeight();
+      // if the xmits is smaller than 1, the xmitsSubmitted should be set to 1
+      // because if it set to zero, we cannot to measure the xmits submitted
+      int xmitsSubmitted = Math.max((int) (getXmits() * xmitWeight), 1);
+      getDatanode().decrementXmitsInProgress(xmitsSubmitted);
       final DataNodeMetrics metrics = getDatanode().getMetrics();
       metrics.incrECReconstructionTasks();
       metrics.incrECReconstructionBytesRead(getBytesRead());

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java

@@ -275,4 +275,8 @@ abstract class StripedReconstructor {
   DataNode getDatanode() {
     return datanode;
   }
+
+  public ErasureCodingWorker getErasureCodingWorker() {
+    return erasureCodingWorker;
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -412,7 +412,7 @@ Usage:
 | `-getDatanodeInfo` \<datanode\_host:ipc\_port\> | Get the information about the given datanode. See [Rolling Upgrade document](./HdfsRollingUpgrade.html#dfsadmin_-getDatanodeInfo) for the detail. |
 | `-metasave` filename | Save Namenode's primary data structures to *filename* in the directory specified by hadoop.log.dir property. *filename* is overwritten if it exists. *filename* will contain one line for each of the following<br/>1. Datanodes heart beating with Namenode<br/>2. Blocks waiting to be replicated<br/>3. Blocks currently being replicated<br/>4. Blocks waiting to be deleted |
 | `-triggerBlockReport` `[-incremental]` \<datanode\_host:ipc\_port\> | Trigger a block report for the given datanode. If 'incremental' is specified, it will be otherwise, it will be a full block report. |
-| `-listOpenFiles` `[-blockingDecommission]` `[-path <path>]` | List all open files currently managed by the NameNode along with client name and client machine accessing them. Open files list will be filtered by given type and path. |
+| `-listOpenFiles` `[-blockingDecommission]` `[-path <path>]` | List all open files currently managed by the NameNode along with client name and client machine accessing them. Open files list will be filtered by given type and path. Add -blockingDecommission option if you only want to list open files that are blocking the DataNode decommissioning. |
 | `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 
 Runs a HDFS dfsadmin client.

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java

@@ -514,6 +514,8 @@ public class TestReconstructStripedFile {
 
   @Test(timeout = 180000)
   public void testErasureCodingWorkerXmitsWeight() throws Exception {
+    testErasureCodingWorkerXmitsWeight(0.5f,
+        (int) (ecPolicy.getNumDataUnits() * 0.5f));
     testErasureCodingWorkerXmitsWeight(1f, ecPolicy.getNumDataUnits());
     testErasureCodingWorkerXmitsWeight(0f, 1);
     testErasureCodingWorkerXmitsWeight(10f, 10 * ecPolicy.getNumDataUnits());
@@ -567,6 +569,10 @@ public class TestReconstructStripedFile {
     } finally {
       barrier.await();
       DataNodeFaultInjector.set(oldInjector);
+      for (final DataNode curDn : cluster.getDataNodes()) {
+        GenericTestUtils.waitFor(() -> curDn.getXceiverCount() > 1, 10, 60000);
+        assertEquals(0, curDn.getXmitsInProgress());
+      }
     }
   }
 }

+ 3 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java

@@ -192,7 +192,8 @@ public class TaskHeartbeatHandler extends AbstractService {
             (currentTime > (entry.getValue().getLastProgress() + taskTimeOut));
         // when container in NM not started in a long time,
         // we think the taskAttempt is stuck
-        boolean taskStuck = (!entry.getValue().isReported()) &&
+        boolean taskStuck = (taskStuckTimeOut > 0) &&
+            (!entry.getValue().isReported()) &&
             (currentTime >
                 (entry.getValue().getLastProgress() + taskStuckTimeOut));
 
@@ -225,7 +226,7 @@ public class TaskHeartbeatHandler extends AbstractService {
   }
 
   @VisibleForTesting
-  ConcurrentMap getRunningAttempts(){
+  ConcurrentMap<TaskAttemptId, ReportTime> getRunningAttempts(){
     return runningAttempts;
   }
 

+ 12 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -968,16 +968,20 @@ public class RMContainerAllocator extends RMContainerRequestor
 
   @VisibleForTesting
   public TaskAttemptEvent createContainerFinishedEvent(ContainerStatus cont,
-      TaskAttemptId attemptID) {
-    if (cont.getExitStatus() == ContainerExitStatus.ABORTED
-        || cont.getExitStatus() == ContainerExitStatus.PREEMPTED) {
-      // killed by framework
-      return new TaskAttemptEvent(attemptID,
-          TaskAttemptEventType.TA_KILL);
-    } else {
-      return new TaskAttemptEvent(attemptID,
+      TaskAttemptId attemptId) {
+    TaskAttemptEvent event;
+    switch (cont.getExitStatus()) {
+    case ContainerExitStatus.ABORTED:
+    case ContainerExitStatus.PREEMPTED:
+    case ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER:
+      // killed by YARN
+      event = new TaskAttemptEvent(attemptId, TaskAttemptEventType.TA_KILL);
+      break;
+    default:
+      event = new TaskAttemptEvent(attemptId,
           TaskAttemptEventType.TA_CONTAINER_COMPLETED);
     }
+    return event;
   }
   
   @SuppressWarnings("unchecked")

+ 42 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app;
 import static org.junit.Assert.assertFalse;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
@@ -48,7 +49,7 @@ import java.util.concurrent.ConcurrentMap;
 
 public class TestTaskHeartbeatHandler {
   
-  @SuppressWarnings({ "rawtypes", "unchecked" })
+  @SuppressWarnings("unchecked")
   @Test
   public void testTaskTimeout() throws InterruptedException {
     EventHandler mockHandler = mock(EventHandler.class);
@@ -81,6 +82,46 @@ public class TestTaskHeartbeatHandler {
     }
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testTaskTimeoutDisable() throws InterruptedException {
+    EventHandler mockHandler = mock(EventHandler.class);
+    Clock clock = SystemClock.getInstance();
+    TaskHeartbeatHandler hb = new TaskHeartbeatHandler(mockHandler, clock, 1);
+
+    Configuration conf = new Configuration();
+    conf.setLong(MRJobConfig.TASK_STUCK_TIMEOUT_MS, 0); // no timeout
+    conf.setInt(MRJobConfig.TASK_TIMEOUT, 0); // no timeout
+    // set TASK_PROGRESS_REPORT_INTERVAL to a value smaller than TASK_TIMEOUT
+    // so that TASK_TIMEOUT is not overridden
+    conf.setLong(MRJobConfig.TASK_PROGRESS_REPORT_INTERVAL, 0);
+    conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 10); //10 ms
+
+    hb.init(conf);
+    hb.start();
+    try {
+      ApplicationId appId = ApplicationId.newInstance(0L, 5);
+      JobId jobId = MRBuilderUtils.newJobId(appId, 4);
+      TaskId tid = MRBuilderUtils.newTaskId(jobId, 3, TaskType.MAP);
+      TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 2);
+      hb.register(taid);
+
+      ConcurrentMap<TaskAttemptId, TaskHeartbeatHandler.ReportTime>
+          runningAttempts = hb.getRunningAttempts();
+      for (Map.Entry<TaskAttemptId, TaskHeartbeatHandler.ReportTime> entry
+          : runningAttempts.entrySet()) {
+        assertFalse(entry.getValue().isReported());
+      }
+
+      Thread.sleep(100);
+
+      // Timeout is disabled, so the task should not be canceled
+      verify(mockHandler, never()).handle(any(Event.class));
+    } finally {
+      hb.stop();
+    }
+  }
+
   @SuppressWarnings("unchecked")
   @Test
   public void testTaskStuck() throws InterruptedException {

+ 22 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java

@@ -2430,6 +2430,8 @@ public class TestRMContainerAllocator {
     ApplicationId applicationId = ApplicationId.newInstance(1, 1);
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, 1);
+
+    // ABORTED
     ContainerId containerId =
         ContainerId.newContainerId(applicationAttemptId, 1);
     ContainerStatus status = ContainerStatus.newInstance(
@@ -2448,6 +2450,7 @@ public class TestRMContainerAllocator {
         abortedStatus, attemptId);
     Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent.getType());
 
+    // PREEMPTED
     ContainerId containerId2 =
         ContainerId.newContainerId(applicationAttemptId, 2);
     ContainerStatus status2 = ContainerStatus.newInstance(containerId2,
@@ -2464,6 +2467,25 @@ public class TestRMContainerAllocator {
     TaskAttemptEvent abortedEvent2 = allocator.createContainerFinishedEvent(
         preemptedStatus, attemptId);
     Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent2.getType());
+
+    // KILLED_BY_CONTAINER_SCHEDULER
+    ContainerId containerId3 =
+        ContainerId.newContainerId(applicationAttemptId, 3);
+    ContainerStatus status3 = ContainerStatus.newInstance(containerId3,
+        ContainerState.RUNNING, "", 0);
+
+    ContainerStatus killedByContainerSchedulerStatus =
+        ContainerStatus.newInstance(containerId3, ContainerState.RUNNING, "",
+            ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER);
+
+    TaskAttemptEvent event3 = allocator.createContainerFinishedEvent(status3,
+        attemptId);
+    Assert.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED,
+        event3.getType());
+
+    TaskAttemptEvent abortedEvent3 = allocator.createContainerFinishedEvent(
+        killedByContainerSchedulerStatus, attemptId);
+    Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent3.getType());
   }
 
   @Test

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -282,6 +282,7 @@
   <description>The max timeout before receiving remote task's first heartbeat.
     This parameter is in order to avoid waiting for the container
     to start indefinitely, which made task stuck in the NEW state.
+    A value of 0 disables the timeout.
   </description>
 </property>
 

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

@@ -528,6 +528,23 @@ public class OzoneBucket extends WithMetadata {
             recursive);
   }
 
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param keyName    Absolute path of the entry to be listed
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @return list of file status
+   */
+  public List<OzoneFileStatus> listStatus(String keyName, boolean recursive,
+      String startKey, long numEntries) throws IOException {
+    return proxy
+        .listStatus(volumeName, name, keyName, recursive, startKey, numEntries);
+  }
+
   /**
    * An Iterator to iterate over {@link OzoneKey} list.
    */

+ 18 - 1
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -604,4 +604,21 @@ public interface ClientProtocol {
   OzoneOutputStream createFile(String volumeName, String bucketName,
       String keyName, long size, ReplicationType type, ReplicationFactor factor,
       boolean overWrite, boolean recursive) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param volumeName Volume name
+   * @param bucketName Bucket name
+   * @param keyName    Absolute path of the entry to be listed
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(String volumeName, String bucketName,
+      String keyName, boolean recursive, String startKey, long numEntries)
+      throws IOException;
 }

+ 11 - 3
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java

@@ -112,7 +112,7 @@ public class RestClient implements ClientProtocol {
   private final URI ozoneRestUri;
   private final CloseableHttpClient httpClient;
   private final UserGroupInformation ugi;
-  private final OzoneAcl.OzoneACLRights userRights;
+  // private final OzoneAcl.OzoneACLRights userRights;
 
    /**
     * Creates RestClient instance with the given configuration.
@@ -161,8 +161,8 @@ public class RestClient implements ClientProtocol {
                   .build())
           .build();
 
-      this.userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS,
-          OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
+//      this.userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS,
+//          OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
 
       // TODO: Add new configuration parameter to configure RestServerSelector.
       RestServerSelector defaultSelector = new DefaultRestServerSelector();
@@ -1113,4 +1113,12 @@ public class RestClient implements ClientProtocol {
     throw new UnsupportedOperationException(
         "Ozone REST protocol does not " + "support this operation.");
   }
+
+  @Override
+  public List<OzoneFileStatus> listStatus(String volumeName, String bucketName,
+      String keyName, boolean recursive, String startKey, long numEntries)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "Ozone REST protocol does not " + "support this operation.");
+  }
 }

+ 27 - 11
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

@@ -73,7 +73,6 @@ import org.apache.hadoop.ozone.om.protocolPB
     .OzoneManagerProtocolClientSideTranslatorPB;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.ServicePort;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -85,6 +84,9 @@ import org.apache.hadoop.hdds.scm.protocolPB
 import org.apache.hadoop.hdds.scm.protocolPB
     .StorageContainerLocationProtocolPB;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
+import org.apache.hadoop.ozone.security.acl.OzoneAclConfig;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -121,8 +123,8 @@ public class RpcClient implements ClientProtocol, KeyProviderTokenIssuer {
   private final int bytesPerChecksum;
   private boolean verifyChecksum;
   private final UserGroupInformation ugi;
-  private final OzoneAcl.OzoneACLRights userRights;
-  private final OzoneAcl.OzoneACLRights groupRights;
+  private final ACLType userRights;
+  private final ACLType groupRights;
   private final long streamBufferFlushSize;
   private final long streamBufferMaxSize;
   private final long blockSize;
@@ -141,10 +143,11 @@ public class RpcClient implements ClientProtocol, KeyProviderTokenIssuer {
     Preconditions.checkNotNull(conf);
     this.conf = new OzoneConfiguration(conf);
     this.ugi = UserGroupInformation.getCurrentUser();
-    this.userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS,
-        OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
-    this.groupRights = conf.getEnum(OMConfigKeys.OZONE_OM_GROUP_RIGHTS,
-        OMConfigKeys.OZONE_OM_GROUP_RIGHTS_DEFAULT);
+    // Get default acl rights for user and group.
+    OzoneAclConfig aclConfig = this.conf.getObject(OzoneAclConfig.class);
+    this.userRights = aclConfig.getUserDefaultRights();
+    this.groupRights = aclConfig.getGroupDefaultRights();
+
     this.ozoneManagerClient = new OzoneManagerProtocolClientSideTranslatorPB(
         this.conf, clientId.toString(), ugi);
     long scmVersion =
@@ -256,13 +259,13 @@ public class RpcClient implements ClientProtocol, KeyProviderTokenIssuer {
         OzoneQuota.parseQuota(volArgs.getQuota()).sizeInBytes();
     List<OzoneAcl> listOfAcls = new ArrayList<>();
     //User ACL
-    listOfAcls.add(new OzoneAcl(OzoneAcl.OzoneACLType.USER,
+    listOfAcls.add(new OzoneAcl(ACLIdentityType.USER,
             owner, userRights));
     //Group ACLs of the User
     List<String> userGroups = Arrays.asList(UserGroupInformation
         .createRemoteUser(owner).getGroupNames());
     userGroups.stream().forEach((group) -> listOfAcls.add(
-        new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights)));
+        new OzoneAcl(ACLIdentityType.GROUP, group, groupRights)));
     //ACLs from VolumeArgs
     if(volArgs.getAcls() != null) {
       listOfAcls.addAll(volArgs.getAcls());
@@ -403,13 +406,13 @@ public class RpcClient implements ClientProtocol, KeyProviderTokenIssuer {
 
     List<OzoneAcl> listOfAcls = new ArrayList<>();
     //User ACL
-    listOfAcls.add(new OzoneAcl(OzoneAcl.OzoneACLType.USER,
+    listOfAcls.add(new OzoneAcl(ACLIdentityType.USER,
         ugi.getUserName(), userRights));
     //Group ACLs of the User
     List<String> userGroups = Arrays.asList(UserGroupInformation
         .createRemoteUser(ugi.getUserName()).getGroupNames());
     userGroups.stream().forEach((group) -> listOfAcls.add(
-        new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights)));
+        new OzoneAcl(ACLIdentityType.GROUP, group, groupRights)));
     //ACLs from BucketArgs
     if(bucketArgs.getAcls() != null) {
       listOfAcls.addAll(bucketArgs.getAcls());
@@ -993,6 +996,19 @@ public class RpcClient implements ClientProtocol, KeyProviderTokenIssuer {
         factor);
   }
 
+  @Override
+  public List<OzoneFileStatus> listStatus(String volumeName, String bucketName,
+      String keyName, boolean recursive, String startKey, long numEntries)
+      throws IOException {
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .build();
+    return ozoneManagerClient
+        .listStatus(keyArgs, recursive, startKey, numEntries);
+  }
+
   private OzoneInputStream createInputStream(OmKeyInfo keyInfo,
       String requestId) throws IOException {
     LengthInputStream lengthInputStream = KeyInputStream

+ 1 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java

@@ -189,6 +189,7 @@ public final class OmUtils {
     case ListMultiPartUploadParts:
     case GetFileStatus:
     case LookupFile:
+    case ListStatus:
       return true;
     case CreateVolume:
     case SetVolumeProperty:

+ 43 - 69
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java → hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java

@@ -19,6 +19,11 @@
 
 package org.apache.hadoop.ozone;
 
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
+
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Objects;
 
 /**
@@ -32,9 +37,9 @@ import java.util.Objects;
  * </ul>
  */
 public class OzoneAcl {
-  private OzoneACLType type;
+  private ACLIdentityType type;
   private String name;
-  private OzoneACLRights rights;
+  private List<ACLType> rights;
 
   /**
    * Constructor for OzoneAcl.
@@ -47,16 +52,37 @@ public class OzoneAcl {
    *
    * @param type - Type
    * @param name - Name of user
-   * @param rights - Rights
+   * @param acl - Rights
    */
-  public OzoneAcl(OzoneACLType type, String name, OzoneACLRights rights) {
+  public OzoneAcl(ACLIdentityType type, String name, ACLType acl) {
     this.name = name;
-    this.rights = rights;
+    this.rights = new ArrayList<>();
+    this.rights.add(acl);
     this.type = type;
-    if (type == OzoneACLType.WORLD && name.length() != 0) {
+    if (type == ACLIdentityType.WORLD && name.length() != 0) {
       throw new IllegalArgumentException("Unexpected name part in world type");
     }
-    if (((type == OzoneACLType.USER) || (type == OzoneACLType.GROUP))
+    if (((type == ACLIdentityType.USER) || (type == ACLIdentityType.GROUP))
+        && (name.length() == 0)) {
+      throw new IllegalArgumentException("User or group name is required");
+    }
+  }
+
+  /**
+   * Constructor for OzoneAcl.
+   *
+   * @param type - Type
+   * @param name - Name of user
+   * @param acls - Rights
+   */
+  public OzoneAcl(ACLIdentityType type, String name, List<ACLType> acls) {
+    this.name = name;
+    this.rights = acls;
+    this.type = type;
+    if (type == ACLIdentityType.WORLD && name.length() != 0) {
+      throw new IllegalArgumentException("Unexpected name part in world type");
+    }
+    if (((type == ACLIdentityType.USER) || (type == ACLIdentityType.GROUP))
         && (name.length() == 0)) {
       throw new IllegalArgumentException("User or group name is required");
     }
@@ -78,17 +104,20 @@ public class OzoneAcl {
       throw new IllegalArgumentException("ACLs are not in expected format");
     }
 
-    OzoneACLType aclType = OzoneACLType.valueOf(parts[0].toUpperCase());
-    OzoneACLRights rights = OzoneACLRights.getACLRight(parts[2].toLowerCase());
+    ACLIdentityType aclType = ACLIdentityType.valueOf(parts[0].toUpperCase());
+    List<ACLType> acls = new ArrayList<>();
+    for (char ch : parts[2].toCharArray()) {
+      acls.add(ACLType.getACLRight(String.valueOf(ch)));
+    }
 
     // TODO : Support sanitation of these user names by calling into
     // userAuth Interface.
-    return new OzoneAcl(aclType, parts[1], rights);
+    return new OzoneAcl(aclType, parts[1], acls);
   }
 
   @Override
   public String toString() {
-    return type + ":" + name + ":" + OzoneACLRights.getACLRightsString(rights);
+    return type + ":" + name + ":" + ACLType.getACLString(rights);
   }
 
   /**
@@ -120,7 +149,7 @@ public class OzoneAcl {
    *
    * @return - Rights
    */
-  public OzoneACLRights getRights() {
+  public List<ACLType> getRights() {
     return rights;
   }
 
@@ -129,7 +158,7 @@ public class OzoneAcl {
    *
    * @return type
    */
-  public OzoneACLType getType() {
+  public ACLIdentityType getType() {
     return type;
   }
 
@@ -150,9 +179,7 @@ public class OzoneAcl {
       return false;
     }
     OzoneAcl otherAcl = (OzoneAcl) obj;
-    return otherAcl.getName().equals(this.getName()) &&
-        otherAcl.getRights() == this.getRights() &&
-        otherAcl.getType() == this.getType();
+    return otherAcl.toString().equals(this.toString());
   }
 
   /**
@@ -177,57 +204,4 @@ public class OzoneAcl {
       value = val;
     }
   }
-
-  /**
-   * ACL rights.
-   */
-  public enum OzoneACLRights {
-    READ, WRITE, READ_WRITE;
-
-    /**
-     * Returns the ACL rights based on passed in String.
-     *
-     * @param type ACL right string
-     *
-     * @return OzoneACLRights
-     */
-    public static OzoneACLRights getACLRight(String type) {
-      if (type == null || type.isEmpty()) {
-        throw new IllegalArgumentException("ACL right cannot be empty");
-      }
-
-      switch (type) {
-      case OzoneConsts.OZONE_ACL_READ:
-        return OzoneACLRights.READ;
-      case OzoneConsts.OZONE_ACL_WRITE:
-        return OzoneACLRights.WRITE;
-      case OzoneConsts.OZONE_ACL_READ_WRITE:
-      case OzoneConsts.OZONE_ACL_WRITE_READ:
-        return OzoneACLRights.READ_WRITE;
-      default:
-        throw new IllegalArgumentException("ACL right is not recognized");
-      }
-
-    }
-
-    /**
-     * Returns String representation of ACL rights.
-     * @param acl OzoneACLRights
-     * @return String representation of acl
-     */
-    public static String getACLRightsString(OzoneACLRights acl) {
-      switch(acl) {
-      case READ:
-        return OzoneConsts.OZONE_ACL_READ;
-      case WRITE:
-        return OzoneConsts.OZONE_ACL_WRITE;
-      case READ_WRITE:
-        return OzoneConsts.OZONE_ACL_READ_WRITE;
-      default:
-        throw new IllegalArgumentException("ACL right is not recognized");
-      }
-    }
-
-  }
-
 }

+ 2 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java

@@ -58,7 +58,8 @@ public enum OMAction implements AuditAction {
   GET_FILE_STATUS,
   CREATE_DIRECTORY,
   CREATE_FILE,
-  LOOKUP_FILE;
+  LOOKUP_FILE,
+  LIST_STATUS;
 
   @Override
   public String getAction() {

+ 0 - 12
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.om;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.ratis.util.TimeDuration;
 
 /**
@@ -78,17 +77,6 @@ public final class OMConfigKeys {
       "ozone.om.user.max.volume";
   public static final int OZONE_OM_USER_MAX_VOLUME_DEFAULT = 1024;
 
-  // OM Default user/group permissions
-  public static final String OZONE_OM_USER_RIGHTS =
-      "ozone.om.user.rights";
-  public static final OzoneAcl.OzoneACLRights OZONE_OM_USER_RIGHTS_DEFAULT =
-      OzoneAcl.OzoneACLRights.READ_WRITE;
-
-  public static final String OZONE_OM_GROUP_RIGHTS =
-      "ozone.om.group.rights";
-  public static final OzoneAcl.OzoneACLRights OZONE_OM_GROUP_RIGHTS_DEFAULT =
-      OzoneAcl.OzoneACLRights.READ_WRITE;
-
   public static final String OZONE_KEY_DELETING_LIMIT_PER_TASK =
       "ozone.key.deleting.limit.per.task";
   public static final int OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT = 1000;

+ 42 - 18
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmOzoneAclMap.java

@@ -37,7 +37,7 @@ import java.util.HashMap;
 @SuppressWarnings("ProtocolBufferOrdinal")
 public class OmOzoneAclMap {
   // per Acl Type user:rights map
-  private ArrayList<Map<String, OzoneAclRights>> aclMaps;
+  private ArrayList<Map<String, List<OzoneAclRights>>> aclMaps;
 
   OmOzoneAclMap() {
     aclMaps = new ArrayList<>();
@@ -46,51 +46,75 @@ public class OmOzoneAclMap {
     }
   }
 
-  private Map<String, OzoneAclRights> getMap(OzoneAclType type) {
+  private Map<String, List<OzoneAclRights>> getMap(OzoneAclType type) {
     return aclMaps.get(type.ordinal());
   }
 
   // For a given acl type and user, get the stored acl
-  private OzoneAclRights getAcl(OzoneAclType type, String user) {
+  private List<OzoneAclRights> getAcl(OzoneAclType type, String user) {
     return getMap(type).get(user);
   }
 
   // Add a new acl to the map
   public void addAcl(OzoneAclInfo acl) {
-    getMap(acl.getType()).put(acl.getName(), acl.getRights());
+    getMap(acl.getType()).put(acl.getName(), acl.getRightsList());
   }
 
   // for a given acl, check if the user has access rights
   public boolean hasAccess(OzoneAclInfo acl) {
-    OzoneAclRights storedRights = getAcl(acl.getType(), acl.getName());
-    if (storedRights != null) {
-      switch (acl.getRights()) {
-      case READ:
-        return (storedRights == OzoneAclRights.READ)
-            || (storedRights == OzoneAclRights.READ_WRITE);
+    if (acl == null) {
+      return false;
+    }
+
+    List<OzoneAclRights> storedRights = getAcl(acl.getType(), acl.getName());
+    if(storedRights == null) {
+      return false;
+    }
+
+    for (OzoneAclRights right : storedRights) {
+      switch (right) {
+      case CREATE:
+        return (right == OzoneAclRights.CREATE)
+            || (right == OzoneAclRights.ALL);
+      case LIST:
+        return (right == OzoneAclRights.LIST)
+            || (right == OzoneAclRights.ALL);
       case WRITE:
-        return (storedRights == OzoneAclRights.WRITE)
-            || (storedRights == OzoneAclRights.READ_WRITE);
-      case READ_WRITE:
-        return (storedRights == OzoneAclRights.READ_WRITE);
+        return (right == OzoneAclRights.WRITE)
+            || (right == OzoneAclRights.ALL);
+      case READ:
+        return (right == OzoneAclRights.READ)
+            || (right == OzoneAclRights.ALL);
+      case DELETE:
+        return (right == OzoneAclRights.DELETE)
+            || (right == OzoneAclRights.ALL);
+      case READ_ACL:
+        return (right == OzoneAclRights.READ_ACL)
+            || (right == OzoneAclRights.ALL);
+      case WRITE_ACL:
+        return (right == OzoneAclRights.WRITE_ACL)
+            || (right == OzoneAclRights.ALL);
+      case ALL:
+        return (right == OzoneAclRights.ALL);
+      case NONE:
+        return !(right == OzoneAclRights.NONE);
       default:
         return false;
       }
-    } else {
-      return false;
     }
+    return false;
   }
 
   // Convert this map to OzoneAclInfo Protobuf List
   public List<OzoneAclInfo> ozoneAclGetProtobuf() {
     List<OzoneAclInfo> aclList = new LinkedList<>();
     for (OzoneAclType type: OzoneAclType.values()) {
-      for (Map.Entry<String, OzoneAclRights> entry :
+      for (Map.Entry<String, List<OzoneAclRights>> entry :
           aclMaps.get(type.ordinal()).entrySet()) {
         OzoneAclInfo aclInfo = OzoneAclInfo.newBuilder()
             .setName(entry.getKey())
             .setType(type)
-            .setRights(entry.getValue())
+            .addAllRights(entry.getValue())
             .build();
         aclList.add(aclInfo);
       }

+ 89 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java

@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om.helpers;
+
+import org.apache.hadoop.fs.Path;
+
+import java.nio.file.Paths;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+
+/**
+ * Utility class for OzoneFileSystem.
+ */
+public final class OzoneFSUtils {
+
+  private OzoneFSUtils() {}
+
+  /**
+   * Returns string representation of path after removing the leading slash.
+   */
+  public static String pathToKey(Path path) {
+    return path.toString().substring(1);
+  }
+
+  /**
+   * Returns string representation of the input path parent. The function adds
+   * a trailing slash if it does not exist and returns an empty string if the
+   * parent is root.
+   */
+  public static String getParent(String keyName) {
+    java.nio.file.Path parentDir = Paths.get(keyName).getParent();
+    if (parentDir == null) {
+      return "";
+    }
+    return addTrailingSlashIfNeeded(parentDir.toString());
+  }
+
+  /**
+   * The function returns immediate child of given ancestor in a particular
+   * descendant. For example if ancestor is /a/b and descendant is /a/b/c/d/e
+   * the function should return /a/b/c/. If the descendant itself is the
+   * immediate child then it is returned as is without adding a trailing slash.
+   * This is done to distinguish files from a directory as in ozone files do
+   * not carry a trailing slash.
+   */
+  public static String getImmediateChild(String descendant, String ancestor) {
+    ancestor =
+        !ancestor.isEmpty() ? addTrailingSlashIfNeeded(ancestor) : ancestor;
+    if (!descendant.startsWith(ancestor)) {
+      return null;
+    }
+    java.nio.file.Path descendantPath = Paths.get(descendant);
+    java.nio.file.Path ancestorPath = Paths.get(ancestor);
+    int ancestorPathNameCount =
+        ancestor.isEmpty() ? 0 : ancestorPath.getNameCount();
+    if (descendantPath.getNameCount() - ancestorPathNameCount > 1) {
+      return addTrailingSlashIfNeeded(
+          ancestor + descendantPath.getName(ancestorPathNameCount));
+    }
+    return descendant;
+  }
+
+  public static String addTrailingSlashIfNeeded(String key) {
+    if (!key.endsWith(OZONE_URI_DELIMITER)) {
+      return key + OZONE_URI_DELIMITER;
+    } else {
+      return key;
+    }
+  }
+
+  public static boolean isFile(String keyName) {
+    return !keyName.endsWith(OZONE_URI_DELIMITER);
+  }
+}

+ 6 - 5
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFileStatus.java

@@ -18,11 +18,11 @@
 
 package org.apache.hadoop.ozone.om.helpers;
 
-import org.apache.hadoop.fs.FSProtos.FileStatusProto;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.protocolPB.PBHelper;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneFileStatusProto;
 
 import java.io.IOException;
 import java.net.URI;
@@ -53,13 +53,14 @@ public class OzoneFileStatus extends FileStatus {
     super(0, true, 0, 0, 0, getPath(keyName));
   }
 
-  public FileStatusProto getProtobuf() throws IOException {
-    return PBHelper.convert(this);
+  public OzoneFileStatusProto getProtobuf() throws IOException {
+    return OzoneFileStatusProto.newBuilder().setStatus(PBHelper.convert(this))
+        .build();
   }
 
-  public static OzoneFileStatus getFromProtobuf(FileStatusProto response)
+  public static OzoneFileStatus getFromProtobuf(OzoneFileStatusProto response)
       throws IOException {
-    return new OzoneFileStatus(PBHelper.convert(response));
+    return new OzoneFileStatus(PBHelper.convert(response.getStatus()));
   }
 
   public static Path getPath(String keyName) {

+ 14 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java

@@ -450,5 +450,19 @@ public interface OzoneManagerProtocol
    *                     invalid arguments
    */
   OmKeyInfo lookupFile(OmKeyArgs keyArgs) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param keyArgs    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs keyArgs, boolean recursive,
+      String startKey, long numEntries) throws IOException;
 }
 

+ 40 - 7
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java

@@ -55,7 +55,13 @@ import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneFileStatusProto;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LookupFileRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LookupFileResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateFileRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateFileResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListStatusRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListStatusResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateDirectoryRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusRequest;
@@ -1281,14 +1287,13 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
         .setBucketName(args.getBucketName())
         .setKeyName(args.getKeyName())
         .build();
-    OzoneManagerProtocolProtos.LookupFileRequest lookupFileRequest =
-        OzoneManagerProtocolProtos.LookupFileRequest.newBuilder()
+    LookupFileRequest lookupFileRequest = LookupFileRequest.newBuilder()
             .setKeyArgs(keyArgs)
             .build();
     OMRequest omRequest = createOMRequest(Type.LookupFile)
         .setLookupFileRequest(lookupFileRequest)
         .build();
-    OzoneManagerProtocolProtos.LookupFileResponse resp =
+    LookupFileResponse resp =
         handleError(submitRequest(omRequest)).getLookupFileResponse();
     return OmKeyInfo.getFromProtobuf(resp.getKeyInfo());
   }
@@ -1304,8 +1309,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
         .setType(args.getType())
         .setFactor(args.getFactor())
         .build();
-    OzoneManagerProtocolProtos.CreateFileRequest createFileRequest =
-        OzoneManagerProtocolProtos.CreateFileRequest.newBuilder()
+    CreateFileRequest createFileRequest = CreateFileRequest.newBuilder()
             .setKeyArgs(keyArgs)
             .setIsOverwrite(overWrite)
             .setIsRecursive(recursive)
@@ -1313,9 +1317,38 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
     OMRequest omRequest = createOMRequest(Type.CreateFile)
         .setCreateFileRequest(createFileRequest)
         .build();
-    OzoneManagerProtocolProtos.CreateFileResponse resp =
+    CreateFileResponse resp =
         handleError(submitRequest(omRequest)).getCreateFileResponse();
     return new OpenKeySession(resp.getID(),
         OmKeyInfo.getFromProtobuf(resp.getKeyInfo()), resp.getOpenVersion());
   }
+
+  @Override
+  public List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+      String startKey, long numEntries) throws IOException {
+    KeyArgs keyArgs = KeyArgs.newBuilder()
+        .setVolumeName(args.getVolumeName())
+        .setBucketName(args.getBucketName())
+        .setKeyName(args.getKeyName())
+        .build();
+    ListStatusRequest listStatusRequest =
+        ListStatusRequest.newBuilder()
+            .setKeyArgs(keyArgs)
+            .setRecursive(recursive)
+            .setStartKey(startKey)
+            .setNumEntries(numEntries)
+            .build();
+    OMRequest omRequest = createOMRequest(Type.ListStatus)
+        .setListStatusRequest(listStatusRequest)
+        .build();
+    ListStatusResponse listStatusResponse =
+        handleError(submitRequest(omRequest)).getListStatusResponse();
+    List<OzoneFileStatus> statusList =
+        new ArrayList<>(listStatusResponse.getStatusesCount());
+    for (OzoneFileStatusProto fileStatus : listStatusResponse
+        .getStatusesList()) {
+      statusList.add(OzoneFileStatus.getFromProtobuf(fileStatus));
+    }
+    return statusList;
+  }
 }

+ 32 - 33
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java

@@ -41,9 +41,15 @@ import org.apache.hadoop.ozone.protocol.proto
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 
+import java.util.List;
+import java.util.ArrayList;
+
 /**
  * Utilities for converting protobuf classes.
  */
@@ -59,7 +65,7 @@ public final class OMPBHelper {
    */
   public static OzoneAclInfo convertOzoneAcl(OzoneAcl acl) {
     OzoneAclInfo.OzoneAclType aclType;
-    switch(acl.getType()) {
+    switch (acl.getType()) {
     case USER:
       aclType = OzoneAclType.USER;
       break;
@@ -69,27 +75,24 @@ public final class OMPBHelper {
     case WORLD:
       aclType = OzoneAclType.WORLD;
       break;
-    default:
-      throw new IllegalArgumentException("ACL type is not recognized");
-    }
-    OzoneAclInfo.OzoneAclRights aclRights;
-    switch(acl.getRights()) {
-    case READ:
-      aclRights = OzoneAclRights.READ;
+    case ANONYMOUS:
+      aclType = OzoneAclType.ANONYMOUS;
       break;
-    case WRITE:
-      aclRights = OzoneAclRights.WRITE;
-      break;
-    case READ_WRITE:
-      aclRights = OzoneAclRights.READ_WRITE;
+    case CLIENT_IP:
+      aclType = OzoneAclType.CLIENT_IP;
       break;
     default:
-      throw new IllegalArgumentException("ACL right is not recognized");
+      throw new IllegalArgumentException("ACL type is not recognized");
+    }
+    List<OzoneAclRights> aclRights = new ArrayList<>();
+
+    for (ACLType right : acl.getRights()) {
+      aclRights.add(OzoneAclRights.valueOf(right.name()));
     }
 
     return OzoneAclInfo.newBuilder().setType(aclType)
         .setName(acl.getName())
-        .setRights(aclRights)
+        .addAllRights(aclRights)
         .build();
   }
 
@@ -98,35 +101,31 @@ public final class OMPBHelper {
    * @return OzoneAcl
    */
   public static OzoneAcl convertOzoneAcl(OzoneAclInfo aclInfo) {
-    OzoneAcl.OzoneACLType aclType;
-    switch(aclInfo.getType()) {
+    ACLIdentityType aclType;
+    switch (aclInfo.getType()) {
     case USER:
-      aclType = OzoneAcl.OzoneACLType.USER;
+      aclType = ACLIdentityType.USER;
       break;
     case GROUP:
-      aclType = OzoneAcl.OzoneACLType.GROUP;
+      aclType = ACLIdentityType.GROUP;
       break;
     case WORLD:
-      aclType = OzoneAcl.OzoneACLType.WORLD;
+      aclType = ACLIdentityType.WORLD;
       break;
-    default:
-      throw new IllegalArgumentException("ACL type is not recognized");
-    }
-    OzoneAcl.OzoneACLRights aclRights;
-    switch(aclInfo.getRights()) {
-    case READ:
-      aclRights = OzoneAcl.OzoneACLRights.READ;
+    case ANONYMOUS:
+      aclType = ACLIdentityType.ANONYMOUS;
       break;
-    case WRITE:
-      aclRights = OzoneAcl.OzoneACLRights.WRITE;
-      break;
-    case READ_WRITE:
-      aclRights = OzoneAcl.OzoneACLRights.READ_WRITE;
+    case CLIENT_IP:
+      aclType = ACLIdentityType.CLIENT_IP;
       break;
     default:
-      throw new IllegalArgumentException("ACL right is not recognized");
+      throw new IllegalArgumentException("ACL type is not recognized");
     }
 
+    List<IAccessAuthorizer.ACLType> aclRights = new ArrayList<>();
+    for (OzoneAclRights acl : aclInfo.getRightsList()) {
+      aclRights.add(ACLType.valueOf(acl.name()));
+    }
     return new OzoneAcl(aclType, aclInfo.getName(), aclRights);
   }
 

+ 16 - 4
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/IAccessAuthorizer.java

@@ -20,6 +20,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ozone.OzoneConsts;
 
+import java.util.List;
+
 /**
  * Public API for Ozone ACLs. Security providers providing support for Ozone
  * ACLs should implement this.
@@ -84,7 +86,8 @@ public interface IAccessAuthorizer {
       case OzoneConsts.OZONE_ACL_NONE:
         return ACLType.NONE;
       default:
-        throw new IllegalArgumentException("ACL right is not recognized");
+        throw new IllegalArgumentException(type + " ACL right is not " +
+            "recognized");
       }
 
     }
@@ -92,10 +95,18 @@ public interface IAccessAuthorizer {
     /**
      * Returns String representation of ACL rights.
      *
-     * @param acl ACLType
+     * @param acls ACLType
      * @return String representation of acl
      */
-    public static String getACLRightsString(ACLType acl) {
+    public static String getACLString(List<ACLType> acls) {
+      StringBuffer sb = new StringBuffer();
+      acls.forEach(acl -> {
+        sb.append(getAclString(acl));
+      });
+      return sb.toString();
+    }
+
+    public static String getAclString(ACLType acl) {
       switch (acl) {
       case READ:
         return OzoneConsts.OZONE_ACL_READ;
@@ -129,7 +140,8 @@ public interface IAccessAuthorizer {
     USER(OzoneConsts.OZONE_ACL_USER_TYPE),
     GROUP(OzoneConsts.OZONE_ACL_GROUP_TYPE),
     CLIENT_IP(OzoneConsts.OZONE_ACL_IP_TYPE),
-    WORLD(OzoneConsts.OZONE_ACL_WORLD_TYPE);
+    WORLD(OzoneConsts.OZONE_ACL_WORLD_TYPE),
+    ANONYMOUS(OzoneConsts.OZONE_ACL_ANONYMOUS_TYPE);
 
     @Override
     public String toString() {

+ 65 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/acl/OzoneAclConfig.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.security.acl;
+
+import org.apache.hadoop.hdds.conf.Config;
+import org.apache.hadoop.hdds.conf.ConfigGroup;
+import org.apache.hadoop.hdds.conf.ConfigTag;
+import org.apache.hadoop.hdds.conf.ConfigType;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
+
+/**
+ * Ozone ACL config pojo.
+ * */
+@ConfigGroup(prefix = "ozone.om")
+public class OzoneAclConfig {
+  // OM Default user/group permissions
+  private ACLType userDefaultRights = ACLType.ALL;
+  private ACLType groupDefaultRights = ACLType.ALL;
+
+  @Config(key = "user.rights",
+      defaultValue = "ALL",
+      type = ConfigType.STRING,
+      tags = {ConfigTag.OM, ConfigTag.SECURITY},
+      description = "Default user permissions set for an object in " +
+          "OzoneManager."
+  )
+  public void setUserDefaultRights(String userRights) {
+    this.userDefaultRights = ACLType.valueOf(userRights);
+  }
+
+  @Config(key = "group.rights",
+      defaultValue = "ALL",
+      type = ConfigType.STRING,
+      tags = {ConfigTag.OM, ConfigTag.SECURITY},
+      description = "Default group permissions set for an object in " +
+          "OzoneManager."
+  )
+  public void setGroupDefaultRights(String groupRights) {
+    this.groupDefaultRights = ACLType.valueOf(groupRights);
+  }
+
+  public ACLType getUserDefaultRights() {
+    return userDefaultRights;
+  }
+
+  public ACLType getGroupDefaultRights() {
+    return groupDefaultRights;
+  }
+
+}

+ 32 - 5
hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto

@@ -85,6 +85,7 @@ enum Type {
   CreateDirectory = 71;
   CreateFile = 72;
   LookupFile = 73;
+  ListStatus = 74;
 }
 
 message OMRequest {
@@ -141,6 +142,7 @@ message OMRequest {
   optional CreateDirectoryRequest           createDirectoryRequest         = 71;
   optional CreateFileRequest                createFileRequest              = 72;
   optional LookupFileRequest                lookupFileRequest              = 73;
+  optional ListStatusRequest                listStatusRequest              = 74;
 }
 
 message OMResponse {
@@ -200,6 +202,7 @@ message OMResponse {
   optional CreateDirectoryResponse            createDirectoryResponse      = 71;
   optional CreateFileResponse                 createFileResponse           = 72;
   optional LookupFileResponse                 lookupFileResponse           = 73;
+  optional ListStatusResponse                 listStatusResponse           = 74;
 }
 
 enum Status {
@@ -448,15 +451,24 @@ message OzoneAclInfo {
         USER = 1;
         GROUP = 2;
         WORLD = 3;
+        ANONYMOUS = 4;
+        CLIENT_IP = 5;
     }
+
     enum OzoneAclRights {
-        READ = 1;
-        WRITE = 2;
-        READ_WRITE = 3;
+        CREATE = 1;
+        LIST = 2;
+        DELETE = 3;
+        READ = 4;
+        WRITE = 5;
+        READ_ACL = 6;
+        WRITE_ACL = 7;
+        ALL = 8;
+        NONE = 9;
     }
     required OzoneAclType type = 1;
     required string name = 2;
-    required OzoneAclRights rights = 3;
+    repeated OzoneAclRights rights = 3;
 }
 
 message CreateBucketRequest {
@@ -561,12 +573,16 @@ message KeyInfo {
     optional FileEncryptionInfoProto fileEncryptionInfo = 12;
 }
 
+message OzoneFileStatusProto {
+    required hadoop.fs.FileStatusProto status = 1;
+}
+
 message GetFileStatusRequest {
     required KeyArgs keyArgs = 1;
 }
 
 message GetFileStatusResponse {
-    required hadoop.fs.FileStatusProto status = 1;
+    required OzoneFileStatusProto status = 1;
 }
 
 message CreateDirectoryRequest {
@@ -599,6 +615,17 @@ message LookupFileResponse {
     optional KeyInfo keyInfo = 1;
 }
 
+message ListStatusRequest {
+    required KeyArgs keyArgs = 1;
+    required bool recursive = 2;
+    required string startKey = 3;
+    required uint64 numEntries = 4;
+}
+
+message ListStatusResponse {
+    repeated OzoneFileStatusProto statuses = 1;
+}
+
 message CreateKeyRequest {
     required KeyArgs keyArgs = 1;
 }

+ 55 - 24
hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java → hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOzoneAcls.java

@@ -18,16 +18,20 @@
 
 package org.apache.hadoop.ozone;
 
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
+
 import org.junit.Test;
 
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Set;
 
+import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 /**
- * This class is to test acl stoarge and retreival in ozone store.
+ * This class is to test acl storage and retrieval in ozone store.
  */
 public class TestOzoneAcls {
 
@@ -39,8 +43,8 @@ public class TestOzoneAcls {
     testMatrix.put("user:bilbo:r", Boolean.TRUE);
     testMatrix.put("user:bilbo:w", Boolean.TRUE);
     testMatrix.put("user:bilbo:rw", Boolean.TRUE);
-    testMatrix.put("user:bilbo:wr", Boolean.TRUE);
-    testMatrix.put("    user:bilbo:wr   ", Boolean.TRUE);
+    testMatrix.put("user:bilbo:a", Boolean.TRUE);
+    testMatrix.put("    user:bilbo:a   ", Boolean.TRUE);
 
 
     // ACLs makes no judgement on the quality of
@@ -53,7 +57,16 @@ public class TestOzoneAcls {
     testMatrix.put("", Boolean.FALSE);
     testMatrix.put(null, Boolean.FALSE);
     testMatrix.put(" user:bilbo:", Boolean.FALSE);
-    testMatrix.put(" user:bilbo:rx", Boolean.FALSE);
+    testMatrix.put(" user:bilbo:rx", Boolean.TRUE);
+    testMatrix.put(" user:bilbo:rwdlncxy", Boolean.TRUE);
+    testMatrix.put(" group:bilbo:rwdlncxy", Boolean.TRUE);
+    testMatrix.put(" world::rwdlncxy", Boolean.TRUE);
+    testMatrix.put(" user:bilbo:rncxy", Boolean.TRUE);
+    testMatrix.put(" group:bilbo:ncxy", Boolean.TRUE);
+    testMatrix.put(" world::ncxy", Boolean.TRUE);
+    testMatrix.put(" user:bilbo:rwcxy", Boolean.TRUE);
+    testMatrix.put(" group:bilbo:rwcxy", Boolean.TRUE);
+    testMatrix.put(" world::rwcxy", Boolean.TRUE);
     testMatrix.put(" user:bilbo:mk", Boolean.FALSE);
     testMatrix.put(" user::rw", Boolean.FALSE);
     testMatrix.put("user11:bilbo:rw", Boolean.FALSE);
@@ -62,12 +75,12 @@ public class TestOzoneAcls {
     testMatrix.put(" group:hobbit:r", Boolean.TRUE);
     testMatrix.put(" group:hobbit:w", Boolean.TRUE);
     testMatrix.put(" group:hobbit:rw", Boolean.TRUE);
-    testMatrix.put(" group:hobbit:wr", Boolean.TRUE);
+    testMatrix.put(" group:hobbit:a", Boolean.TRUE);
     testMatrix.put(" group:*:rw", Boolean.TRUE);
     testMatrix.put(" group:~!:rw", Boolean.TRUE);
 
     testMatrix.put(" group:hobbit:", Boolean.FALSE);
-    testMatrix.put(" group:hobbit:rx", Boolean.FALSE);
+    testMatrix.put(" group:hobbit:rx", Boolean.TRUE);
     testMatrix.put(" group:hobbit:mk", Boolean.FALSE);
     testMatrix.put(" group::", Boolean.FALSE);
     testMatrix.put(" group::rw", Boolean.FALSE);
@@ -77,14 +90,14 @@ public class TestOzoneAcls {
     testMatrix.put("JUNK group:hobbit:r", Boolean.FALSE);
     testMatrix.put("JUNK group:hobbit:w", Boolean.FALSE);
     testMatrix.put("JUNK group:hobbit:rw", Boolean.FALSE);
-    testMatrix.put("JUNK group:hobbit:wr", Boolean.FALSE);
+    testMatrix.put("JUNK group:hobbit:a", Boolean.FALSE);
     testMatrix.put("JUNK group:*:rw", Boolean.FALSE);
     testMatrix.put("JUNK group:~!:rw", Boolean.FALSE);
 
     testMatrix.put(" world::r", Boolean.TRUE);
     testMatrix.put(" world::w", Boolean.TRUE);
     testMatrix.put(" world::rw", Boolean.TRUE);
-    testMatrix.put(" world::wr", Boolean.TRUE);
+    testMatrix.put(" world::a", Boolean.TRUE);
 
     testMatrix.put(" world:bilbo:w", Boolean.FALSE);
     testMatrix.put(" world:bilbo:rw", Boolean.FALSE);
@@ -97,7 +110,7 @@ public class TestOzoneAcls {
         try {
           OzoneAcl.parseAcl(key);
           // should never get here since parseAcl will throw
-          fail("An exception was expected but did not happen.");
+          fail("An exception was expected but did not happen. Key: " + key);
         } catch (IllegalArgumentException e) {
           // nothing to do
         }
@@ -109,33 +122,51 @@ public class TestOzoneAcls {
   public void testAclValues() {
     OzoneAcl acl = OzoneAcl.parseAcl("user:bilbo:rw");
     assertEquals(acl.getName(), "bilbo");
-    assertEquals(OzoneAcl.OzoneACLRights.READ_WRITE, acl.getRights());
-    assertEquals(OzoneAcl.OzoneACLType.USER, acl.getType());
+    assertEquals(Arrays.asList(READ, WRITE), acl.getRights());
+    assertEquals(ACLIdentityType.USER, acl.getType());
 
-    acl = OzoneAcl.parseAcl("user:bilbo:wr");
+    acl = OzoneAcl.parseAcl("user:bilbo:a");
     assertEquals("bilbo", acl.getName());
-    assertEquals(OzoneAcl.OzoneACLRights.READ_WRITE, acl.getRights());
-    assertEquals(OzoneAcl.OzoneACLType.USER, acl.getType());
+    assertEquals(Arrays.asList(ALL), acl.getRights());
+    assertEquals(ACLIdentityType.USER, acl.getType());
 
     acl = OzoneAcl.parseAcl("user:bilbo:r");
     assertEquals("bilbo", acl.getName());
-    assertEquals(OzoneAcl.OzoneACLRights.READ, acl.getRights());
-    assertEquals(OzoneAcl.OzoneACLType.USER, acl.getType());
+    assertEquals(Arrays.asList(READ), acl.getRights());
+    assertEquals(ACLIdentityType.USER, acl.getType());
 
     acl = OzoneAcl.parseAcl("user:bilbo:w");
     assertEquals("bilbo", acl.getName());
-    assertEquals(OzoneAcl.OzoneACLRights.WRITE, acl.getRights());
-    assertEquals(OzoneAcl.OzoneACLType.USER, acl.getType());
+    assertEquals(Arrays.asList(WRITE), acl.getRights());
+    assertEquals(ACLIdentityType.USER, acl.getType());
 
-    acl = OzoneAcl.parseAcl("group:hobbit:wr");
+    acl = OzoneAcl.parseAcl("group:hobbit:a");
     assertEquals(acl.getName(), "hobbit");
-    assertEquals(OzoneAcl.OzoneACLRights.READ_WRITE, acl.getRights());
-    assertEquals(OzoneAcl.OzoneACLType.GROUP, acl.getType());
+    assertEquals(Arrays.asList(ALL), acl.getRights());
+    assertEquals(ACLIdentityType.GROUP, acl.getType());
+
+    acl = OzoneAcl.parseAcl("world::a");
+    assertEquals(acl.getName(), "");
+    assertEquals(Arrays.asList(ALL), acl.getRights());
+    assertEquals(ACLIdentityType.WORLD, acl.getType());
+
+    acl = OzoneAcl.parseAcl("user:bilbo:rwdlncxy");
+    assertEquals(acl.getName(), "bilbo");
+    assertEquals(Arrays.asList(READ, WRITE, DELETE, LIST, NONE, CREATE,
+        READ_ACL, WRITE_ACL), acl.getRights());
+    assertEquals(ACLIdentityType.USER, acl.getType());
+
+    acl = OzoneAcl.parseAcl("group:hadoop:rwdlncxy");
+    assertEquals(acl.getName(), "hadoop");
+    assertEquals(Arrays.asList(READ, WRITE, DELETE, LIST, NONE, CREATE,
+        READ_ACL, WRITE_ACL), acl.getRights());
+    assertEquals(ACLIdentityType.GROUP, acl.getType());
 
-    acl = OzoneAcl.parseAcl("world::wr");
+    acl = OzoneAcl.parseAcl("world::rwdlncxy");
     assertEquals(acl.getName(), "");
-    assertEquals(OzoneAcl.OzoneACLRights.READ_WRITE, acl.getRights());
-    assertEquals(OzoneAcl.OzoneACLType.WORLD, acl.getType());
+    assertEquals(Arrays.asList(READ, WRITE, DELETE, LIST, NONE, CREATE,
+        READ_ACL, WRITE_ACL), acl.getRights());
+    assertEquals(ACLIdentityType.WORLD, acl.getType());
   }
 
 }

+ 21 - 0
hadoop-ozone/dev-support/intellij/install-runconfigs.sh

@@ -0,0 +1,21 @@
+#!/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.
+SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+SRC_DIR="$SCRIPT_DIR/runConfigurations"
+DEST_DIR="$SCRIPT_DIR/../../../.idea/runConfigurations/"
+mkdir -p "$DEST_DIR"
+#shellcheck disable=SC2010
+ls -1 "$SRC_DIR" | grep -v ozone-site.xml | xargs -n1 -I FILE cp "$SRC_DIR/FILE" "$DEST_DIR"

+ 18 - 0
hadoop-ozone/dev-support/intellij/log4j.properties

@@ -0,0 +1,18 @@
+#   Licensed 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.
+# log4j configuration used during build and unit tests
+log4j.rootLogger=INFO,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+log4j.logger.io.jagertraecing=DEBUG

+ 66 - 0
hadoop-ozone/dev-support/intellij/ozone-site.xml

@@ -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.
+-->
+<configuration>
+  <property>
+    <name>hdds.profiler.endpoint.enabled</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>ozone.scm.block.client.address</name>
+    <value>localhost</value>
+  </property>
+  <property>
+    <name>ozone.enabled</name>
+    <value>True</value>
+  </property>
+  <property>
+    <name>ozone.scm.datanode.id</name>
+    <value>/tmp/datanode.id</value>
+  </property>
+  <property>
+    <name>ozone.scm.client.address</name>
+    <value>localhost</value>
+  </property>
+  <property>
+    <name>ozone.metadata.dirs</name>
+    <value>/tmp/metadata</value>
+  </property>
+  <property>
+    <name>ozone.scm.names</name>
+    <value>localhost</value>
+  </property>
+  <property>
+    <name>ozone.om.address</name>
+    <value>localhost</value>
+  </property>
+  <property>
+    <name>ozone.enabled</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>ozone.scm.container.size</name>
+    <value>10MB</value>
+  </property>
+  <property>
+    <name>ozone.scm.block.size</name>
+    <value>1MB</value>
+  </property>
+  <property>
+    <name>hdds.datanode.storage.utilization.critical.threshold</name>
+    <value>0.99</value>
+  </property>
+</configuration>

+ 33 - 0
hadoop-ozone/dev-support/intellij/runConfigurations/Datanode.xml

@@ -0,0 +1,33 @@
+<!--
+   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.
+-->
+<component name="ProjectRunConfigurationManager">
+  <configuration default="false" name="Datanode" type="Application" factoryName="Application">
+    <option name="MAIN_CLASS_NAME" value="org.apache.hadoop.ozone.HddsDatanodeService" />
+    <module name="hadoop-ozone-datanode" />
+    <option name="PROGRAM_PARAMETERS" value="-conf=hadoop-ozone/dev-support/intellij/ozone-site.xml" />
+    <option name="VM_PARAMETERS" value="-Dlog4j.configuration=file:hadoop-ozone/dev-support/intellij/log4j.properties" />
+    <extension name="coverage">
+      <pattern>
+        <option name="PATTERN" value="org.apache.hadoop.ozone.*" />
+        <option name="ENABLED" value="true" />
+      </pattern>
+    </extension>
+    <method v="2">
+      <option name="Make" enabled="true" />
+    </method>
+  </configuration>
+</component>

+ 33 - 0
hadoop-ozone/dev-support/intellij/runConfigurations/FreonStandalone.xml

@@ -0,0 +1,33 @@
+<!--
+   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.
+-->
+<component name="ProjectRunConfigurationManager">
+  <configuration default="false" name="FreonStandalone" type="Application" factoryName="Application">
+    <option name="MAIN_CLASS_NAME" value="org.apache.hadoop.ozone.freon.Freon" />
+    <module name="hadoop-ozone-tools" />
+    <option name="PROGRAM_PARAMETERS" value="-conf=hadoop-ozone/dev-support/intellij/ozone-site.xml rk" />
+    <option name="VM_PARAMETERS" value="-Dlog4j.configuration=file:hadoop-ozone/dev-support/intellij/log4j.properties" />
+    <extension name="coverage">
+      <pattern>
+        <option name="PATTERN" value="org.apache.hadoop.ozone.*" />
+        <option name="ENABLED" value="true" />
+      </pattern>
+    </extension>
+    <method v="2">
+      <option name="Make" enabled="true" />
+    </method>
+  </configuration>
+</component>

+ 33 - 0
hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManager.xml

@@ -0,0 +1,33 @@
+<!--
+   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.
+-->
+<component name="ProjectRunConfigurationManager">
+  <configuration default="false" name="OzoneManager" type="Application" factoryName="Application">
+    <option name="MAIN_CLASS_NAME" value="org.apache.hadoop.ozone.om.OzoneManager" />
+    <module name="hadoop-ozone-ozone-manager" />
+    <option name="PROGRAM_PARAMETERS" value="-conf=hadoop-ozone/dev-support/intellij/ozone-site.xml" />
+    <option name="VM_PARAMETERS" value="-Dlog4j.configuration=file:hadoop-ozone/dev-support/intellij/log4j.properties" />
+    <extension name="coverage">
+      <pattern>
+        <option name="PATTERN" value="org.apache.hadoop.ozone.*" />
+        <option name="ENABLED" value="true" />
+      </pattern>
+    </extension>
+    <method v="2">
+      <option name="Make" enabled="true" />
+    </method>
+  </configuration>
+</component>

Vissa filer visades inte eftersom för många filer har ändrats