You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/05/04 10:49:13 UTC

[GitHub] [ozone] lokeshj1703 opened a new pull request #2214: HDDS-5184. Use separate DB profile for Datanodes.

lokeshj1703 opened a new pull request #2214:
URL: https://github.com/apache/ozone/pull/2214


   ## What changes were proposed in this pull request?
   
   Currently datanodes shares a common DB profile with other components like SCM and OM etc. The jira aims to create a different profile for datanodes which also makes sure that column family options are shared by containers.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5184
   
   ## How was this patch tested?
   
   Existing UT, PR adds a UT to test columnfamilyoptions are shared.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] lokeshj1703 commented on a change in pull request #2214: HDDS-5184. Use separate DB profile for Datanodes.

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on a change in pull request #2214:
URL: https://github.com/apache/ozone/pull/2214#discussion_r671112127



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/db/DatanodeDBProfile.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.ozone.container.common.utils.db;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.db.DBProfile;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.DBOptions;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.LRUCache;
+import org.rocksdb.util.SizeUnit;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT;
+
+/**
+ * The class manages DBProfiles for Datanodes. Since ColumnFamilyOptions need to
+ * be shared across containers the options are maintained in the profile itself.
+ */
+public abstract class DatanodeDBProfile {
+
+  /**
+   * Returns DBOptions to be used for rocksDB in datanodes.
+   */
+  public abstract DBOptions getDBOptions();
+
+  /**
+   * Returns ColumnFamilyOptions to be used for rocksDB column families in
+   * datanodes.
+   */
+  public abstract ColumnFamilyOptions getColumnFamilyOptions(
+      ConfigurationSource config);
+
+  /**
+   * Returns DatanodeDBProfile for corresponding storage type.
+   */
+  public static DatanodeDBProfile getProfile(DBProfile dbProfile) {
+    switch (dbProfile) {
+    case SSD:
+      return new SSD();
+    case DISK:
+      return new Disk();
+    default:
+      throw new IllegalArgumentException(
+          "DatanodeDBProfile does not exist for " + dbProfile);
+    }
+  }
+
+
+
+  public static class SSD extends DatanodeDBProfile {
+    private static final StorageBasedProfile SSD_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.SSD);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return SSD_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return SSD_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  public static class Disk extends DatanodeDBProfile {
+    private static final StorageBasedProfile DISK_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.DISK);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return DISK_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return DISK_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  private static final class StorageBasedProfile {
+    private final AtomicReference<ColumnFamilyOptions> cfOptions =
+        new AtomicReference<>();
+    private final DBProfile baseProfile;
+
+    private StorageBasedProfile(DBProfile profile) {
+      baseProfile = profile;
+    }
+
+    private DBOptions getDBOptions() {
+      return baseProfile.getDBOptions();
+    }
+
+    private ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      cfOptions.updateAndGet(op -> op != null ? op :
+          baseProfile.getColumnFamilyOptions()
+              .setTableFormatConfig(getBlockBasedTableConfig(config)));
+      return cfOptions.get();
+    }
+
+    private BlockBasedTableConfig getBlockBasedTableConfig(
+        ConfigurationSource config) {
+      BlockBasedTableConfig blockBasedTableConfig =
+          baseProfile.getBlockBasedTableConfig();
+      if (config == null) {
+        return blockBasedTableConfig;
+      }
+
+      long cacheSize = (long) config
+          .getStorageSize(HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE,
+              HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT,
+              StorageUnit.BYTES);
+      blockBasedTableConfig
+          .setBlockCache(new LRUCache(cacheSize * SizeUnit.MB));

Review comment:
       Added a UT in new commit.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] lokeshj1703 commented on a change in pull request #2214: HDDS-5184. Use separate DB profile for Datanodes.

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on a change in pull request #2214:
URL: https://github.com/apache/ozone/pull/2214#discussion_r675330050



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/db/DatanodeDBProfile.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.ozone.container.common.utils.db;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.db.DBProfile;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.DBOptions;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.LRUCache;
+import org.rocksdb.util.SizeUnit;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT;
+
+/**
+ * The class manages DBProfiles for Datanodes. Since ColumnFamilyOptions need to
+ * be shared across containers the options are maintained in the profile itself.
+ */
+public abstract class DatanodeDBProfile {
+
+  /**
+   * Returns DBOptions to be used for rocksDB in datanodes.
+   */
+  public abstract DBOptions getDBOptions();
+
+  /**
+   * Returns ColumnFamilyOptions to be used for rocksDB column families in
+   * datanodes.
+   */
+  public abstract ColumnFamilyOptions getColumnFamilyOptions(
+      ConfigurationSource config);
+
+  /**
+   * Returns DatanodeDBProfile for corresponding storage type.
+   */
+  public static DatanodeDBProfile getProfile(DBProfile dbProfile) {
+    switch (dbProfile) {
+    case SSD:
+      return new SSD();
+    case DISK:
+      return new Disk();
+    default:
+      throw new IllegalArgumentException(
+          "DatanodeDBProfile does not exist for " + dbProfile);
+    }
+  }
+
+
+
+  public static class SSD extends DatanodeDBProfile {
+    private static final StorageBasedProfile SSD_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.SSD);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return SSD_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return SSD_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  public static class Disk extends DatanodeDBProfile {
+    private static final StorageBasedProfile DISK_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.DISK);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return DISK_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return DISK_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  private static final class StorageBasedProfile {
+    private final AtomicReference<ColumnFamilyOptions> cfOptions =
+        new AtomicReference<>();
+    private final DBProfile baseProfile;
+
+    private StorageBasedProfile(DBProfile profile) {
+      baseProfile = profile;
+    }
+
+    private DBOptions getDBOptions() {
+      return baseProfile.getDBOptions();
+    }
+
+    private ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      cfOptions.updateAndGet(op -> op != null ? op :
+          baseProfile.getColumnFamilyOptions()
+              .setTableFormatConfig(getBlockBasedTableConfig(config)));
+      return cfOptions.get();
+    }
+
+    private BlockBasedTableConfig getBlockBasedTableConfig(
+        ConfigurationSource config) {
+      BlockBasedTableConfig blockBasedTableConfig =
+          baseProfile.getBlockBasedTableConfig();
+      if (config == null) {
+        return blockBasedTableConfig;
+      }
+
+      long cacheSize = (long) config
+          .getStorageSize(HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE,
+              HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT,
+              StorageUnit.BYTES);
+      blockBasedTableConfig
+          .setBlockCache(new LRUCache(cacheSize * SizeUnit.MB));

Review comment:
       This was a bug @bshashikant. I have changed it in latest commits.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] bshashikant commented on a change in pull request #2214: HDDS-5184. Use separate DB profile for Datanodes.

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2214:
URL: https://github.com/apache/ozone/pull/2214#discussion_r674910424



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/db/DatanodeDBProfile.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.ozone.container.common.utils.db;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.db.DBProfile;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.DBOptions;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.LRUCache;
+import org.rocksdb.util.SizeUnit;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT;
+
+/**
+ * The class manages DBProfiles for Datanodes. Since ColumnFamilyOptions need to
+ * be shared across containers the options are maintained in the profile itself.
+ */
+public abstract class DatanodeDBProfile {
+
+  /**
+   * Returns DBOptions to be used for rocksDB in datanodes.
+   */
+  public abstract DBOptions getDBOptions();
+
+  /**
+   * Returns ColumnFamilyOptions to be used for rocksDB column families in
+   * datanodes.
+   */
+  public abstract ColumnFamilyOptions getColumnFamilyOptions(
+      ConfigurationSource config);
+
+  /**
+   * Returns DatanodeDBProfile for corresponding storage type.
+   */
+  public static DatanodeDBProfile getProfile(DBProfile dbProfile) {
+    switch (dbProfile) {
+    case SSD:
+      return new SSD();
+    case DISK:
+      return new Disk();
+    default:
+      throw new IllegalArgumentException(
+          "DatanodeDBProfile does not exist for " + dbProfile);
+    }
+  }
+
+
+
+  public static class SSD extends DatanodeDBProfile {
+    private static final StorageBasedProfile SSD_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.SSD);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return SSD_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return SSD_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  public static class Disk extends DatanodeDBProfile {
+    private static final StorageBasedProfile DISK_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.DISK);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return DISK_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return DISK_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  private static final class StorageBasedProfile {
+    private final AtomicReference<ColumnFamilyOptions> cfOptions =
+        new AtomicReference<>();
+    private final DBProfile baseProfile;
+
+    private StorageBasedProfile(DBProfile profile) {
+      baseProfile = profile;
+    }
+
+    private DBOptions getDBOptions() {
+      return baseProfile.getDBOptions();
+    }
+
+    private ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      cfOptions.updateAndGet(op -> op != null ? op :
+          baseProfile.getColumnFamilyOptions()
+              .setTableFormatConfig(getBlockBasedTableConfig(config)));
+      return cfOptions.get();
+    }
+
+    private BlockBasedTableConfig getBlockBasedTableConfig(
+        ConfigurationSource config) {
+      BlockBasedTableConfig blockBasedTableConfig =
+          baseProfile.getBlockBasedTableConfig();
+      if (config == null) {
+        return blockBasedTableConfig;
+      }
+
+      long cacheSize = (long) config
+          .getStorageSize(HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE,
+              HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT,
+              StorageUnit.BYTES);
+      blockBasedTableConfig
+          .setBlockCache(new LRUCache(cacheSize * SizeUnit.MB));

Review comment:
       @lokeshj1703 , why multiplying with MB is required? can you please explain?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] mukul1987 commented on a change in pull request #2214: HDDS-5184. Use separate DB profile for Datanodes.

Posted by GitBox <gi...@apache.org>.
mukul1987 commented on a change in pull request #2214:
URL: https://github.com/apache/ozone/pull/2214#discussion_r632896550



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/db/DatanodeDBProfile.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.ozone.container.common.utils.db;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.db.DBProfile;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.DBOptions;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.LRUCache;
+import org.rocksdb.util.SizeUnit;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT;
+
+/**
+ * The class manages DBProfiles for Datanodes. Since ColumnFamilyOptions need to
+ * be shared across containers the options are maintained in the profile itself.
+ */
+public abstract class DatanodeDBProfile {
+
+  /**
+   * Returns DBOptions to be used for rocksDB in datanodes.
+   */
+  public abstract DBOptions getDBOptions();
+
+  /**
+   * Returns ColumnFamilyOptions to be used for rocksDB column families in
+   * datanodes.
+   */
+  public abstract ColumnFamilyOptions getColumnFamilyOptions(
+      ConfigurationSource config);
+
+  /**
+   * Returns DatanodeDBProfile for corresponding storage type.
+   */
+  public static DatanodeDBProfile getProfile(DBProfile dbProfile) {
+    switch (dbProfile) {
+    case SSD:
+      return new SSD();
+    case DISK:
+      return new Disk();
+    default:
+      throw new IllegalArgumentException(
+          "DatanodeDBProfile does not exist for " + dbProfile);
+    }
+  }
+
+
+
+  public static class SSD extends DatanodeDBProfile {
+    private static final StorageBasedProfile SSD_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.SSD);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return SSD_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return SSD_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  public static class Disk extends DatanodeDBProfile {
+    private static final StorageBasedProfile DISK_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.DISK);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return DISK_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return DISK_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  private static final class StorageBasedProfile {
+    private final AtomicReference<ColumnFamilyOptions> cfOptions =
+        new AtomicReference<>();
+    private final DBProfile baseProfile;
+
+    private StorageBasedProfile(DBProfile profile) {
+      baseProfile = profile;
+    }
+
+    private DBOptions getDBOptions() {
+      return baseProfile.getDBOptions();
+    }
+
+    private ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      cfOptions.updateAndGet(op -> op != null ? op :
+          baseProfile.getColumnFamilyOptions()
+              .setTableFormatConfig(getBlockBasedTableConfig(config)));
+      return cfOptions.get();
+    }
+
+    private BlockBasedTableConfig getBlockBasedTableConfig(
+        ConfigurationSource config) {
+      BlockBasedTableConfig blockBasedTableConfig =
+          baseProfile.getBlockBasedTableConfig();
+      if (config == null) {
+        return blockBasedTableConfig;
+      }
+
+      long cacheSize = (long) config
+          .getStorageSize(HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE,
+              HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT,
+              StorageUnit.BYTES);
+      blockBasedTableConfig
+          .setBlockCache(new LRUCache(cacheSize * SizeUnit.MB));

Review comment:
       we should not multiply by MB here ? as cache size is already in bytes ? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] mukul1987 commented on a change in pull request #2214: HDDS-5184. Use separate DB profile for Datanodes.

Posted by GitBox <gi...@apache.org>.
mukul1987 commented on a change in pull request #2214:
URL: https://github.com/apache/ozone/pull/2214#discussion_r632896550



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/db/DatanodeDBProfile.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.ozone.container.common.utils.db;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.db.DBProfile;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.DBOptions;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.LRUCache;
+import org.rocksdb.util.SizeUnit;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT;
+
+/**
+ * The class manages DBProfiles for Datanodes. Since ColumnFamilyOptions need to
+ * be shared across containers the options are maintained in the profile itself.
+ */
+public abstract class DatanodeDBProfile {
+
+  /**
+   * Returns DBOptions to be used for rocksDB in datanodes.
+   */
+  public abstract DBOptions getDBOptions();
+
+  /**
+   * Returns ColumnFamilyOptions to be used for rocksDB column families in
+   * datanodes.
+   */
+  public abstract ColumnFamilyOptions getColumnFamilyOptions(
+      ConfigurationSource config);
+
+  /**
+   * Returns DatanodeDBProfile for corresponding storage type.
+   */
+  public static DatanodeDBProfile getProfile(DBProfile dbProfile) {
+    switch (dbProfile) {
+    case SSD:
+      return new SSD();
+    case DISK:
+      return new Disk();
+    default:
+      throw new IllegalArgumentException(
+          "DatanodeDBProfile does not exist for " + dbProfile);
+    }
+  }
+
+
+
+  public static class SSD extends DatanodeDBProfile {
+    private static final StorageBasedProfile SSD_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.SSD);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return SSD_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return SSD_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  public static class Disk extends DatanodeDBProfile {
+    private static final StorageBasedProfile DISK_STORAGE_BASED_PROFILE =
+        new StorageBasedProfile(DBProfile.DISK);
+
+    @Override
+    public DBOptions getDBOptions() {
+      return DISK_STORAGE_BASED_PROFILE.getDBOptions();
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      return DISK_STORAGE_BASED_PROFILE.getColumnFamilyOptions(config);
+    }
+  }
+
+
+
+  private static final class StorageBasedProfile {
+    private final AtomicReference<ColumnFamilyOptions> cfOptions =
+        new AtomicReference<>();
+    private final DBProfile baseProfile;
+
+    private StorageBasedProfile(DBProfile profile) {
+      baseProfile = profile;
+    }
+
+    private DBOptions getDBOptions() {
+      return baseProfile.getDBOptions();
+    }
+
+    private ColumnFamilyOptions getColumnFamilyOptions(
+        ConfigurationSource config) {
+      cfOptions.updateAndGet(op -> op != null ? op :
+          baseProfile.getColumnFamilyOptions()
+              .setTableFormatConfig(getBlockBasedTableConfig(config)));
+      return cfOptions.get();
+    }
+
+    private BlockBasedTableConfig getBlockBasedTableConfig(
+        ConfigurationSource config) {
+      BlockBasedTableConfig blockBasedTableConfig =
+          baseProfile.getBlockBasedTableConfig();
+      if (config == null) {
+        return blockBasedTableConfig;
+      }
+
+      long cacheSize = (long) config
+          .getStorageSize(HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE,
+              HDDS_DATANODE_METADATA_ROCKSDB_CACHE_SIZE_DEFAULT,
+              StorageUnit.BYTES);
+      blockBasedTableConfig
+          .setBlockCache(new LRUCache(cacheSize * SizeUnit.MB));

Review comment:
       we should not multiply by MB here ? as cache size is already in bytes ? :D 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] mukul1987 merged pull request #2214: HDDS-5184. Use separate DB profile for Datanodes.

Posted by GitBox <gi...@apache.org>.
mukul1987 merged pull request #2214:
URL: https://github.com/apache/ozone/pull/2214


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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