|
@@ -21,12 +21,14 @@ package org.apache.hadoop.ozone.container.common.utils;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import org.apache.commons.collections.MapIterator;
|
|
|
import org.apache.commons.collections.map.LRUMap;
|
|
|
-import org.apache.commons.logging.Log;
|
|
|
-import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.utils.MetadataStore;
|
|
|
+import org.apache.hadoop.utils.MetadataStoreBuilder;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.util.concurrent.locks.Lock;
|
|
|
import java.util.concurrent.locks.ReentrantLock;
|
|
@@ -35,7 +37,8 @@ import java.util.concurrent.locks.ReentrantLock;
|
|
|
* container cache is a LRUMap that maintains the DB handles.
|
|
|
*/
|
|
|
public final class ContainerCache extends LRUMap {
|
|
|
- static final Log LOG = LogFactory.getLog(ContainerCache.class);
|
|
|
+ private static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(ContainerCache.class);
|
|
|
private final Lock lock = new ReentrantLock();
|
|
|
private static ContainerCache cache;
|
|
|
private static final float LOAD_FACTOR = 0.75f;
|
|
@@ -115,17 +118,31 @@ public final class ContainerCache extends LRUMap {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Returns a DB handle if available, null otherwise.
|
|
|
+ * Returns a DB handle if available, create the handler otherwise.
|
|
|
*
|
|
|
* @param containerName - Name of the container.
|
|
|
* @return MetadataStore.
|
|
|
*/
|
|
|
- public MetadataStore getDB(String containerName) {
|
|
|
+ public MetadataStore getDB(String containerName, String containerDBPath)
|
|
|
+ throws IOException {
|
|
|
Preconditions.checkNotNull(containerName);
|
|
|
Preconditions.checkState(!containerName.isEmpty());
|
|
|
lock.lock();
|
|
|
try {
|
|
|
- return (MetadataStore) this.get(containerName);
|
|
|
+ MetadataStore db = (MetadataStore) this.get(containerName);
|
|
|
+
|
|
|
+ if (db == null) {
|
|
|
+ db = MetadataStoreBuilder.newBuilder()
|
|
|
+ .setDbFile(new File(containerDBPath))
|
|
|
+ .setCreateIfMissing(false)
|
|
|
+ .build();
|
|
|
+ this.put(containerName, db);
|
|
|
+ }
|
|
|
+ return db;
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.error("Error opening DB. Container:{} ContainerPath:{}",
|
|
|
+ containerName, containerDBPath, e);
|
|
|
+ throw e;
|
|
|
} finally {
|
|
|
lock.unlock();
|
|
|
}
|
|
@@ -141,28 +158,11 @@ public final class ContainerCache extends LRUMap {
|
|
|
Preconditions.checkState(!containerName.isEmpty());
|
|
|
lock.lock();
|
|
|
try {
|
|
|
- MetadataStore db = this.getDB(containerName);
|
|
|
+ MetadataStore db = (MetadataStore)this.get(containerName);
|
|
|
closeDB(containerName, db);
|
|
|
this.remove(containerName);
|
|
|
} finally {
|
|
|
lock.unlock();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Add a new DB to the cache.
|
|
|
- *
|
|
|
- * @param containerName - Name of the container
|
|
|
- * @param db - DB handle
|
|
|
- */
|
|
|
- public void putDB(String containerName, MetadataStore db) {
|
|
|
- Preconditions.checkNotNull(containerName);
|
|
|
- Preconditions.checkState(!containerName.isEmpty());
|
|
|
- lock.lock();
|
|
|
- try {
|
|
|
- this.put(containerName, db);
|
|
|
- } finally {
|
|
|
- lock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
}
|