|
@@ -20,7 +20,9 @@ package org.apache.hadoop.cblock.jscsiHelper;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import com.google.common.primitives.Longs;
|
|
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
+import org.apache.hadoop.cblock.CBlockConfigKeys;
|
|
|
import org.apache.hadoop.cblock.jscsiHelper.cache.LogicalBlock;
|
|
|
+import org.apache.hadoop.cblock.jscsiHelper.cache.impl.AsyncBlockWriter;
|
|
|
import org.apache.hadoop.cblock.jscsiHelper.cache.impl.DiskBlock;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
@@ -100,6 +102,7 @@ public class ContainerCacheFlusher implements Runnable {
|
|
|
private final CBlockTargetMetrics metrics;
|
|
|
private AtomicBoolean shutdown;
|
|
|
private final long levelDBCacheSize;
|
|
|
+ private final int maxRetryCount;
|
|
|
|
|
|
private final ConcurrentMap<String, FinishCounter> finishCountMap;
|
|
|
|
|
@@ -152,28 +155,33 @@ public class ContainerCacheFlusher implements Runnable {
|
|
|
this.remoteIO = new AtomicLong();
|
|
|
|
|
|
this.finishCountMap = new ConcurrentHashMap<>();
|
|
|
+ this.maxRetryCount =
|
|
|
+ config.getInt(CBlockConfigKeys.DFS_CBLOCK_CACHE_MAX_RETRY_KEY,
|
|
|
+ CBlockConfigKeys.DFS_CBLOCK_CACHE_MAX_RETRY_DEFAULT);
|
|
|
}
|
|
|
|
|
|
- private void checkExistingDirtyLog(File dbPath) {
|
|
|
+ private void checkExistingLog(String prefixFileName, File dbPath) {
|
|
|
if (!dbPath.exists()) {
|
|
|
LOG.debug("No existing dirty log found at {}", dbPath);
|
|
|
return;
|
|
|
}
|
|
|
LOG.debug("Need to check and requeue existing dirty log {}", dbPath);
|
|
|
HashMap<String, ArrayList<String>> allFiles = new HashMap<>();
|
|
|
- traverse(dbPath, allFiles);
|
|
|
+ traverse(prefixFileName, dbPath, allFiles);
|
|
|
for (Map.Entry<String, ArrayList<String>> entry : allFiles.entrySet()) {
|
|
|
String parentPath = entry.getKey();
|
|
|
for (String fileName : entry.getValue()) {
|
|
|
- LOG.info("found this {} with {}", parentPath, fileName);
|
|
|
+ LOG.info("found {} {} with prefix {}",
|
|
|
+ parentPath, fileName, prefixFileName);
|
|
|
processDirtyBlocks(parentPath, fileName);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void traverse(File path, HashMap<String, ArrayList<String>> files) {
|
|
|
+ private void traverse(String prefixFileName, File path,
|
|
|
+ HashMap<String, ArrayList<String>> files) {
|
|
|
if (path.isFile()) {
|
|
|
- if (path.getName().startsWith("DirtyLog")) {
|
|
|
+ if (path.getName().startsWith(prefixFileName)) {
|
|
|
LOG.debug("found this {} with {}", path.getParent(), path.getName());
|
|
|
if (!files.containsKey(path.getParent())) {
|
|
|
files.put(path.getParent(), new ArrayList<>());
|
|
@@ -184,7 +192,7 @@ public class ContainerCacheFlusher implements Runnable {
|
|
|
File[] listFiles = path.listFiles();
|
|
|
if (listFiles != null) {
|
|
|
for (File subPath : listFiles) {
|
|
|
- traverse(subPath, files);
|
|
|
+ traverse(prefixFileName, subPath, files);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -274,19 +282,28 @@ public class ContainerCacheFlusher implements Runnable {
|
|
|
public void register(String dbPath, Pipeline[] containerList) {
|
|
|
File dbFile = Paths.get(dbPath).toFile();
|
|
|
pipelineMap.put(dbPath, containerList);
|
|
|
- checkExistingDirtyLog(dbFile);
|
|
|
+ checkExistingLog(AsyncBlockWriter.DIRTY_LOG_PREFIX, dbFile);
|
|
|
+ checkExistingLog(AsyncBlockWriter.RETRY_LOG_PREFIX, dbFile);
|
|
|
}
|
|
|
|
|
|
private String getDBFileName(String dbPath) {
|
|
|
return dbPath + ".db";
|
|
|
}
|
|
|
|
|
|
- LevelDBStore getCacheDB(String dbPath) {
|
|
|
- return dbMap.get(dbPath).db;
|
|
|
+ public LevelDBStore getCacheDB(String dbPath) throws IOException {
|
|
|
+ return openDB(dbPath);
|
|
|
}
|
|
|
|
|
|
+ public void releaseCacheDB(String dbPath) {
|
|
|
+ try {
|
|
|
+ closeDB(dbPath);
|
|
|
+ } catch (Exception e) {
|
|
|
+ metrics.incNumFailedReleaseLevelDB();
|
|
|
+ LOG.error("LevelDB close failed, dbPath:" + dbPath, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
/**
|
|
|
- * Close the DB if we don't have any outstanding refrences.
|
|
|
+ * Close the DB if we don't have any outstanding references.
|
|
|
*
|
|
|
* @param dbPath - dbPath
|
|
|
* @throws IOException
|
|
@@ -348,18 +365,28 @@ public class ContainerCacheFlusher implements Runnable {
|
|
|
message.getDbPath(), message.getFileName());
|
|
|
String fullPath = Paths.get(message.getDbPath(),
|
|
|
message.getFileName()).toString();
|
|
|
+ String[] fileNameParts = message.getFileName().split("\\.");
|
|
|
+ Preconditions.checkState(fileNameParts.length > 1);
|
|
|
+ String fileType = fileNameParts[0];
|
|
|
+ boolean isDirtyLogFile =
|
|
|
+ fileType.equalsIgnoreCase(AsyncBlockWriter.DIRTY_LOG_PREFIX);
|
|
|
ReadableByteChannel fileChannel = new FileInputStream(fullPath)
|
|
|
.getChannel();
|
|
|
// TODO: We can batch and unique the IOs here. First getting the code
|
|
|
// to work, we will add those later.
|
|
|
int bytesRead = fileChannel.read(blockIDBuffer);
|
|
|
+ fileChannel.close();
|
|
|
LOG.debug("Read blockID log of size: {} position {} remaining {}",
|
|
|
bytesRead, blockIDBuffer.position(), blockIDBuffer.remaining());
|
|
|
// current position of in the buffer in bytes, divided by number of
|
|
|
// bytes per long (which is calculated by number of bits per long
|
|
|
// divided by number of bits per byte) gives the number of blocks
|
|
|
int blockCount = blockIDBuffer.position()/(Long.SIZE / Byte.SIZE);
|
|
|
- getTargetMetrics().incNumBytesDirtyLogRead(bytesRead);
|
|
|
+ if (isDirtyLogFile) {
|
|
|
+ getTargetMetrics().incNumBytesDirtyLogRead(bytesRead);
|
|
|
+ } else {
|
|
|
+ getTargetMetrics().incNumBytesRetryLogRead(bytesRead);
|
|
|
+ }
|
|
|
if (finishCountMap.containsKey(message.getFileName())) {
|
|
|
// In theory this should never happen. But if it happened,
|
|
|
// we need to know it...
|
|
@@ -375,14 +402,22 @@ public class ContainerCacheFlusher implements Runnable {
|
|
|
// should be flip instead of rewind, because we also need to make sure
|
|
|
// the end position is correct.
|
|
|
blockIDBuffer.flip();
|
|
|
- LOG.debug("Remaining blocks count {} and {}", blockIDBuffer.remaining(),
|
|
|
+ LOG.info("Remaining blocks count {} and {}", blockIDBuffer.remaining(),
|
|
|
blockCount);
|
|
|
while (blockIDBuffer.remaining() >= (Long.SIZE / Byte.SIZE)) {
|
|
|
- getTargetMetrics().incNumDirtyLogBlockRead();
|
|
|
long blockID = blockIDBuffer.getLong();
|
|
|
+ int retryCount = 0;
|
|
|
+ if (isDirtyLogFile) {
|
|
|
+ getTargetMetrics().incNumDirtyLogBlockRead();
|
|
|
+ } else {
|
|
|
+ getTargetMetrics().incNumRetryLogBlockRead();
|
|
|
+ Preconditions.checkState(fileNameParts.length == 4);
|
|
|
+ retryCount = Integer.parseInt(fileNameParts[3]);
|
|
|
+ }
|
|
|
LogicalBlock block = new DiskBlock(blockID, null, false);
|
|
|
BlockWriterTask blockWriterTask = new BlockWriterTask(block, this,
|
|
|
- message.getDbPath(), message.getFileName());
|
|
|
+ message.getDbPath(), retryCount, message.getFileName(),
|
|
|
+ maxRetryCount);
|
|
|
threadPoolExecutor.submit(blockWriterTask);
|
|
|
}
|
|
|
blockIDBuffer.clear();
|
|
@@ -491,7 +526,6 @@ public class ContainerCacheFlusher implements Runnable {
|
|
|
this.currentCount = new AtomicLong(0);
|
|
|
this.fileDeleted = new AtomicBoolean(false);
|
|
|
this.flusher = flusher;
|
|
|
- this.flusher.openDB(dbPath);
|
|
|
}
|
|
|
|
|
|
public boolean isFileDeleted() {
|
|
@@ -505,7 +539,6 @@ public class ContainerCacheFlusher implements Runnable {
|
|
|
LOG.debug(
|
|
|
"Deleting {} with count {} {}", filePath, count, expectedCount);
|
|
|
try {
|
|
|
- flusher.closeDB(dbPath);
|
|
|
Path path = Paths.get(filePath);
|
|
|
Files.delete(path);
|
|
|
// the following part tries to remove the directory if it is empty
|