Browse Source

HDDS-92. Use DBType during parsing datanode .container files. Contributed by Bharat Viswanadham.

Xiaoyu Yao 7 years ago
parent
commit
ee1e0e2036

+ 33 - 7
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStoreBuilder.java

@@ -18,12 +18,15 @@
 
 package org.apache.hadoop.utils;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.iq80.leveldb.Options;
 import org.rocksdb.BlockBasedTableConfig;
 import org.rocksdb.Statistics;
 import org.rocksdb.StatsLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
@@ -44,10 +47,14 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
  */
 public class MetadataStoreBuilder {
 
+  @VisibleForTesting
+  static final Logger LOG =
+      LoggerFactory.getLogger(MetadataStoreBuilder.class);
   private File dbFile;
   private long cacheSize;
   private boolean createIfMissing = true;
   private Configuration conf;
+  private String dbType;
 
   public static MetadataStoreBuilder newBuilder() {
     return new MetadataStoreBuilder();
@@ -73,6 +80,17 @@ public class MetadataStoreBuilder {
     return this;
   }
 
+  /**
+   * Set the container DB Type.
+   * @param type
+   * @return MetadataStoreBuilder
+   */
+  public MetadataStoreBuilder setDBType(String type) {
+    this.dbType = type;
+    return this;
+  }
+
+
   public MetadataStore build() throws IOException {
     if (dbFile == null) {
       throw new IllegalArgumentException("Failed to build metadata store, "
@@ -81,18 +99,26 @@ public class MetadataStoreBuilder {
 
     // Build db store based on configuration
     MetadataStore store = null;
-    String impl = conf == null ?
-        OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT :
-        conf.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
-            OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
-    if (OZONE_METADATA_STORE_IMPL_LEVELDB.equals(impl)) {
+
+    if(dbType == null) {
+      LOG.debug("dbType is null, using ");
+      dbType = conf == null ?
+          OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT :
+          conf.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
+              OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
+      LOG.debug("dbType is null, using dbType {} from ozone configuration",
+          dbType);
+    } else {
+      LOG.debug("Using dbType {} for metastore", dbType);
+    }
+    if (OZONE_METADATA_STORE_IMPL_LEVELDB.equals(dbType)) {
       Options options = new Options();
       options.createIfMissing(createIfMissing);
       if (cacheSize > 0) {
         options.cacheSize(cacheSize);
       }
       store = new LevelDBStore(dbFile, options);
-    } else if (OZONE_METADATA_STORE_IMPL_ROCKSDB.equals(impl)) {
+    } else if (OZONE_METADATA_STORE_IMPL_ROCKSDB.equals(dbType)) {
       org.rocksdb.Options opts = new org.rocksdb.Options();
       opts.setCreateIfMissing(createIfMissing);
 
@@ -119,7 +145,7 @@ public class MetadataStoreBuilder {
           + OzoneConfigKeys.OZONE_METADATA_STORE_IMPL
           + ". Expecting " + OZONE_METADATA_STORE_IMPL_LEVELDB
           + " or " + OZONE_METADATA_STORE_IMPL_ROCKSDB
-          + ", but met " + impl);
+          + ", but met " + dbType);
     }
     return store;
   }

+ 53 - 4
hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java → hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.ozone;
+package org.apache.hadoop.utils;
 
 import com.google.common.collect.Lists;
 import org.apache.commons.io.FileUtils;
@@ -24,12 +24,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.utils.BatchOperation;
 import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
-import org.apache.hadoop.utils.MetadataStore;
-import org.apache.hadoop.utils.MetadataStoreBuilder;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -38,6 +36,7 @@ import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.slf4j.event.Level;
 
 import java.io.File;
 import java.io.IOException;
@@ -50,6 +49,7 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static junit.framework.TestCase.assertTrue;
 import static org.junit.runners.Parameterized.Parameters;
 
 /**
@@ -102,6 +102,55 @@ public class TestMetadataStore {
     }
   }
 
+  @Test
+  public void testMetaStoreConfigDifferentFromType() throws IOException {
+
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+    String dbType;
+    GenericTestUtils.setLogLevel(MetadataStoreBuilder.LOG, Level.DEBUG);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(MetadataStoreBuilder.LOG);
+    if(storeImpl.equals(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB)) {
+      dbType = "RocksDB";
+    } else {
+      dbType = "LevelDB";
+    }
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + dbType.toLowerCase() + "-test");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder().setConf(conf)
+        .setCreateIfMissing(true).setDbFile(dbDir).setDBType(dbType).build();
+    assertTrue(logCapturer.getOutput().contains("Using dbType " + dbType + "" +
+        " for metastore"));
+    dbStore.close();
+    dbStore.destroy();
+    FileUtils.deleteDirectory(dbDir);
+
+  }
+
+  @Test
+  public void testdbTypeNotSet() throws IOException {
+
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+    GenericTestUtils.setLogLevel(MetadataStoreBuilder.LOG, Level.DEBUG);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(MetadataStoreBuilder.LOG);
+
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase() + "-test");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder().setConf(conf)
+        .setCreateIfMissing(true).setDbFile(dbDir).build();
+    assertTrue(logCapturer.getOutput().contains("dbType is null, using dbType" +
+        " " + storeImpl));
+    dbStore.close();
+    dbStore.destroy();
+    FileUtils.deleteDirectory(dbDir);
+
+  }
+
   @After
   public void cleanup() throws IOException {
     store.close();

+ 2 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java

@@ -63,7 +63,8 @@ public final class KeyUtils {
     ContainerCache cache = ContainerCache.getInstance(conf);
     Preconditions.checkNotNull(cache);
     try {
-      return cache.getDB(container.getContainerID(), container.getDBPath());
+      return cache.getDB(container.getContainerID(), container
+          .getContainerDBType(), container.getDBPath());
     } catch (IOException ex) {
       String message =
           String.format("Unable to open DB. DB Name: %s, Path: %s. ex: %s",

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

@@ -121,9 +121,12 @@ public final class ContainerCache extends LRUMap {
    * Returns a DB handle if available, create the handler otherwise.
    *
    * @param containerID - ID of the container.
+   * @param containerDBType - DB type of the container.
+   * @param containerDBPath - DB path of the container.
    * @return MetadataStore.
    */
-  public MetadataStore getDB(long containerID, String containerDBPath)
+  public MetadataStore getDB(long containerID, String containerDBType, String
+                             containerDBPath)
       throws IOException {
     Preconditions.checkState(containerID >= 0, "Container ID cannot be negative.");
     lock.lock();
@@ -134,6 +137,7 @@ public final class ContainerCache extends LRUMap {
         db = MetadataStoreBuilder.newBuilder()
             .setDbFile(new File(containerDBPath))
             .setCreateIfMissing(false)
+            .setDBType(containerDBType)
             .build();
         this.put(containerID, db);
       }