You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by sa...@apache.org on 2020/07/22 13:00:47 UTC

[hadoop-ozone] 37/39: HDDS-3892. Datanode initialization is too slow when there are thousan… (#1147)

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

sammichen pushed a commit to branch ozone-0.6.0
in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git

commit 2a79c8977d85c6c6e86d840325250e09c3be0b27
Author: Sammi Chen <sa...@apache.org>
AuthorDate: Wed Jul 22 14:11:08 2020 +0800

    HDDS-3892. Datanode initialization is too slow when there are thousan… (#1147)
    
    (cherry picked from commit 783a18c4a47c2b1eccbc85611211753943598ea1)
---
 .../org/apache/hadoop/ozone/OzoneConfigKeys.java   |  3 +
 .../common/src/main/resources/ozone-default.xml    | 10 +++
 .../container/common/utils/ContainerCache.java     | 71 ++++++++++++++++------
 .../ozone/container/ozoneimpl/ContainerReader.java |  2 +
 .../ozone/container/ozoneimpl/OzoneContainer.java  |  3 +
 .../ozone/container/common/TestContainerCache.java | 52 ++++++++++++++++
 .../container/ozoneimpl/TestContainerReader.java   | 69 ++++++++++++++++++++-
 7 files changed, 189 insertions(+), 21 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index dc8b231..d89fef9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -95,6 +95,9 @@ public final class OzoneConfigKeys {
   public static final String OZONE_CONTAINER_CACHE_SIZE =
       "ozone.container.cache.size";
   public static final int OZONE_CONTAINER_CACHE_DEFAULT = 1024;
+  public static final String OZONE_CONTAINER_CACHE_LOCK_STRIPES =
+      "ozone.container.cache.lock.stripes";
+  public static final int OZONE_CONTAINER_CACHE_LOCK_STRIPES_DEFAULT = 1024;
 
   public static final String OZONE_SCM_BLOCK_SIZE =
       "ozone.scm.block.size";
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index b474ac3..a07807b 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -38,6 +38,16 @@
     </description>
   </property>
   <property>
+    <name>ozone.container.cache.lock.stripes</name>
+    <value>1024</value>
+    <tag>PERFORMANCE, CONTAINER, STORAGE</tag>
+    <description>Container DB open is an exclusive operation. We use a stripe
+      lock to guarantee that different threads can open different container DBs
+      concurrently, while for one container DB, only one thread can open it at
+      the same time. This setting controls the lock stripes.
+    </description>
+  </property>
+  <property>
     <name>dfs.container.ipc</name>
     <value>9859</value>
     <tag>OZONE, CONTAINER, MANAGEMENT</tag>
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
index d2d2901..f4d8f43 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
+import com.google.common.util.concurrent.Striped;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.utils.MetadataStore;
 import org.apache.hadoop.hdds.utils.MetadataStoreBuilder;
@@ -43,12 +44,14 @@ public final class ContainerCache extends LRUMap {
   private final Lock lock = new ReentrantLock();
   private static ContainerCache cache;
   private static final float LOAD_FACTOR = 0.75f;
+  private final Striped<Lock> rocksDBLock;
   /**
    * Constructs a cache that holds DBHandle references.
    */
-  private ContainerCache(int maxSize, float loadFactor, boolean
+  private ContainerCache(int maxSize, int stripes, float loadFactor, boolean
       scanUntilRemovable) {
     super(maxSize, loadFactor, scanUntilRemovable);
+    rocksDBLock = Striped.lazyWeakLock(stripes);
   }
 
   /**
@@ -63,7 +66,10 @@ public final class ContainerCache extends LRUMap {
     if (cache == null) {
       int cacheSize = conf.getInt(OzoneConfigKeys.OZONE_CONTAINER_CACHE_SIZE,
           OzoneConfigKeys.OZONE_CONTAINER_CACHE_DEFAULT);
-      cache = new ContainerCache(cacheSize, LOAD_FACTOR, true);
+      int stripes = conf.getInt(
+          OzoneConfigKeys.OZONE_CONTAINER_CACHE_LOCK_STRIPES,
+          OzoneConfigKeys.OZONE_CONTAINER_CACHE_LOCK_STRIPES_DEFAULT);
+      cache = new ContainerCache(cacheSize, stripes, LOAD_FACTOR, true);
     }
     return cache;
   }
@@ -117,30 +123,57 @@ public final class ContainerCache extends LRUMap {
       throws IOException {
     Preconditions.checkState(containerID >= 0,
         "Container ID cannot be negative.");
-    lock.lock();
+    ReferenceCountedDB db;
+    Lock containerLock = rocksDBLock.get(containerDBPath);
+    containerLock.lock();
     try {
-      ReferenceCountedDB db = (ReferenceCountedDB) this.get(containerDBPath);
+      lock.lock();
+      try {
+        db = (ReferenceCountedDB) this.get(containerDBPath);
+        if (db != null) {
+          db.incrementReference();
+          return db;
+        }
+      } finally {
+        lock.unlock();
+      }
 
-      if (db == null) {
+      try {
         MetadataStore metadataStore =
             MetadataStoreBuilder.newBuilder()
-            .setDbFile(new File(containerDBPath))
-            .setCreateIfMissing(false)
-            .setConf(conf)
-            .setDBType(containerDBType)
-            .build();
+                .setDbFile(new File(containerDBPath))
+                .setCreateIfMissing(false)
+                .setConf(conf)
+                .setDBType(containerDBType)
+                .build();
         db = new ReferenceCountedDB(metadataStore, containerDBPath);
-        this.put(containerDBPath, db);
+      } catch (Exception e) {
+        LOG.error("Error opening DB. Container:{} ContainerPath:{}",
+            containerID, containerDBPath, e);
+        throw e;
+      }
+
+      lock.lock();
+      try {
+        ReferenceCountedDB currentDB =
+            (ReferenceCountedDB) this.get(containerDBPath);
+        if (currentDB != null) {
+          // increment the reference before returning the object
+          currentDB.incrementReference();
+          // clean the db created in previous step
+          db.cleanup();
+          return currentDB;
+        } else {
+          this.put(containerDBPath, db);
+          // increment the reference before returning the object
+          db.incrementReference();
+          return db;
+        }
+      } finally {
+        lock.unlock();
       }
-      // increment the reference before returning the object
-      db.incrementReference();
-      return db;
-    } catch (Exception e) {
-      LOG.error("Error opening DB. Container:{} ContainerPath:{}",
-          containerID, containerDBPath, e);
-      throw e;
     } finally {
-      lock.unlock();
+      containerLock.unlock();
     }
   }
 
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index 1b9b3d6..fa63cf1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -120,6 +120,7 @@ public class ContainerReader implements Runnable {
       return;
     }
 
+    LOG.info("Start to verify containers on volume {}", hddsVolumeRootDir);
     for (File scmLoc : scmDir) {
       File currentDir = new File(scmLoc, Storage.STORAGE_DIR_CURRENT);
       File[] containerTopDirs = currentDir.listFiles();
@@ -144,6 +145,7 @@ public class ContainerReader implements Runnable {
         }
       }
     }
+    LOG.info("Finish verifying containers on volume {}", hddsVolumeRootDir);
   }
 
   private void verifyContainerFile(long containerID, File containerFile) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 62fd5a4..abe0382 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -163,6 +163,7 @@ public class OzoneContainer {
     Iterator<HddsVolume> volumeSetIterator = volumeSet.getVolumesList()
         .iterator();
     ArrayList<Thread> volumeThreads = new ArrayList<Thread>();
+    long startTime = System.currentTimeMillis();
 
     //TODO: diskchecker should be run before this, to see how disks are.
     // And also handle disk failure tolerance need to be added
@@ -183,6 +184,8 @@ public class OzoneContainer {
       Thread.currentThread().interrupt();
     }
 
+    LOG.info("Build ContainerSet costs {}s",
+        (System.currentTimeMillis() - startTime) / 1000);
   }
 
   /**
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestContainerCache.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestContainerCache.java
index 947a087..2e38990 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestContainerCache.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestContainerCache.java
@@ -31,6 +31,13 @@ import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
 import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 
 /**
@@ -63,6 +70,8 @@ public class TestContainerCache {
     conf.setInt(OzoneConfigKeys.OZONE_CONTAINER_CACHE_SIZE, 2);
 
     ContainerCache cache = ContainerCache.getInstance(conf);
+    cache.clear();
+    Assert.assertEquals(0, cache.size());
     File containerDir1 = new File(root, "cont1");
     File containerDir2 = new File(root, "cont2");
     File containerDir3 = new File(root, "cont3");
@@ -123,4 +132,47 @@ public class TestContainerCache {
     thrown.expect(IllegalArgumentException.class);
     db5.close();
   }
+
+  @Test
+  public void testConcurrentDBGet() throws Exception {
+    File root = new File(testRoot);
+    root.mkdirs();
+    root.deleteOnExit();
+
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.setInt(OzoneConfigKeys.OZONE_CONTAINER_CACHE_SIZE, 2);
+    ContainerCache cache = ContainerCache.getInstance(conf);
+    cache.clear();
+    Assert.assertEquals(0, cache.size());
+    File containerDir = new File(root, "cont1");
+    createContainerDB(conf, containerDir);
+    ExecutorService executorService = Executors.newFixedThreadPool(2);
+    Runnable task = () -> {
+      try {
+        ReferenceCountedDB db1 = cache.getDB(1, "RocksDB",
+            containerDir.getPath(), conf);
+        Assert.assertNotNull(db1);
+      } catch (IOException e) {
+        Assert.fail("Should get the DB instance");
+      }
+    };
+    List<Future> futureList = new ArrayList<>();
+    futureList.add(executorService.submit(task));
+    futureList.add(executorService.submit(task));
+    for (Future future: futureList) {
+      try {
+        future.get();
+      } catch (InterruptedException| ExecutionException e) {
+        Assert.fail("Should get the DB instance");
+      }
+    }
+
+    ReferenceCountedDB db = cache.getDB(1, "RocksDB",
+        containerDir.getPath(), conf);
+    db.close();
+    db.close();
+    db.close();
+    Assert.assertEquals(1, cache.size());
+    db.cleanup();
+  }
 }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerReader.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerReader.java
index e1c5f33..15c0236 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerReader.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerReader.java
@@ -22,15 +22,16 @@ import com.google.common.primitives.Longs;
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.StringUtils;
 import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
 import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
@@ -68,7 +69,7 @@ public class TestContainerReader {
   private MutableVolumeSet volumeSet;
   private HddsVolume hddsVolume;
   private ContainerSet containerSet;
-  private ConfigurationSource conf;
+  private OzoneConfiguration conf;
 
 
   private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
@@ -219,4 +220,68 @@ public class TestContainerReader {
           keyValueContainerData.getNumPendingDeletionBlocks());
     }
   }
+
+  @Test
+  public void testMultipleContainerReader() throws Exception {
+    final int volumeNum = 10;
+    StringBuffer datanodeDirs = new StringBuffer();
+    File[] volumeDirs = new File[volumeNum];
+    for (int i = 0; i < volumeNum; i++) {
+      volumeDirs[i] = tempDir.newFolder();
+      datanodeDirs = datanodeDirs.append(volumeDirs[i]).append(",");
+    }
+    conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY,
+        datanodeDirs.toString());
+    MutableVolumeSet volumeSets =
+        new MutableVolumeSet(datanodeId.toString(), conf);
+    ContainerCache cache = ContainerCache.getInstance(conf);
+    cache.clear();
+
+    RoundRobinVolumeChoosingPolicy policy =
+        new RoundRobinVolumeChoosingPolicy();
+
+    final int containerCount = 100;
+    blockCount = containerCount;
+    for (int i = 0; i < containerCount; i++) {
+      KeyValueContainerData keyValueContainerData =
+          new KeyValueContainerData(i, ChunkLayOutVersion.FILE_PER_BLOCK,
+              (long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(),
+              datanodeId.toString());
+
+      KeyValueContainer keyValueContainer =
+          new KeyValueContainer(keyValueContainerData,
+              conf);
+      keyValueContainer.create(volumeSets, policy, scmId);
+
+      List<Long> blkNames;
+      if (i % 2 == 0) {
+        blkNames = addBlocks(keyValueContainer, true);
+        markBlocksForDelete(keyValueContainer, true, blkNames, i);
+      } else {
+        blkNames = addBlocks(keyValueContainer, false);
+        markBlocksForDelete(keyValueContainer, false, blkNames, i);
+      }
+    }
+
+    List<HddsVolume> hddsVolumes = volumeSets.getVolumesList();
+    ContainerReader[] containerReaders = new ContainerReader[volumeNum];
+    Thread[] threads = new Thread[volumeNum];
+    for (int i = 0; i < volumeNum; i++) {
+      containerReaders[i] = new ContainerReader(volumeSets,
+          hddsVolumes.get(i), containerSet, conf);
+      threads[i] = new Thread(containerReaders[i]);
+    }
+    long startTime = System.currentTimeMillis();
+    for (int i = 0; i < volumeNum; i++) {
+      threads[i].start();
+    }
+    for (int i = 0; i < volumeNum; i++) {
+      threads[i].join();
+    }
+    System.out.println("Open " + volumeNum + " Volume with " + containerCount +
+        " costs " + (System.currentTimeMillis() - startTime) / 1000 + "s");
+    Assert.assertEquals(containerCount,
+        containerSet.getContainerMap().entrySet().size());
+    Assert.assertEquals(containerCount, cache.size());
+  }
 }


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