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 xk...@apache.org on 2018/08/31 16:10:47 UTC

[21/47] hadoop git commit: HDDS-359. RocksDB Profiles support. Contributed by Anu Engineer.

HDDS-359. RocksDB Profiles support. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c61824a1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c61824a1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c61824a1

Branch: refs/heads/HDFS-12943
Commit: c61824a18940ef37dc7201717a3115a78bf942d4
Parents: df21e1b
Author: Márton Elek <el...@apache.org>
Authored: Tue Aug 28 19:22:30 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Tue Aug 28 19:33:13 2018 +0200

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/HddsConfigKeys.java  |   6 +
 .../hadoop/utils/db/DBConfigFromFile.java       | 134 +++++++++++++
 .../org/apache/hadoop/utils/db/DBProfile.java   | 120 +++++++++++
 .../apache/hadoop/utils/db/DBStoreBuilder.java  | 201 +++++++++++++++++++
 .../org/apache/hadoop/utils/db/RDBStore.java    |  32 +--
 .../org/apache/hadoop/utils/db/TableConfig.java |  93 +++++++++
 .../common/src/main/resources/ozone-default.xml |  10 +
 .../hadoop/utils/db/TestDBConfigFromFile.java   | 116 +++++++++++
 .../hadoop/utils/db/TestDBStoreBuilder.java     | 174 ++++++++++++++++
 .../apache/hadoop/utils/db/TestRDBStore.java    |  17 +-
 .../hadoop/utils/db/TestRDBTableStore.java      |  11 +-
 .../common/src/test/resources/test.db.ini       | 145 +++++++++++++
 hadoop-hdds/pom.xml                             |   1 +
 13 files changed, 1040 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index d25af80..8272ed7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdds;
 
+import org.apache.hadoop.utils.db.DBProfile;
+
 /**
  * This class contains constants for configuration keys and default values
  * used in hdds.
@@ -58,4 +60,8 @@ public final class HddsConfigKeys {
   public static final String HDDS_DATANODE_VOLUME_CHOOSING_POLICY =
       "hdds.datanode.volume.choosing.policy";
 
+  // DB Profiles used by ROCKDB instances.
+  public static final String HDDS_DB_PROFILE = "hdds.db.profile";
+  public static final DBProfile HDDS_DEFAULT_DB_PROFILE = DBProfile.SSD;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBConfigFromFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBConfigFromFile.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBConfigFromFile.java
new file mode 100644
index 0000000..753a460
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBConfigFromFile.java
@@ -0,0 +1,134 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import org.eclipse.jetty.util.StringUtil;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Env;
+import org.rocksdb.OptionsUtil;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+
+/**
+ * A Class that controls the standard config options of RocksDB.
+ * <p>
+ * Important : Some of the functions in this file are magic functions designed
+ * for the use of OZONE developers only. Due to that this information is
+ * documented in this files only and is *not* intended for end user consumption.
+ * Please do not use this information to tune your production environments.
+ * Please remember the SpiderMan principal; with great power comes great
+ * responsibility.
+ */
+public final class DBConfigFromFile {
+  public static final String CONFIG_DIR = "HADOOP_CONF_DIR";
+
+  private DBConfigFromFile() {
+  }
+
+  public static File getConfigLocation() throws IOException {
+    String path = System.getenv(CONFIG_DIR);
+
+    // Make testing easy.
+    // If there is No Env. defined, let us try to read the JVM property
+    if (StringUtil.isBlank(path)) {
+      path = System.getProperty(CONFIG_DIR);
+    }
+
+    if (StringUtil.isBlank(path)) {
+      throw new IOException("Unable to find the configuration directory. "
+          + "Please make sure that HADOOP_CONF_DIR is setup correctly ");
+    }
+    return new File(path);
+
+  }
+
+  /**
+   * This class establishes a magic pattern where we look for DBFile.ini as the
+   * options for RocksDB.
+   *
+   * @param dbFileName - The DBFile Name. For example, OzoneManager.db
+   * @return Name of the DB File options
+   */
+  public static String getOptionsFileNameFromDB(String dbFileName) {
+    Preconditions.checkNotNull(dbFileName);
+    return dbFileName + ".ini";
+  }
+
+  /**
+   * One of the Magic functions designed for the use of Ozone Developers *ONLY*.
+   * This function takes the name of DB file and looks up the a .ini file that
+   * follows the ROCKSDB config format and uses that file for DBOptions and
+   * Column family Options. The Format for this file is specified by RockDB.
+   * <p>
+   * Here is a sample config from RocksDB sample Repo.
+   * <p>
+   * https://github.com/facebook/rocksdb/blob/master/examples
+   * /rocksdb_option_file_example.ini
+   * <p>
+   * We look for a specific pattern, say OzoneManager.db will have its configs
+   * specified in OzoneManager.db.ini. This option is used only by the
+   * performance testing group to allow tuning of all parameters freely.
+   * <p>
+   * For the end users we offer a set of Predefined options that is easy to use
+   * and the user does not need to become an expert in RockDB config.
+   * <p>
+   * This code assumes the .ini file is placed in the same directory as normal
+   * config files. That is in $HADOOP_DIR/etc/hadoop. For example, if we want to
+   * control OzoneManager.db configs from a file, we need to create a file
+   * called OzoneManager.db.ini and place that file in $HADOOP_DIR/etc/hadoop.
+   *
+   * @param dbFileName - The DB File Name, for example, OzoneManager.db.
+   * @param cfDescs - ColumnFamily Handles.
+   * @return DBOptions, Options to be used for opening/creating the DB.
+   * @throws IOException
+   */
+  public static DBOptions readFromFile(String dbFileName,
+      List<ColumnFamilyDescriptor> cfDescs) throws IOException {
+    Preconditions.checkNotNull(dbFileName);
+    Preconditions.checkNotNull(cfDescs);
+    Preconditions.checkArgument(cfDescs.size() > 0);
+
+    //TODO: Add Documentation on how to support RocksDB Mem Env.
+    Env env = Env.getDefault();
+    DBOptions options = null;
+    Path optionsFile = Paths.get(getConfigLocation().toString(),
+        getOptionsFileNameFromDB(dbFileName));
+
+    if (optionsFile.toFile().exists()) {
+      options = new DBOptions();
+      try {
+        OptionsUtil.loadOptionsFromFile(optionsFile.toString(),
+            env, options, cfDescs, true);
+
+      } catch (RocksDBException rdEx) {
+        RDBTable.toIOException("Unable to find/open Options file.", rdEx);
+      }
+    }
+    return options;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBProfile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBProfile.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBProfile.java
new file mode 100644
index 0000000..4d3d6bc
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBProfile.java
@@ -0,0 +1,120 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.StorageUnit;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.BloomFilter;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.CompactionStyle;
+import org.rocksdb.DBOptions;
+
+import java.math.BigDecimal;
+
+/**
+ * User visible configs based RocksDB tuning page. Documentation for Options.
+ * <p>
+ * https://github.com/facebook/rocksdb/blob/master/include/rocksdb/options.h
+ * <p>
+ * Most tuning parameters are based on this URL.
+ * <p>
+ * https://github.com/facebook/rocksdb/wiki/Setup-Options-and-Basic-Tuning
+ */
+public enum DBProfile {
+  //TODO : Add more profiles like TEST etc.
+  SSD {
+    @Override
+    public String toString() {
+      return "DBProfile.SSD";
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions() {
+
+      // Set BlockCacheSize to 256 MB. This should not be an issue for HADOOP.
+      final long blockCacheSize = toLong(StorageUnit.MB.toBytes(256.00));
+
+      // Set the Default block size to 16KB
+      final long blockSize = toLong(StorageUnit.KB.toBytes(16));
+
+      // Write Buffer Size -- set to 128 MB
+      final long writeBufferSize = toLong(StorageUnit.MB.toBytes(128));
+
+      return new ColumnFamilyOptions()
+          .setLevelCompactionDynamicLevelBytes(true)
+          .setWriteBufferSize(writeBufferSize)
+          .setTableFormatConfig(
+              new BlockBasedTableConfig()
+                  .setBlockCacheSize(blockCacheSize)
+                  .setBlockSize(blockSize)
+                  .setCacheIndexAndFilterBlocks(true)
+                  .setPinL0FilterAndIndexBlocksInCache(true)
+                  .setFilter(new BloomFilter()));
+    }
+
+    @Override
+    public DBOptions getDBOptions() {
+      final int maxBackgroundCompactions = 4;
+      final int maxBackgroundFlushes = 2;
+      final long bytesPerSync = toLong(StorageUnit.MB.toBytes(1.00));
+      final boolean createIfMissing = true;
+      final boolean createMissingColumnFamilies = true;
+      return new DBOptions()
+          .setIncreaseParallelism(Runtime.getRuntime().availableProcessors())
+          .setMaxBackgroundCompactions(maxBackgroundCompactions)
+          .setMaxBackgroundFlushes(maxBackgroundFlushes)
+          .setBytesPerSync(bytesPerSync)
+          .setCreateIfMissing(createIfMissing)
+          .setCreateMissingColumnFamilies(createMissingColumnFamilies);
+    }
+
+
+  },
+  DISK {
+    @Override
+    public String toString() {
+      return "DBProfile.DISK";
+    }
+
+    @Override
+    public DBOptions getDBOptions() {
+      final long readAheadSize = toLong(StorageUnit.MB.toBytes(4.00));
+      return SSD.getDBOptions().setCompactionReadaheadSize(readAheadSize);
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions() {
+      ColumnFamilyOptions columnFamilyOptions = SSD.getColumnFamilyOptions();
+      columnFamilyOptions.setCompactionStyle(CompactionStyle.LEVEL);
+      return columnFamilyOptions;
+    }
+
+
+  };
+
+  private static long toLong(double value) {
+    BigDecimal temp = new BigDecimal(value);
+    return temp.longValue();
+  }
+
+  public abstract DBOptions getDBOptions();
+
+  public abstract ColumnFamilyOptions getColumnFamilyOptions();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStoreBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStoreBuilder.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStoreBuilder.java
new file mode 100644
index 0000000..5ba9b9c
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStoreBuilder.java
@@ -0,0 +1,201 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.eclipse.jetty.util.StringUtil;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DB_PROFILE;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_DB_PROFILE;
+
+/**
+ * DBStore Builder.
+ */
+public final class DBStoreBuilder {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DBStoreBuilder.class);
+  private Set<TableConfig> tables;
+  private DBProfile dbProfile;
+  private DBOptions rocksDBOption;
+  private String dbname;
+  private Path dbPath;
+  private List<String> tableNames;
+  private Configuration configuration;
+
+  private DBStoreBuilder(Configuration configuration) {
+    tables = new HashSet<>();
+    tableNames = new LinkedList<>();
+    this.configuration = configuration;
+  }
+
+  public static DBStoreBuilder newBuilder(Configuration configuration) {
+    return new DBStoreBuilder(configuration);
+  }
+
+  public DBStoreBuilder setProfile(DBProfile profile) {
+    dbProfile = profile;
+    return this;
+  }
+
+  public DBStoreBuilder setName(String name) {
+    dbname = name;
+    return this;
+  }
+
+  public DBStoreBuilder addTable(String tableName) {
+    tableNames.add(tableName);
+    return this;
+  }
+
+  public DBStoreBuilder addTable(String tableName, ColumnFamilyOptions option)
+      throws IOException {
+    TableConfig tableConfig = new TableConfig(tableName, option);
+    if (!tables.add(tableConfig)) {
+      String message = "Unable to add the table: " + tableName +
+          ".  Please check if this table name is already in use.";
+      LOG.error(message);
+      throw new IOException(message);
+    }
+    LOG.info("using custom profile for table: {}", tableName);
+    return this;
+  }
+
+  public DBStoreBuilder setDBOption(DBOptions option) {
+    rocksDBOption = option;
+    return this;
+  }
+
+  public DBStoreBuilder setPath(Path path) {
+    Preconditions.checkNotNull(path);
+    dbPath = path;
+    return this;
+  }
+
+  /**
+   * Builds a DBStore instance and returns that.
+   *
+   * @return DBStore
+   */
+  public DBStore build() throws IOException {
+    if(StringUtil.isBlank(dbname) || (dbPath == null)) {
+      LOG.error("Required Parameter missing.");
+      throw new IOException("Required parameter is missing. Please make sure "
+          + "sure Path and DB name is provided.");
+    }
+    processDBProfile();
+    processTables();
+    DBOptions options = getDbProfile();
+    File dbFile = getDBFile();
+    if (!dbFile.getParentFile().exists()) {
+      throw new IOException("The DB destination directory should exist.");
+    }
+    return new RDBStore(dbFile, options, tables);
+  }
+
+  /**
+   * if the DBProfile is not set, we will default to using default from the
+   * config file.
+   */
+  private void processDBProfile() {
+    if (dbProfile == null) {
+      dbProfile = this.configuration.getEnum(HDDS_DB_PROFILE,
+          HDDS_DEFAULT_DB_PROFILE);
+    }
+  }
+
+  private void processTables() throws IOException {
+    if (tableNames.size() > 0) {
+      for (String name : tableNames) {
+        addTable(name, dbProfile.getColumnFamilyOptions());
+        LOG.info("Using default column profile:{} for Table:{}",
+            dbProfile.toString(), name);
+      }
+    }
+    addTable(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+        dbProfile.getColumnFamilyOptions());
+    LOG.info("Using default column profile:{} for Table:{}",
+        dbProfile.toString(),
+        DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY));
+  }
+
+  private DBOptions getDbProfile() {
+    if (rocksDBOption != null) {
+      return rocksDBOption;
+    }
+    DBOptions option = null;
+    if (StringUtil.isNotBlank(dbname)) {
+      List<ColumnFamilyDescriptor> columnFamilyDescriptors = new LinkedList<>();
+
+      for (TableConfig tc : tables) {
+        columnFamilyDescriptors.add(tc.getDescriptor());
+      }
+
+      if (columnFamilyDescriptors.size() > 0) {
+        try {
+          option = DBConfigFromFile.readFromFile(dbname,
+              columnFamilyDescriptors);
+          if(option != null) {
+            LOG.info("Using Configs from {}.ini file", dbname);
+          }
+        } catch (IOException ex) {
+          LOG.info("Unable to read ROCKDB config", ex);
+        }
+      }
+    }
+
+    if (option == null) {
+      LOG.info("Using default options. {}", dbProfile.toString());
+      return dbProfile.getDBOptions();
+    }
+    return option;
+  }
+
+  private File getDBFile() throws IOException {
+    if (dbPath == null) {
+      LOG.error("DB path is required.");
+      throw new IOException("A Path to for DB file is needed.");
+    }
+
+    if (StringUtil.isBlank(dbname)) {
+      LOG.error("DBName is a required.");
+      throw new IOException("A valid DB name is required.");
+    }
+    return Paths.get(dbPath.toString(), dbname).toFile();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
index c719d31..85508d5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.utils.RocksDBStoreMBean;
 import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
 import org.rocksdb.ColumnFamilyDescriptor;
 import org.rocksdb.ColumnFamilyHandle;
-import org.rocksdb.ColumnFamilyOptions;
+
 import org.rocksdb.DBOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
@@ -38,12 +38,12 @@ import org.slf4j.LoggerFactory;
 import javax.management.ObjectName;
 import java.io.File;
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Hashtable;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * RocksDB Store that supports creating Tables in DB.
@@ -58,7 +58,7 @@ public class RDBStore implements DBStore {
   private final Hashtable<String, ColumnFamilyHandle> handleTable;
   private ObjectName statMBeanName;
 
-  public RDBStore(File dbFile, DBOptions options, List<String> families)
+  public RDBStore(File dbFile, DBOptions options, Set<TableConfig> families)
       throws IOException {
     Preconditions.checkNotNull(dbFile, "DB file location cannot be null");
     Preconditions.checkNotNull(families);
@@ -69,10 +69,8 @@ public class RDBStore implements DBStore {
         new ArrayList<>();
     final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
 
-    for (String family : families) {
-      columnFamilyDescriptors.add(
-          new ColumnFamilyDescriptor(family.getBytes(StandardCharsets.UTF_8),
-              new ColumnFamilyOptions()));
+    for (TableConfig family : families) {
+      columnFamilyDescriptors.add(family.getDescriptor());
     }
 
     dbOptions = options;
@@ -141,19 +139,23 @@ public class RDBStore implements DBStore {
     for (final ColumnFamilyHandle handle : handleTable.values()) {
       handle.close();
     }
-    if (dbOptions != null) {
-      dbOptions.close();
-    }
-    if (writeOptions != null) {
-      writeOptions.close();
-    }
+
     if (statMBeanName != null) {
       MBeans.unregister(statMBeanName);
       statMBeanName = null;
     }
+
     if (db != null) {
       db.close();
     }
+
+    if (dbOptions != null) {
+      dbOptions.close();
+    }
+
+    if (writeOptions != null) {
+      writeOptions.close();
+    }
   }
 
   @Override
@@ -221,7 +223,7 @@ public class RDBStore implements DBStore {
   @Override
   public long getEstimatedKeyCount() throws IOException {
     try {
-      return Long.parseLong(db.getProperty("rocksdb.estimate-num-keys"));
+      return db.getLongProperty("rocksdb.estimate-num-keys");
     } catch (RocksDBException e) {
       throw toIOException("Unable to get the estimated count.", e);
     }
@@ -244,7 +246,7 @@ public class RDBStore implements DBStore {
   @Override
   public ArrayList<Table> listTables() throws IOException {
     ArrayList<Table> returnList = new ArrayList<>();
-    for (ColumnFamilyHandle handle: handleTable.values())  {
+    for (ColumnFamilyHandle handle : handleTable.values()) {
       returnList.add(new RDBTable(db, handle, writeOptions));
     }
     return returnList;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableConfig.java
new file mode 100644
index 0000000..897028a
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableConfig.java
@@ -0,0 +1,93 @@
+/*
+ * 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;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+
+/**
+ * Class that maintains Table Configuration.
+ */
+public class TableConfig {
+  private final String name;
+  private final ColumnFamilyOptions columnFamilyOptions;
+
+
+  /**
+   * Constructs a Table Config.
+   * @param name - Name of the Table.
+   * @param columnFamilyOptions - Column Family options.
+   */
+  public TableConfig(String name, ColumnFamilyOptions columnFamilyOptions) {
+    this.name = name;
+    this.columnFamilyOptions = columnFamilyOptions;
+  }
+
+  /**
+   * Returns the Name for this Table.
+   * @return - Name String
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Returns a ColumnFamilyDescriptor for this table.
+   * @return ColumnFamilyDescriptor
+   */
+  public ColumnFamilyDescriptor getDescriptor() {
+    return  new ColumnFamilyDescriptor(DFSUtil.string2Bytes(name),
+        columnFamilyOptions);
+  }
+
+  /**
+   * Returns Column family options for this Table.
+   * @return  ColumnFamilyOptions used for the Table.
+   */
+  public ColumnFamilyOptions getColumnFamilyOptions() {
+    return columnFamilyOptions;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    TableConfig that = (TableConfig) o;
+    return new EqualsBuilder()
+        .append(getName(), that.getName())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getName())
+        .toHashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index c3e3095..6d2ee09 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1098,4 +1098,14 @@
       datanode unless the datanode confirms the completion.
     </description>
   </property>
+
+  <property>
+    <name>ozone.db.profile</name>
+    <value>DBProfile.SSD</value>
+    <tag>OZONE, OM, PERFORMANCE, REQUIRED</tag>
+    <description>This property allows user to pick a configuration
+    that tunes the RocksDB settings for the hardware it is running
+    on. Right now, we have SSD and DISK as profile options.</description>
+  </property>
+
 </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBConfigFromFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBConfigFromFile.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBConfigFromFile.java
new file mode 100644
index 0000000..b20ca70
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBConfigFromFile.java
@@ -0,0 +1,116 @@
+/*
+ * 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;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.hadoop.utils.db.DBConfigFromFile.getOptionsFileNameFromDB;
+
+/**
+ * DBConf tests.
+ */
+public class TestDBConfigFromFile {
+  private final static String DB_FILE = "test.db";
+  private final static String INI_FILE = getOptionsFileNameFromDB(DB_FILE);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(DBConfigFromFile.CONFIG_DIR,
+        folder.newFolder().toString());
+    ClassLoader classLoader = getClass().getClassLoader();
+    File testData = new File(classLoader.getResource(INI_FILE).getFile());
+    File dest = Paths.get(
+        System.getProperty(DBConfigFromFile.CONFIG_DIR), INI_FILE).toFile();
+    FileUtils.copyFile(testData, dest);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void readFromFile() throws IOException {
+    final List<String> families =
+        Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+            "First", "Second", "Third",
+            "Fourth", "Fifth",
+            "Sixth");
+    final List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        new ArrayList<>();
+    for (String family : families) {
+      columnFamilyDescriptors.add(
+          new ColumnFamilyDescriptor(family.getBytes(StandardCharsets.UTF_8),
+              new ColumnFamilyOptions()));
+    }
+
+    final DBOptions options = DBConfigFromFile.readFromFile(DB_FILE,
+        columnFamilyDescriptors);
+
+    // Some Random Values Defined in the test.db.ini, we verify that we are
+    // able to get values that are defined in the test.db.ini.
+    Assert.assertNotNull(options);
+    Assert.assertEquals(551615L, options.maxManifestFileSize());
+    Assert.assertEquals(1000L, options.keepLogFileNum());
+    Assert.assertEquals(1048576, options.writableFileMaxBufferSize());
+  }
+
+  @Test
+  public void readFromFileInvalidConfig() throws IOException {
+    final List<String> families =
+        Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+            "First", "Second", "Third",
+            "Fourth", "Fifth",
+            "Sixth");
+    final List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        new ArrayList<>();
+    for (String family : families) {
+      columnFamilyDescriptors.add(
+          new ColumnFamilyDescriptor(family.getBytes(StandardCharsets.UTF_8),
+              new ColumnFamilyOptions()));
+    }
+
+    final DBOptions options = DBConfigFromFile.readFromFile("badfile.db.ini",
+        columnFamilyDescriptors);
+
+    // This has to return a Null, since we have config defined for badfile.db
+    Assert.assertNull(options);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java
new file mode 100644
index 0000000..3e1f364
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java
@@ -0,0 +1,174 @@
+/*
+ * 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;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Tests RDBStore creation.
+ */
+public class TestDBStoreBuilder {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(DBConfigFromFile.CONFIG_DIR,
+        folder.newFolder().toString());
+  }
+
+  @Test
+  public void builderWithoutAnyParams() throws IOException {
+    Configuration conf = new Configuration();
+    thrown.expect(IOException.class);
+    DBStoreBuilder.newBuilder(conf).build();
+  }
+
+  @Test
+  public void builderWithOneParamV1() throws IOException {
+    Configuration conf = new Configuration();
+    thrown.expect(IOException.class);
+    DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .build();
+  }
+
+  @Test
+  public void builderWithOneParamV2() throws IOException {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    thrown.expect(IOException.class);
+    DBStoreBuilder.newBuilder(conf)
+        .setPath(newFolder.toPath())
+        .build();
+  }
+
+  @Test
+  public void builderWithOpenClose() throws Exception {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    DBStore dbStore = DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .setPath(newFolder.toPath())
+        .build();
+    // Nothing to do just open and Close.
+    dbStore.close();
+  }
+
+  @Test
+  public void builderWithDoubleTableName() throws Exception {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    thrown.expect(IOException.class);
+    DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .setPath(newFolder.toPath())
+        .addTable("FIRST")
+        .addTable("FIRST")
+        .build();
+    // Nothing to do , This will throw so we do not have to close.
+
+  }
+
+  @Test
+  public void builderWithDataWrites() throws Exception {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    try (DBStore dbStore = DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .setPath(newFolder.toPath())
+        .addTable("First")
+        .addTable("Second")
+        .build()) {
+      try (Table firstTable = dbStore.getTable("First")) {
+        byte[] key =
+            RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
+        byte[] value =
+            RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
+        firstTable.put(key, value);
+        byte[] temp = firstTable.get(key);
+        Arrays.equals(value, temp);
+      }
+
+      try (Table secondTable = dbStore.getTable("Second")) {
+        Assert.assertTrue(secondTable.isEmpty());
+      }
+    }
+  }
+
+  @Test
+  public void builderWithDiskProfileWrites() throws Exception {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    try (DBStore dbStore = DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .setPath(newFolder.toPath())
+        .addTable("First")
+        .addTable("Second")
+        .setProfile(DBProfile.DISK)
+        .build()) {
+      try (Table firstTable = dbStore.getTable("First")) {
+        byte[] key =
+            RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
+        byte[] value =
+            RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
+        firstTable.put(key, value);
+        byte[] temp = firstTable.get(key);
+        Arrays.equals(value, temp);
+      }
+
+      try (Table secondTable = dbStore.getTable("Second")) {
+        Assert.assertTrue(secondTable.isEmpty());
+      }
+    }
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
index 94a650b..0b673d3 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
@@ -28,6 +28,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.Statistics;
@@ -39,8 +40,10 @@ import java.lang.management.ManagementFactory;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * RDBStore Tests.
@@ -57,6 +60,7 @@ public class TestRDBStore {
   public ExpectedException thrown = ExpectedException.none();
   private RDBStore rdbStore = null;
   private DBOptions options = null;
+  private Set<TableConfig> configSet;
 
   @Before
   public void setUp() throws Exception {
@@ -67,7 +71,12 @@ public class TestRDBStore {
     Statistics statistics = new Statistics();
     statistics.setStatsLevel(StatsLevel.ALL);
     options = options.setStatistics(statistics);
-    rdbStore = new RDBStore(folder.newFolder(), options, families);
+    configSet = new HashSet<>();
+    for(String name : families) {
+      TableConfig newConfig = new TableConfig(name, new ColumnFamilyOptions());
+      configSet.add(newConfig);
+    }
+    rdbStore = new RDBStore(folder.newFolder(), options, configSet);
   }
 
   @After
@@ -80,7 +89,7 @@ public class TestRDBStore {
   @Test
   public void compactDB() throws Exception {
     try (RDBStore newStore =
-             new RDBStore(folder.newFolder(), options, families)) {
+             new RDBStore(folder.newFolder(), options, configSet)) {
       Assert.assertNotNull("DB Store cannot be null", newStore);
       try (Table firstTable = newStore.getTable(families.get(1))) {
         Assert.assertNotNull("Table cannot be null", firstTable);
@@ -101,7 +110,7 @@ public class TestRDBStore {
   @Test
   public void close() throws Exception {
     RDBStore newStore =
-        new RDBStore(folder.newFolder(), options, families);
+        new RDBStore(folder.newFolder(), options, configSet);
     Assert.assertNotNull("DBStore cannot be null", newStore);
     // This test does not assert anything if there is any error this test
     // will throw and fail.
@@ -157,7 +166,7 @@ public class TestRDBStore {
   @Test
   public void getEstimatedKeyCount() throws Exception {
     try (RDBStore newStore =
-             new RDBStore(folder.newFolder(), options, families)) {
+             new RDBStore(folder.newFolder(), options, configSet)) {
       Assert.assertNotNull("DB Store cannot be null", newStore);
       // Write 100 keys to the first table.
       try (Table firstTable = newStore.getTable(families.get(1))) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
index 677a1f9..cd25548 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
@@ -27,6 +27,7 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.Statistics;
@@ -35,8 +36,10 @@ import org.rocksdb.WriteBatch;
 
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Set;
 
 /**
  * Tests for RocksDBTable Store.
@@ -62,7 +65,13 @@ public class TestRDBTableStore {
     Statistics statistics = new Statistics();
     statistics.setStatsLevel(StatsLevel.ALL);
     options = options.setStatistics(statistics);
-    rdbStore = new RDBStore(folder.newFolder(), options, families);
+
+    Set<TableConfig> configSet = new HashSet<>();
+    for(String name : families) {
+      TableConfig newConfig = new TableConfig(name, new ColumnFamilyOptions());
+      configSet.add(newConfig);
+    }
+    rdbStore = new RDBStore(folder.newFolder(), options, configSet);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/resources/test.db.ini
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/resources/test.db.ini b/hadoop-hdds/common/src/test/resources/test.db.ini
new file mode 100644
index 0000000..6666cd2
--- /dev/null
+++ b/hadoop-hdds/common/src/test/resources/test.db.ini
@@ -0,0 +1,145 @@
+# This is a RocksDB option file.
+#
+# A typical RocksDB options file has four sections, which are
+# Version section, DBOptions section, at least one CFOptions
+# section, and one TableOptions section for each column family.
+# The RocksDB options file in general follows the basic INI
+# file format with the following extensions / modifications:
+#
+#  * Escaped characters
+#    We escaped the following characters:
+#     - \n -- line feed - new line
+#     - \r -- carriage return
+#     - \\ -- backslash \
+#     - \: -- colon symbol :
+#     - \# -- hash tag #
+#  * Comments
+#    We support # style comments.  Comments can appear at the ending
+#    part of a line.
+#  * Statements
+#    A statement is of the form option_name = value.
+#    Each statement contains a '=', where extra white-spaces
+#    are supported. However, we don't support multi-lined statement.
+#    Furthermore, each line can only contain at most one statement.
+#  * Sections
+#    Sections are of the form [SecitonTitle "SectionArgument"],
+#    where section argument is optional.
+#  * List
+#    We use colon-separated string to represent a list.
+#    For instance, n1:n2:n3:n4 is a list containing four values.
+#
+# Below is an example of a RocksDB options file:
+
+
+#----------------------IMPORTANT------------------------------------#
+### FAKE VALUES FOR TESTING ONLY ### DO NOT USE THESE FOR PRODUCTION.
+#----------------------IMPORTANT------------------------------------#
+[DBOptions]
+  stats_dump_period_sec=600
+  max_manifest_file_size=551615
+  bytes_per_sync=8388608
+  delayed_write_rate=2097152
+  WAL_ttl_seconds=0
+  WAL_size_limit_MB=0
+  max_subcompactions=1
+  wal_dir=
+  wal_bytes_per_sync=0
+  db_write_buffer_size=0
+  keep_log_file_num=1000
+  table_cache_numshardbits=4
+  max_file_opening_threads=1
+  writable_file_max_buffer_size=1048576
+  random_access_max_buffer_size=1048576
+  use_fsync=false
+  max_total_wal_size=0
+  max_open_files=-1
+  skip_stats_update_on_db_open=false
+  max_background_compactions=16
+  manifest_preallocation_size=4194304
+  max_background_flushes=7
+  is_fd_close_on_exec=true
+  max_log_file_size=0
+  advise_random_on_open=true
+  create_missing_column_families=false
+  paranoid_checks=true
+  delete_obsolete_files_period_micros=21600000000
+  log_file_time_to_roll=0
+  compaction_readahead_size=0
+  create_if_missing=false
+  use_adaptive_mutex=false
+  enable_thread_tracking=false
+  allow_fallocate=true
+  error_if_exists=false
+  recycle_log_file_num=0
+  skip_log_error_on_recovery=false
+  db_log_dir=
+  new_table_reader_for_compaction_inputs=true
+  allow_mmap_reads=false
+  allow_mmap_writes=false
+  use_direct_reads=false
+  use_direct_writes=false
+
+
+[CFOptions "default"]
+  compaction_style=kCompactionStyleLevel
+  compaction_filter=nullptr
+  num_levels=6
+  table_factory=BlockBasedTable
+  comparator=leveldb.BytewiseComparator
+  max_sequential_skip_in_iterations=8
+  soft_rate_limit=0.000000
+  max_bytes_for_level_base=1073741824
+  memtable_prefix_bloom_probes=6
+  memtable_prefix_bloom_bits=0
+  memtable_prefix_bloom_huge_page_tlb_size=0
+  max_successive_merges=0
+  arena_block_size=16777216
+  min_write_buffer_number_to_merge=1
+  target_file_size_multiplier=1
+  source_compaction_factor=1
+  max_bytes_for_level_multiplier=8
+  max_bytes_for_level_multiplier_additional=2:3:5
+  compaction_filter_factory=nullptr
+  max_write_buffer_number=8
+  level0_stop_writes_trigger=20
+  compression=kSnappyCompression
+  level0_file_num_compaction_trigger=4
+  purge_redundant_kvs_while_flush=true
+  max_write_buffer_number_to_maintain=0
+  memtable_factory=SkipListFactory
+  max_grandparent_overlap_factor=8
+  expanded_compaction_factor=25
+  hard_pending_compaction_bytes_limit=137438953472
+  inplace_update_num_locks=10000
+  level_compaction_dynamic_level_bytes=true
+  level0_slowdown_writes_trigger=12
+  filter_deletes=false
+  verify_checksums_in_compaction=true
+  min_partial_merge_operands=2
+  paranoid_file_checks=false
+  target_file_size_base=134217728
+  optimize_filters_for_hits=false
+  merge_operator=PutOperator
+  compression_per_level=kNoCompression:kNoCompression:kNoCompression:kSnappyCompression:kSnappyCompression:kSnappyCompression
+  compaction_measure_io_stats=false
+  prefix_extractor=nullptr
+  bloom_locality=0
+  write_buffer_size=134217728
+  disable_auto_compactions=false
+  inplace_update_support=false
+
+[TableOptions/BlockBasedTable "default"]
+  format_version=2
+  whole_key_filtering=true
+  no_block_cache=false
+  checksum=kCRC32c
+  filter_policy=rocksdb.BuiltinBloomFilter
+  block_size_deviation=10
+  block_size=8192
+  block_restart_interval=16
+  cache_index_and_filter_blocks=false
+  pin_l0_filter_and_index_blocks_in_cache=false
+  pin_top_level_index_and_filter=false
+  index_type=kBinarySearch
+  hash_index_allow_collision=true
+  flush_block_policy_factory=FlushBlockBySizePolicyFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index f655c2f..428fd11 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -105,6 +105,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/test/resources/incorrect.container</exclude>
             <exclude>src/test/resources/additionalfields.container</exclude>
             <exclude>src/test/resources/incorrect.checksum.container</exclude>
+            <exclude>src/test/resources/test.db.ini</exclude>
           </excludes>
         </configuration>
       </plugin>


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