|
@@ -23,6 +23,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
import org.apache.hadoop.cblock.jscsiHelper.cache.LogicalBlock;
|
|
import org.apache.hadoop.cblock.jscsiHelper.cache.LogicalBlock;
|
|
import org.apache.hadoop.cblock.jscsiHelper.cache.impl.DiskBlock;
|
|
import org.apache.hadoop.cblock.jscsiHelper.cache.impl.DiskBlock;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.ozone.OzoneConsts;
|
|
import org.apache.hadoop.scm.XceiverClientManager;
|
|
import org.apache.hadoop.scm.XceiverClientManager;
|
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
import org.apache.hadoop.utils.LevelDBStore;
|
|
import org.apache.hadoop.utils.LevelDBStore;
|
|
@@ -77,8 +78,10 @@ import static org.apache.hadoop.cblock.CBlockConfigKeys
|
|
.DFS_CBLOCK_CACHE_THREAD_PRIORITY;
|
|
.DFS_CBLOCK_CACHE_THREAD_PRIORITY;
|
|
import static org.apache.hadoop.cblock.CBlockConfigKeys
|
|
import static org.apache.hadoop.cblock.CBlockConfigKeys
|
|
.DFS_CBLOCK_CACHE_THREAD_PRIORITY_DEFAULT;
|
|
.DFS_CBLOCK_CACHE_THREAD_PRIORITY_DEFAULT;
|
|
-import static org.apache.hadoop.cblock.CBlockConfigKeys.DFS_CBLOCK_DISK_CACHE_PATH_DEFAULT;
|
|
|
|
-import static org.apache.hadoop.cblock.CBlockConfigKeys.DFS_CBLOCK_DISK_CACHE_PATH_KEY;
|
|
|
|
|
|
+import static org.apache.hadoop.cblock.CBlockConfigKeys
|
|
|
|
+ .DFS_CBLOCK_CACHE_LEVELDB_CACHE_SIZE_MB_KEY;
|
|
|
|
+import static org.apache.hadoop.cblock.CBlockConfigKeys
|
|
|
|
+ .DFS_CBLOCK_CACHE_LEVELDB_CACHE_SIZE_MB_DEFAULT;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Class that writes to remote containers.
|
|
* Class that writes to remote containers.
|
|
@@ -96,6 +99,7 @@ public class ContainerCacheFlusher implements Runnable {
|
|
private final XceiverClientManager xceiverClientManager;
|
|
private final XceiverClientManager xceiverClientManager;
|
|
private final CBlockTargetMetrics metrics;
|
|
private final CBlockTargetMetrics metrics;
|
|
private AtomicBoolean shutdown;
|
|
private AtomicBoolean shutdown;
|
|
|
|
+ private final long levelDBCacheSize;
|
|
|
|
|
|
private final ConcurrentMap<String, FinishCounter> finishCountMap;
|
|
private final ConcurrentMap<String, FinishCounter> finishCountMap;
|
|
|
|
|
|
@@ -117,6 +121,8 @@ public class ContainerCacheFlusher implements Runnable {
|
|
DFS_CBLOCK_CACHE_THREAD_PRIORITY_DEFAULT);
|
|
DFS_CBLOCK_CACHE_THREAD_PRIORITY_DEFAULT);
|
|
int blockBufferSize = config.getInt(DFS_CBLOCK_CACHE_BLOCK_BUFFER_SIZE,
|
|
int blockBufferSize = config.getInt(DFS_CBLOCK_CACHE_BLOCK_BUFFER_SIZE,
|
|
DFS_CBLOCK_CACHE_BLOCK_BUFFER_SIZE_DEFAULT) * (Long.SIZE / Byte.SIZE);
|
|
DFS_CBLOCK_CACHE_BLOCK_BUFFER_SIZE_DEFAULT) * (Long.SIZE / Byte.SIZE);
|
|
|
|
+ levelDBCacheSize = config.getInt(DFS_CBLOCK_CACHE_LEVELDB_CACHE_SIZE_MB_KEY,
|
|
|
|
+ DFS_CBLOCK_CACHE_LEVELDB_CACHE_SIZE_MB_DEFAULT) * OzoneConsts.MB;
|
|
|
|
|
|
LOG.info("Cache: Core Pool Size: {}", corePoolSize);
|
|
LOG.info("Cache: Core Pool Size: {}", corePoolSize);
|
|
LOG.info("Cache: Keep Alive: {}", keepAlive);
|
|
LOG.info("Cache: Keep Alive: {}", keepAlive);
|
|
@@ -146,17 +152,14 @@ public class ContainerCacheFlusher implements Runnable {
|
|
this.remoteIO = new AtomicLong();
|
|
this.remoteIO = new AtomicLong();
|
|
|
|
|
|
this.finishCountMap = new ConcurrentHashMap<>();
|
|
this.finishCountMap = new ConcurrentHashMap<>();
|
|
- checkExisitingDirtyLog(config);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- private void checkExisitingDirtyLog(Configuration config) {
|
|
|
|
- File dbPath = Paths.get(config.get(DFS_CBLOCK_DISK_CACHE_PATH_KEY,
|
|
|
|
- DFS_CBLOCK_DISK_CACHE_PATH_DEFAULT)).toFile();
|
|
|
|
|
|
+ private void checkExistingDirtyLog(File dbPath) {
|
|
if (!dbPath.exists()) {
|
|
if (!dbPath.exists()) {
|
|
- LOG.info("No existing dirty log found at {}", dbPath);
|
|
|
|
|
|
+ LOG.debug("No existing dirty log found at {}", dbPath);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- LOG.info("Need to check and requeue existing dirty log {}", dbPath);
|
|
|
|
|
|
+ LOG.debug("Need to check and requeue existing dirty log {}", dbPath);
|
|
HashMap<String, ArrayList<String>> allFiles = new HashMap<>();
|
|
HashMap<String, ArrayList<String>> allFiles = new HashMap<>();
|
|
traverse(dbPath, allFiles);
|
|
traverse(dbPath, allFiles);
|
|
for (Map.Entry<String, ArrayList<String>> entry : allFiles.entrySet()) {
|
|
for (Map.Entry<String, ArrayList<String>> entry : allFiles.entrySet()) {
|
|
@@ -237,11 +240,10 @@ public class ContainerCacheFlusher implements Runnable {
|
|
* Opens a DB if needed or returns a handle to an already open DB.
|
|
* Opens a DB if needed or returns a handle to an already open DB.
|
|
*
|
|
*
|
|
* @param dbPath -- dbPath
|
|
* @param dbPath -- dbPath
|
|
- * @param cacheSize - cacheSize
|
|
|
|
* @return the levelDB on the given path.
|
|
* @return the levelDB on the given path.
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
- public synchronized LevelDBStore openDB(String dbPath, int cacheSize)
|
|
|
|
|
|
+ public synchronized LevelDBStore openDB(String dbPath)
|
|
throws IOException {
|
|
throws IOException {
|
|
if (dbMap.containsKey(dbPath)) {
|
|
if (dbMap.containsKey(dbPath)) {
|
|
RefCountedDB refDB = dbMap.get(dbPath);
|
|
RefCountedDB refDB = dbMap.get(dbPath);
|
|
@@ -249,7 +251,7 @@ public class ContainerCacheFlusher implements Runnable {
|
|
return refDB.db;
|
|
return refDB.db;
|
|
} else {
|
|
} else {
|
|
Options options = new Options();
|
|
Options options = new Options();
|
|
- options.cacheSize(cacheSize * (1024L * 1024L));
|
|
|
|
|
|
+ options.cacheSize(levelDBCacheSize);
|
|
options.createIfMissing(true);
|
|
options.createIfMissing(true);
|
|
LevelDBStore cacheDB = new LevelDBStore(
|
|
LevelDBStore cacheDB = new LevelDBStore(
|
|
new File(getDBFileName(dbPath)), options);
|
|
new File(getDBFileName(dbPath)), options);
|
|
@@ -260,14 +262,19 @@ public class ContainerCacheFlusher implements Runnable {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Updates the contianer map. This data never changes so we will update this
|
|
|
|
|
|
+ * Updates the container map. This data never changes so we will update this
|
|
* during restarts and it should not hurt us.
|
|
* during restarts and it should not hurt us.
|
|
*
|
|
*
|
|
|
|
+ * Once a CBlockLocalCache cache is registered, requeue dirty/retry log files
|
|
|
|
+ * for the volume
|
|
|
|
+ *
|
|
* @param dbPath - DbPath
|
|
* @param dbPath - DbPath
|
|
- * @param containerList - Contianer List.
|
|
|
|
|
|
+ * @param containerList - Container List.
|
|
*/
|
|
*/
|
|
public void register(String dbPath, Pipeline[] containerList) {
|
|
public void register(String dbPath, Pipeline[] containerList) {
|
|
|
|
+ File dbFile = Paths.get(dbPath).toFile();
|
|
pipelineMap.put(dbPath, containerList);
|
|
pipelineMap.put(dbPath, containerList);
|
|
|
|
+ checkExistingDirtyLog(dbFile);
|
|
}
|
|
}
|
|
|
|
|
|
private String getDBFileName(String dbPath) {
|
|
private String getDBFileName(String dbPath) {
|
|
@@ -363,7 +370,7 @@ public class ContainerCacheFlusher implements Runnable {
|
|
}
|
|
}
|
|
finishCountMap.put(message.getFileName(),
|
|
finishCountMap.put(message.getFileName(),
|
|
new FinishCounter(blockCount, message.getDbPath(),
|
|
new FinishCounter(blockCount, message.getDbPath(),
|
|
- message.getFileName()));
|
|
|
|
|
|
+ message.getFileName(), this));
|
|
// should be flip instead of rewind, because we also need to make sure
|
|
// should be flip instead of rewind, because we also need to make sure
|
|
// the end position is correct.
|
|
// the end position is correct.
|
|
blockIDBuffer.flip();
|
|
blockIDBuffer.flip();
|
|
@@ -473,14 +480,17 @@ public class ContainerCacheFlusher implements Runnable {
|
|
private final String dirtyLogPath;
|
|
private final String dirtyLogPath;
|
|
private final AtomicLong currentCount;
|
|
private final AtomicLong currentCount;
|
|
private AtomicBoolean fileDeleted;
|
|
private AtomicBoolean fileDeleted;
|
|
|
|
+ private final ContainerCacheFlusher flusher;
|
|
|
|
|
|
FinishCounter(long expectedCount, String dbPath,
|
|
FinishCounter(long expectedCount, String dbPath,
|
|
- String dirtyLogPath) {
|
|
|
|
|
|
+ String dirtyLogPath, ContainerCacheFlusher flusher) throws IOException {
|
|
this.expectedCount = expectedCount;
|
|
this.expectedCount = expectedCount;
|
|
this.dbPath = dbPath;
|
|
this.dbPath = dbPath;
|
|
this.dirtyLogPath = dirtyLogPath;
|
|
this.dirtyLogPath = dirtyLogPath;
|
|
this.currentCount = new AtomicLong(0);
|
|
this.currentCount = new AtomicLong(0);
|
|
this.fileDeleted = new AtomicBoolean(false);
|
|
this.fileDeleted = new AtomicBoolean(false);
|
|
|
|
+ this.flusher = flusher;
|
|
|
|
+ this.flusher.openDB(dbPath);
|
|
}
|
|
}
|
|
|
|
|
|
public boolean isFileDeleted() {
|
|
public boolean isFileDeleted() {
|
|
@@ -494,6 +504,7 @@ public class ContainerCacheFlusher implements Runnable {
|
|
LOG.debug(
|
|
LOG.debug(
|
|
"Deleting {} with count {} {}", filePath, count, expectedCount);
|
|
"Deleting {} with count {} {}", filePath, count, expectedCount);
|
|
try {
|
|
try {
|
|
|
|
+ flusher.closeDB(dbPath);
|
|
Path path = Paths.get(filePath);
|
|
Path path = Paths.get(filePath);
|
|
Files.delete(path);
|
|
Files.delete(path);
|
|
// the following part tries to remove the directory if it is empty
|
|
// the following part tries to remove the directory if it is empty
|
|
@@ -504,9 +515,8 @@ public class ContainerCacheFlusher implements Runnable {
|
|
Files.delete(parent);
|
|
Files.delete(parent);
|
|
}*/
|
|
}*/
|
|
fileDeleted.set(true);
|
|
fileDeleted.set(true);
|
|
- } catch (IOException e) {
|
|
|
|
- LOG.error(
|
|
|
|
- "Error deleting dirty log file {} {}", filePath, e.toString());
|
|
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.error("Error deleting dirty log file:" + filePath, e);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|