Permalink
Browse files

Add support for one bdb environment per store.

Heavily based on work done by Bhupesh on bdb-split branch.
  • Loading branch information...
1 parent c968873 commit 21de293b0bad522f42913fdca11cf989f3419969 @ijuma ijuma committed Jun 17, 2009
@@ -66,6 +66,7 @@
private int bdbBtreeFanout;
private long bdbCheckpointBytes;
private long bdbCheckpointMs;
+ private boolean bdbOneEnvPerStore;
private String mysqlUsername;
private String mysqlPassword;
@@ -152,6 +153,7 @@ public VoldemortConfig(Props props) {
this.bdbCheckpointBytes = props.getLong("bdb.checkpoint.interval.bytes", 20 * 1024 * 1024);
this.bdbCheckpointMs = props.getLong("bdb.checkpoint.interval.ms", 30 * Time.MS_PER_SECOND);
this.bdbSortedDuplicates = props.getBoolean("bdb.enable.sorted.duplicates", true);
+ this.bdbOneEnvPerStore = props.getBoolean("bdb.one.env.per.store", false);
this.readOnlyFileWaitTimeoutMs = props.getLong("readonly.file.wait.timeout.ms", 4000L);
this.readOnlyBackups = props.getInt("readonly.backups", 1);
@@ -717,6 +719,14 @@ public void setBdbSortedDuplicates(boolean enable) {
this.bdbSortedDuplicates = enable;
}
+ public void setBdbOneEnvPerStore(boolean bdbOneEnvPerStore) {
+ this.bdbOneEnvPerStore = bdbOneEnvPerStore;
+ }
+
+ public boolean isBdbOneEnvPerStore() {
+ return bdbOneEnvPerStore;
+ }
+
public int getSocketBufferSize() {
return socketBufferSize;
}
@@ -768,5 +778,4 @@ public int getNumCleanupPermits() {
public void setNumCleanupPermits(int numCleanupPermits) {
this.numCleanupPermits = numCleanupPermits;
}
-
}
@@ -17,8 +17,8 @@
package voldemort.store.bdb;
import java.io.File;
+import java.util.List;
import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
import org.apache.log4j.Logger;
@@ -30,6 +30,8 @@
import voldemort.utils.ByteArray;
import voldemort.utils.Time;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
import com.sleepycat.je.Database;
import com.sleepycat.je.DatabaseConfig;
import com.sleepycat.je.DatabaseException;
@@ -50,65 +52,89 @@
private static Logger logger = Logger.getLogger(BdbStorageConfiguration.class);
private final Object lock = new Object();
- private final Environment environment;
+ private final List<Environment> environments = Lists.newArrayList();
private final EnvironmentConfig environmentConfig;
private final DatabaseConfig databaseConfig;
- private final Map<String, BdbStorageEngine> stores = new ConcurrentHashMap<String, BdbStorageEngine>();
+ private final Map<String, BdbStorageEngine> stores = Maps.newHashMap();
+ private final String bdbMasterDir;
+ private final boolean useOneEnvPerStore;
public BdbStorageConfiguration(VoldemortConfig config) {
- try {
- environmentConfig = new EnvironmentConfig();
- environmentConfig.setTransactional(true);
- environmentConfig.setCacheSize(config.getBdbCacheSize());
- if(config.isBdbWriteTransactionsEnabled() && config.isBdbFlushTransactionsEnabled()) {
- environmentConfig.setTxnNoSync(false);
- environmentConfig.setTxnWriteNoSync(false);
- } else if(config.isBdbWriteTransactionsEnabled()
- && !config.isBdbFlushTransactionsEnabled()) {
- environmentConfig.setTxnNoSync(false);
- environmentConfig.setTxnWriteNoSync(true);
- } else {
- environmentConfig.setTxnNoSync(true);
- }
- environmentConfig.setAllowCreate(true);
- environmentConfig.setConfigParam(EnvironmentConfig.LOG_FILE_MAX,
- Long.toString(config.getBdbMaxLogFileSize()));
- environmentConfig.setConfigParam(EnvironmentConfig.CHECKPOINTER_BYTES_INTERVAL,
- Long.toString(config.getBdbCheckpointBytes()));
- environmentConfig.setConfigParam(EnvironmentConfig.CHECKPOINTER_WAKEUP_INTERVAL,
- Long.toString(config.getBdbCheckpointMs()
- * Time.US_PER_MS));
- databaseConfig = new DatabaseConfig();
- databaseConfig.setAllowCreate(true);
- databaseConfig.setSortedDuplicates(config.isBdbSortedDuplicatesEnabled());
- databaseConfig.setNodeMaxEntries(config.getBdbBtreeFanout());
- databaseConfig.setTransactional(true);
- File bdbDir = new File(config.getBdbDataDirectory());
- if(!bdbDir.exists()) {
- logger.info("Creating BDB data directory '" + bdbDir.getAbsolutePath() + "'.");
- bdbDir.mkdirs();
- }
- environment = new Environment(bdbDir, environmentConfig);
- } catch(DatabaseException e) {
- throw new StorageInitializationException(e);
+ environmentConfig = new EnvironmentConfig();
+ environmentConfig.setTransactional(true);
+ environmentConfig.setCacheSize(config.getBdbCacheSize());
+ if(config.isBdbWriteTransactionsEnabled() && config.isBdbFlushTransactionsEnabled()) {
+ environmentConfig.setTxnNoSync(false);
+ environmentConfig.setTxnWriteNoSync(false);
+ } else if(config.isBdbWriteTransactionsEnabled() && !config.isBdbFlushTransactionsEnabled()) {
+ environmentConfig.setTxnNoSync(false);
+ environmentConfig.setTxnWriteNoSync(true);
+ } else {
+ environmentConfig.setTxnNoSync(true);
}
+ environmentConfig.setAllowCreate(true);
+ environmentConfig.setConfigParam(EnvironmentConfig.LOG_FILE_MAX,
+ Long.toString(config.getBdbMaxLogFileSize()));
+ environmentConfig.setConfigParam(EnvironmentConfig.CHECKPOINTER_BYTES_INTERVAL,
+ Long.toString(config.getBdbCheckpointBytes()));
+ environmentConfig.setConfigParam(EnvironmentConfig.CHECKPOINTER_WAKEUP_INTERVAL,
+ Long.toString(config.getBdbCheckpointMs() * Time.US_PER_MS));
+ databaseConfig = new DatabaseConfig();
+ databaseConfig.setAllowCreate(true);
+ databaseConfig.setSortedDuplicates(config.isBdbSortedDuplicatesEnabled());
+ databaseConfig.setNodeMaxEntries(config.getBdbBtreeFanout());
+ databaseConfig.setTransactional(true);
+ bdbMasterDir = config.getBdbDataDirectory();
+ useOneEnvPerStore = config.isBdbOneEnvPerStore();
+ if(useOneEnvPerStore)
+ environmentConfig.setSharedCache(true);
}
public StorageEngine<ByteArray, byte[]> getStore(String storeName) {
synchronized(lock) {
- if(stores.containsKey(storeName)) {
+ BdbStorageEngine store = stores.get(storeName);
+ if(store != null)
return stores.get(storeName);
- } else {
- try {
- Database db = environment.openDatabase(null, storeName, databaseConfig);
- BdbStorageEngine engine = new BdbStorageEngine(storeName, environment, db);
- stores.put(storeName, engine);
- return engine;
- } catch(DatabaseException d) {
- throw new StorageInitializationException(d);
- }
+ try {
+ Environment environment = getEnvironment(storeName);
+ Database db = environment.openDatabase(null, storeName, databaseConfig);
+ BdbStorageEngine engine = new BdbStorageEngine(storeName, environment, db);
+ stores.put(storeName, engine);
+ return engine;
+ } catch(DatabaseException d) {
+ throw new StorageInitializationException(d);
+ }
+ }
+ }
+
+ private Environment getEnvironment(String storeName) throws DatabaseException {
+ if(useOneEnvPerStore) {
+ File bdbDir = new File(bdbMasterDir, storeName);
+
+ if(!bdbDir.exists()) {
+ logger.info("Creating BDB data directory '" + bdbDir.getAbsolutePath()
+ + "' for store'" + storeName + "'.");
+ bdbDir.mkdirs();
}
+
+ Environment environment = new Environment(bdbDir, environmentConfig);
+ environments.add(environment);
+ return environment;
+ }
+
+ if(!environments.isEmpty())
+ return environments.get(0);
+
+ File bdbDir = new File(bdbMasterDir);
+
+ if(!bdbDir.exists()) {
+ logger.info("Creating BDB data directory '" + bdbDir.getAbsolutePath() + "'.");
+ bdbDir.mkdirs();
}
+
+ Environment environment = new Environment(bdbDir, environmentConfig);
+ environments.add(environment);
+ return environment;
}
public String getType() {
@@ -118,8 +144,10 @@ public String getType() {
public void close() {
synchronized(lock) {
try {
- this.environment.sync();
- this.environment.close();
+ for(Environment environment: environments) {
+ environment.sync();
+ environment.close();
+ }
} catch(DatabaseException e) {
throw new VoldemortException(e);
}

0 comments on commit 21de293

Please sign in to comment.