Jelajahi Sumber

HDFS-12908. Ozone: write chunk call fails because of Metrics registry exception. Contributed by Mukul Kumar Singh.

Nanda kumar 7 tahun lalu
induk
melakukan
ce19f09bd0

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java

@@ -76,7 +76,9 @@ public final class OzoneConsts {
   /**
    * level DB names used by SCM and data nodes.
    */
-  public static final String CONTAINER_DB = "container.db";
+  public static final String CONTAINER_DB_SUFFIX = "container.db";
+  public static final String SCM_CONTAINER_DB = "scm-" + CONTAINER_DB_SUFFIX;
+  public static final String DN_CONTAINER_DB = "-dn-"+ CONTAINER_DB_SUFFIX;
   public static final String BLOCK_DB = "block.db";
   public static final String NODEPOOL_DB = "nodepool.db";
   public static final String OPEN_CONTAINERS_DB = "openContainers.db";

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java

@@ -230,13 +230,18 @@ public final class ContainerUtils {
     }
   }
 
+  public static String getContainerDbFileName(String containerName) {
+    return containerName + OzoneConsts.DN_CONTAINER_DB;
+  }
+
   /**
    * creates a Metadata DB for the specified container.
    *
    * @param containerPath - Container Path.
    * @throws IOException
    */
-  public static Path createMetadata(Path containerPath, Configuration conf)
+  public static Path createMetadata(Path containerPath, String containerName,
+      Configuration conf)
       throws IOException {
     Logger log = LoggerFactory.getLogger(ContainerManagerImpl.class);
     Preconditions.checkNotNull(containerPath);
@@ -250,7 +255,8 @@ public final class ContainerUtils {
     MetadataStore store = MetadataStoreBuilder.newBuilder()
         .setConf(conf)
         .setCreateIfMissing(true)
-        .setDbFile(metadataPath.resolve(OzoneConsts.CONTAINER_DB).toFile())
+        .setDbFile(metadataPath
+            .resolve(getContainerDbFileName(containerName)).toFile())
         .build();
 
     // we close since the SCM pre-creates containers.

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java

@@ -60,7 +60,7 @@ import java.util.HashSet;
 import java.util.Set;
 
 import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_SUFFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.KSM_DB_NAME;
 import static org.apache.hadoop.ozone.OzoneConsts.KSM_USER_PREFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.KSM_BUCKET_PREFIX;
@@ -293,7 +293,7 @@ public class SQLCLI  extends Configured implements Tool {
       }
     }
     LOG.info("Parent path [{}] db name [{}]", parentPath, dbName);
-    if (dbName.toString().equals(CONTAINER_DB)) {
+    if (dbName.toString().endsWith(CONTAINER_DB_SUFFIX)) {
       LOG.info("Converting container DB");
       convertContainerDB(dbPath, outPath);
     } else if (dbName.toString().equals(BLOCK_DB)) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java

@@ -55,7 +55,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CONTAINER_DB;
 import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE;
 
 /**
@@ -102,7 +102,7 @@ public class ContainerMapping implements Mapping {
     File metaDir = OzoneUtils.getOzoneMetaDirPath(conf);
 
     // Write the container name to pipeline mapping.
-    File containerDBPath = new File(metaDir, CONTAINER_DB);
+    File containerDBPath = new File(metaDir, SCM_CONTAINER_DB);
     containerStore =
         MetadataStoreBuilder.newBuilder()
             .setConf(conf)

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/utils/RocksDBStore.java

@@ -23,11 +23,8 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
-import org.rocksdb.DBOptions;
 import org.rocksdb.RocksIterator;
 import org.rocksdb.Options;
-import org.rocksdb.Statistics;
-import org.rocksdb.StatsLevel;
 import org.rocksdb.WriteOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
@@ -76,6 +73,10 @@ public class RocksDBStore implements MetadataStore {
         jmxProperties.put("dbName", dbFile.getName());
         statMBeanName = MBeans.register("Ozone", "RocksDbStore", jmxProperties,
             new RocksDBStoreMBean(dbOptions.statistics()));
+        if (statMBeanName == null) {
+          LOG.warn("jmx registration failed during RocksDB init, db path :{}",
+              dbFile.getAbsolutePath());
+        }
       }
     } catch (RocksDBException e) {
       throw new IOException(

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java

@@ -164,7 +164,7 @@ public class TestContainerPersistence {
     }
 
     // Clean up SCM metadata
-    log.info("Deletting {}", path);
+    log.info("Deleting {}", path);
     FileUtils.deleteDirectory(new File(path));
 
     // Clean up SCM datanode container metadata/data

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java

@@ -53,7 +53,7 @@ import java.util.Collection;
 import java.util.HashMap;
 
 import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CONTAINER_DB;
 import static org.apache.hadoop.ozone.OzoneConsts.KB;
 import static org.apache.hadoop.ozone.OzoneConsts.NODEPOOL_DB;
 //import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
@@ -234,7 +234,7 @@ public class TestContainerSQLCli {
     // TODO : the following will fail due to empty Datanode list, need to fix.
     //String dnUUID = cluster.getDataNodes().get(0).getDatanodeUuid();
     String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
-    String dbPath = dbRootPath + "/" + CONTAINER_DB;
+    String dbPath = dbRootPath + "/" + SCM_CONTAINER_DB;
     String[] args = {"-p", dbPath, "-o", dbOutPath};
     Connection conn;
     String sql;