You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by na...@apache.org on 2019/04/12 09:30:14 UTC

[hadoop] branch trunk updated: HDDS-1363. ozone.metadata.dirs doesn't pick multiple dirs (#691)

This is an automated email from the ASF dual-hosted git repository.

nanda pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 3b08ac4  HDDS-1363. ozone.metadata.dirs doesn't pick multiple dirs (#691)
3b08ac4 is described below

commit 3b08ac46591e11323d8b1b1be742028bf5502bc0
Author: Doroszlai, Attila <64...@users.noreply.github.com>
AuthorDate: Fri Apr 12 11:30:08 2019 +0200

    HDDS-1363. ozone.metadata.dirs doesn't pick multiple dirs (#691)
---
 .../java/org/apache/hadoop/hdds/HddsUtils.java     |  24 ----
 .../org/apache/hadoop/hdds/scm/HddsServerUtil.java |  22 ++++
 .../apache/hadoop/ozone/HddsDatanodeService.java   |   5 +-
 .../common/states/datanode/InitDatanodeState.java  |   4 +-
 .../org/apache/hadoop/hdds/server/ServerUtils.java |  25 ++---
 .../apache/hadoop/hdds/server/TestServerUtils.java | 123 +++++++++++++++++++++
 .../java/org/apache/hadoop/hdds/scm/ScmUtils.java  |  20 +---
 .../hadoop/hdds/scm/TestHddsServerUtils.java       |  49 --------
 .../apache/hadoop/ozone/web/utils/OzoneUtils.java  |  10 --
 .../org/apache/hadoop/ozone/recon/ReconUtils.java  |   2 +-
 10 files changed, 164 insertions(+), 120 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
index 2ca42d5..92ed9b6 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
@@ -24,7 +24,6 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
-import java.nio.file.Paths;
 import java.util.Calendar;
 import java.util.Collection;
 import java.util.HashSet;
@@ -50,7 +49,6 @@ import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 
-import com.google.common.base.Strings;
 import com.google.common.net.HostAndPort;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
@@ -313,28 +311,6 @@ public final class HddsUtils {
 
 
   /**
-   * Get the path for datanode id file.
-   *
-   * @param conf - Configuration
-   * @return the path of datanode id as string
-   */
-  public static String getDatanodeIdFilePath(Configuration conf) {
-    String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
-    if (dataNodeIDPath == null) {
-      String metaPath = conf.get(HddsConfigKeys.OZONE_METADATA_DIRS);
-      if (Strings.isNullOrEmpty(metaPath)) {
-        // this means meta data is not found, in theory should not happen at
-        // this point because should've failed earlier.
-        throw new IllegalArgumentException("Unable to locate meta data" +
-            "directory when getting datanode id path");
-      }
-      dataNodeIDPath = Paths.get(metaPath,
-          ScmConfigKeys.OZONE_SCM_DATANODE_ID_PATH_DEFAULT).toString();
-    }
-    return dataNodeIDPath;
-  }
-
-  /**
    * Returns the hostname for this datanode. If the hostname is not
    * explicitly configured in the given config, then it is determined
    * via the DNS class.
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java
index 3ff6e66..9d1880c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java
@@ -372,4 +372,26 @@ public final class HddsServerUtil {
     File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
     return (new File(metaDirPath, "ratis")).getPath();
   }
+
+  /**
+   * Get the path for datanode id file.
+   *
+   * @param conf - Configuration
+   * @return the path of datanode id as string
+   */
+  public static String getDatanodeIdFilePath(Configuration conf) {
+    String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
+    if (dataNodeIDPath == null) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode id path");
+      }
+      dataNodeIDPath = new File(metaDirPath,
+          ScmConfigKeys.OZONE_SCM_DATANODE_ID_PATH_DEFAULT).toString();
+    }
+    return dataNodeIDPath;
+  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index 4ddfb2f..5228335 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -27,6 +27,7 @@ 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.scm.HddsServerUtil;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
@@ -308,7 +309,7 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
    */
   private DatanodeDetails initializeDatanodeDetails()
       throws IOException {
-    String idFilePath = HddsUtils.getDatanodeIdFilePath(conf);
+    String idFilePath = HddsServerUtil.getDatanodeIdFilePath(conf);
     if (idFilePath == null || idFilePath.isEmpty()) {
       LOG.error("A valid file path is needed for config setting {}",
           ScmConfigKeys.OZONE_SCM_DATANODE_ID);
@@ -338,7 +339,7 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
    */
   private void persistDatanodeDetails(DatanodeDetails dnDetails)
       throws IOException {
-    String idFilePath = HddsUtils.getDatanodeIdFilePath(conf);
+    String idFilePath = HddsServerUtil.getDatanodeIdFilePath(conf);
     if (idFilePath == null || idFilePath.isEmpty()) {
       LOG.error("A valid file path is needed for config setting {}",
           ScmConfigKeys.OZONE_SCM_DATANODE_ID);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
index 995f172..00e2bf0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
@@ -18,8 +18,8 @@ package org.apache.hadoop.ozone.container.common.states.datanode;
 
 import com.google.common.base.Strings;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.HddsServerUtil;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.statemachine
@@ -117,7 +117,7 @@ public class InitDatanodeState implements DatanodeState,
    * Persist DatanodeDetails to datanode.id file.
    */
   private void persistContainerDatanodeDetails() {
-    String dataNodeIDPath = HddsUtils.getDatanodeIdFilePath(conf);
+    String dataNodeIDPath = HddsServerUtil.getDatanodeIdFilePath(conf);
     if (Strings.isNullOrEmpty(dataNodeIDPath)) {
       LOG.error("A valid file path is needed for config setting {}",
           ScmConfigKeys.OZONE_SCM_DATANODE_ID);
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
index 22b50c0..a5aacd7 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
@@ -125,8 +125,8 @@ public final class ServerUtils {
    * @return
    */
   public static File getScmDbDir(Configuration conf) {
-    File metadataDir = getDirectoryFromConfig(conf, ScmConfigKeys
-        .OZONE_SCM_DB_DIRS, "SCM");
+    File metadataDir = getDirectoryFromConfig(conf,
+        ScmConfigKeys.OZONE_SCM_DB_DIRS, "SCM");
     if (metadataDir != null) {
       return metadataDir;
     }
@@ -146,8 +146,8 @@ public final class ServerUtils {
    * @return File created from the value of the key in conf.
    */
   public static File getDirectoryFromConfig(Configuration conf,
-                                             String key,
-                                             String componentName) {
+                                            String key,
+                                            String componentName) {
     final Collection<String> metadirs = conf.getTrimmedStringCollection(key);
 
     if (metadirs.size() > 1) {
@@ -162,10 +162,11 @@ public final class ServerUtils {
       if (!dbDirPath.exists() && !dbDirPath.mkdirs()) {
         throw new IllegalArgumentException("Unable to create directory " +
             dbDirPath + " specified in configuration setting " +
-            componentName);
+            key);
       }
       return dbDirPath;
     }
+
     return null;
   }
 
@@ -173,22 +174,16 @@ public final class ServerUtils {
    * Checks and creates Ozone Metadir Path if it does not exist.
    *
    * @param conf - Configuration
-   *
    * @return File MetaDir
+   * @throws IllegalArgumentException if the configuration setting is not set
    */
   public static File getOzoneMetaDirPath(Configuration conf) {
-    String metaDirPath = conf.getTrimmed(HddsConfigKeys.OZONE_METADATA_DIRS);
-
-    if (metaDirPath == null || metaDirPath.isEmpty()) {
+    File dirPath = getDirectoryFromConfig(conf,
+        HddsConfigKeys.OZONE_METADATA_DIRS, "Ozone");
+    if (dirPath == null) {
       throw new IllegalArgumentException(
           HddsConfigKeys.OZONE_METADATA_DIRS + " must be defined.");
     }
-
-    File dirPath = new File(metaDirPath);
-    if (!dirPath.exists() && !dirPath.mkdirs()) {
-      throw new IllegalArgumentException("Unable to create paths. Path: " +
-          dirPath);
-    }
     return dirPath;
   }
 
diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/TestServerUtils.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/TestServerUtils.java
new file mode 100644
index 0000000..9735d2c
--- /dev/null
+++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/TestServerUtils.java
@@ -0,0 +1,123 @@
+/*
+ * 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.hdds.server;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.test.PathUtils;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.File;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unit tests for {@link ServerUtils}.
+ */
+public class TestServerUtils {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  /**
+   * Test {@link ServerUtils#getScmDbDir}.
+   */
+  @Test
+  public void testGetScmDbDir() {
+    final File testDir = PathUtils.getTestDir(TestServerUtils.class);
+    final File dbDir = new File(testDir, "scmDbDir");
+    final File metaDir = new File(testDir, "metaDir");
+    final Configuration conf = new OzoneConfiguration();
+    conf.set(ScmConfigKeys.OZONE_SCM_DB_DIRS, dbDir.getPath());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());
+
+    try {
+      assertFalse(metaDir.exists());
+      assertFalse(dbDir.exists());
+      assertEquals(dbDir, ServerUtils.getScmDbDir(conf));
+      assertTrue(dbDir.exists());
+      assertFalse(metaDir.exists());
+    } 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(TestServerUtils.class);
+    final File metaDir = new File(testDir, "metaDir");
+    final Configuration conf = new OzoneConfiguration();
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());
+    try {
+      assertFalse(metaDir.exists());
+      assertEquals(metaDir, ServerUtils.getScmDbDir(conf));
+      assertTrue(metaDir.exists());
+    } finally {
+      FileUtils.deleteQuietly(metaDir);
+    }
+  }
+
+  @Test
+  public void testNoScmDbDirConfigured() {
+    thrown.expect(IllegalArgumentException.class);
+    ServerUtils.getScmDbDir(new OzoneConfiguration());
+  }
+
+  @Test
+  public void ozoneMetadataDirIsMandatory() {
+    thrown.expect(IllegalArgumentException.class);
+    ServerUtils.getOzoneMetaDirPath(new OzoneConfiguration());
+  }
+
+  @Test
+  public void ozoneMetadataDirAcceptsSingleItem() {
+    final File testDir = PathUtils.getTestDir(TestServerUtils.class);
+    final File metaDir = new File(testDir, "metaDir");
+    final Configuration conf = new OzoneConfiguration();
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());
+
+    try {
+      assertFalse(metaDir.exists());
+      assertEquals(metaDir, ServerUtils.getOzoneMetaDirPath(conf));
+      assertTrue(metaDir.exists());
+    } finally {
+      FileUtils.deleteQuietly(metaDir);
+    }
+  }
+
+  @Test
+  public void ozoneMetadataDirRejectsList() {
+    final Configuration conf = new OzoneConfiguration();
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, "/data/meta1,/data/meta2");
+    thrown.expect(IllegalArgumentException.class);
+
+    ServerUtils.getOzoneMetaDirPath(conf);
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java
index 5b17de9..3192a74 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java
@@ -29,7 +29,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.util.Collection;
 
 /**
  * SCM utility class.
@@ -55,22 +54,9 @@ public final class ScmUtils {
   }
 
   public static File getDBPath(Configuration conf, String dbDirectory) {
-    final Collection<String> dbDirs =
-        conf.getTrimmedStringCollection(dbDirectory);
-
-    if (dbDirs.size() > 1) {
-      throw new IllegalArgumentException(
-          "Bad configuration setting " + dbDirectory
-              + ". OM does not support multiple metadata dirs currently.");
-    }
-
-    if (dbDirs.size() == 1) {
-      final File dbDirPath = new File(dbDirs.iterator().next());
-      if (!dbDirPath.exists() && !dbDirPath.mkdirs()) {
-        throw new IllegalArgumentException(
-            "Unable to create directory " + dbDirPath
-                + " specified in configuration setting " + dbDirectory);
-      }
+    final File dbDirPath =
+        ServerUtils.getDirectoryFromConfig(conf, dbDirectory, "OM");
+    if (dbDirPath != null) {
       return dbDirPath;
     }
 
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestHddsServerUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestHddsServerUtils.java
index 6f894e8..f75bbe5 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestHddsServerUtils.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestHddsServerUtils.java
@@ -18,12 +18,8 @@
 
 package org.apache.hadoop.hdds.scm;
 
-import org.apache.commons.io.FileUtils;
 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.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -31,7 +27,6 @@ import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.net.InetSocketAddress;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY;
@@ -39,7 +34,6 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_ADDRES
 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.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 /**
  * Unit tests for {@link HddsServerUtil}.
@@ -160,47 +154,4 @@ 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());
-  }
 }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
index 22fff56..31188ee 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
@@ -114,16 +114,6 @@ public final class OzoneUtils {
   }
 
   /**
-   * Get the path for datanode id file.
-   *
-   * @param conf - Configuration
-   * @return the path of datanode id as string
-   */
-  public static String getDatanodeIdFilePath(Configuration conf) {
-    return HddsUtils.getDatanodeIdFilePath(conf);
-  }
-
-  /**
    * Convert time in millisecond to a human readable format required in ozone.
    * @return a human readable string for the input time
    */
diff --git a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
index 7bbbd92..324a369 100644
--- a/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
+++ b/hadoop-ozone/ozone-recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
@@ -63,7 +63,7 @@ public final class ReconUtils {
 
   /**
    * Get configured Recon DB directory value based on config. If not present,
-   * fallback to om.metadata.dirs
+   * fallback to ozone.metadata.dirs
    *
    * @param conf         configuration bag
    * @param dirConfigKey key to check


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org