|
@@ -35,6 +35,7 @@ import java.io.RandomAccessFile;
|
|
import java.nio.MappedByteBuffer;
|
|
import java.nio.MappedByteBuffer;
|
|
import java.nio.channels.FileChannel;
|
|
import java.nio.channels.FileChannel;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.UUID;
|
|
import java.util.UUID;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
@@ -45,14 +46,19 @@ import java.util.concurrent.atomic.AtomicLong;
|
|
*/
|
|
*/
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@InterfaceStability.Unstable
|
|
@InterfaceStability.Unstable
|
|
-public class PmemVolumeManager {
|
|
|
|
|
|
+public final class PmemVolumeManager {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Counts used bytes for persistent memory.
|
|
* Counts used bytes for persistent memory.
|
|
*/
|
|
*/
|
|
- private class UsedBytesCount {
|
|
|
|
|
|
+ private static class UsedBytesCount {
|
|
|
|
+ private final long maxBytes;
|
|
private final AtomicLong usedBytes = new AtomicLong(0);
|
|
private final AtomicLong usedBytes = new AtomicLong(0);
|
|
|
|
|
|
|
|
+ UsedBytesCount(long maxBytes) {
|
|
|
|
+ this.maxBytes = maxBytes;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Try to reserve more bytes.
|
|
* Try to reserve more bytes.
|
|
*
|
|
*
|
|
@@ -65,7 +71,7 @@ public class PmemVolumeManager {
|
|
while (true) {
|
|
while (true) {
|
|
long cur = usedBytes.get();
|
|
long cur = usedBytes.get();
|
|
long next = cur + bytesCount;
|
|
long next = cur + bytesCount;
|
|
- if (next > cacheCapacity) {
|
|
|
|
|
|
+ if (next > maxBytes) {
|
|
return -1;
|
|
return -1;
|
|
}
|
|
}
|
|
if (usedBytes.compareAndSet(cur, next)) {
|
|
if (usedBytes.compareAndSet(cur, next)) {
|
|
@@ -85,42 +91,76 @@ public class PmemVolumeManager {
|
|
return usedBytes.addAndGet(-bytesCount);
|
|
return usedBytes.addAndGet(-bytesCount);
|
|
}
|
|
}
|
|
|
|
|
|
- long get() {
|
|
|
|
|
|
+ long getUsedBytes() {
|
|
return usedBytes.get();
|
|
return usedBytes.get();
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ long getMaxBytes() {
|
|
|
|
+ return maxBytes;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ long getAvailableBytes() {
|
|
|
|
+ return maxBytes - usedBytes.get();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
private static final Logger LOG =
|
|
private static final Logger LOG =
|
|
LoggerFactory.getLogger(PmemVolumeManager.class);
|
|
LoggerFactory.getLogger(PmemVolumeManager.class);
|
|
|
|
+ public static final String CACHE_DIR = "hdfs_pmem_cache";
|
|
|
|
+ private static PmemVolumeManager pmemVolumeManager = null;
|
|
private final ArrayList<String> pmemVolumes = new ArrayList<>();
|
|
private final ArrayList<String> pmemVolumes = new ArrayList<>();
|
|
// Maintain which pmem volume a block is cached to.
|
|
// Maintain which pmem volume a block is cached to.
|
|
private final Map<ExtendedBlockId, Byte> blockKeyToVolume =
|
|
private final Map<ExtendedBlockId, Byte> blockKeyToVolume =
|
|
new ConcurrentHashMap<>();
|
|
new ConcurrentHashMap<>();
|
|
- private final UsedBytesCount usedBytesCount;
|
|
|
|
|
|
+ private final List<UsedBytesCount> usedBytesCounts = new ArrayList<>();
|
|
|
|
|
|
/**
|
|
/**
|
|
* The total cache capacity in bytes of persistent memory.
|
|
* The total cache capacity in bytes of persistent memory.
|
|
- * It is 0L if the specific mappableBlockLoader couldn't cache data to pmem.
|
|
|
|
*/
|
|
*/
|
|
- private final long cacheCapacity;
|
|
|
|
|
|
+ private long cacheCapacity;
|
|
|
|
+ private static long maxBytesPerPmem = -1;
|
|
private int count = 0;
|
|
private int count = 0;
|
|
- // Strict atomic operation is not guaranteed for the performance sake.
|
|
|
|
- private int i = 0;
|
|
|
|
|
|
+ private byte nextIndex = 0;
|
|
|
|
|
|
- PmemVolumeManager(long maxBytes, String[] pmemVolumesConfigured)
|
|
|
|
- throws IOException {
|
|
|
|
- if (pmemVolumesConfigured == null || pmemVolumesConfigured.length == 0) {
|
|
|
|
|
|
+ private PmemVolumeManager(String[] pmemVolumesConfig) throws IOException {
|
|
|
|
+ if (pmemVolumesConfig == null || pmemVolumesConfig.length == 0) {
|
|
throw new IOException("The persistent memory volume, " +
|
|
throw new IOException("The persistent memory volume, " +
|
|
DFSConfigKeys.DFS_DATANODE_CACHE_PMEM_DIRS_KEY +
|
|
DFSConfigKeys.DFS_DATANODE_CACHE_PMEM_DIRS_KEY +
|
|
" is not configured!");
|
|
" is not configured!");
|
|
}
|
|
}
|
|
- this.loadVolumes(pmemVolumesConfigured);
|
|
|
|
- this.usedBytesCount = new UsedBytesCount();
|
|
|
|
- this.cacheCapacity = maxBytes;
|
|
|
|
|
|
+ this.loadVolumes(pmemVolumesConfig);
|
|
|
|
+ cacheCapacity = 0L;
|
|
|
|
+ for (UsedBytesCount counter : usedBytesCounts) {
|
|
|
|
+ cacheCapacity += counter.getMaxBytes();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public synchronized static void init(String[] pmemVolumesConfig)
|
|
|
|
+ throws IOException {
|
|
|
|
+ if (pmemVolumeManager == null) {
|
|
|
|
+ pmemVolumeManager = new PmemVolumeManager(pmemVolumesConfig);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static PmemVolumeManager getInstance() {
|
|
|
|
+ if (pmemVolumeManager == null) {
|
|
|
|
+ throw new RuntimeException(
|
|
|
|
+ "The pmemVolumeManager should be instantiated!");
|
|
|
|
+ }
|
|
|
|
+ return pmemVolumeManager;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ public static void setMaxBytes(long maxBytes) {
|
|
|
|
+ maxBytesPerPmem = maxBytes;
|
|
}
|
|
}
|
|
|
|
|
|
public long getCacheUsed() {
|
|
public long getCacheUsed() {
|
|
- return usedBytesCount.get();
|
|
|
|
|
|
+ long usedBytes = 0L;
|
|
|
|
+ for (UsedBytesCount counter : usedBytesCounts) {
|
|
|
|
+ usedBytes += counter.getUsedBytes();
|
|
|
|
+ }
|
|
|
|
+ return usedBytes;
|
|
}
|
|
}
|
|
|
|
|
|
public long getCacheCapacity() {
|
|
public long getCacheCapacity() {
|
|
@@ -130,24 +170,40 @@ public class PmemVolumeManager {
|
|
/**
|
|
/**
|
|
* Try to reserve more bytes on persistent memory.
|
|
* Try to reserve more bytes on persistent memory.
|
|
*
|
|
*
|
|
|
|
+ * @param key The ExtendedBlockId for a block.
|
|
|
|
+ *
|
|
* @param bytesCount The number of bytes to add.
|
|
* @param bytesCount The number of bytes to add.
|
|
*
|
|
*
|
|
* @return The new number of usedBytes if we succeeded;
|
|
* @return The new number of usedBytes if we succeeded;
|
|
* -1 if we failed.
|
|
* -1 if we failed.
|
|
*/
|
|
*/
|
|
- long reserve(long bytesCount) {
|
|
|
|
- return usedBytesCount.reserve(bytesCount);
|
|
|
|
|
|
+ synchronized long reserve(ExtendedBlockId key, long bytesCount) {
|
|
|
|
+ try {
|
|
|
|
+ byte index = chooseVolume(bytesCount);
|
|
|
|
+ long usedBytes = usedBytesCounts.get(index).reserve(bytesCount);
|
|
|
|
+ // Put the entry into blockKeyToVolume if reserving bytes succeeded.
|
|
|
|
+ if (usedBytes > 0) {
|
|
|
|
+ blockKeyToVolume.put(key, index);
|
|
|
|
+ }
|
|
|
|
+ return usedBytes;
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.warn(e.getMessage());
|
|
|
|
+ return -1L;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Release some bytes that we're using on persistent memory.
|
|
* Release some bytes that we're using on persistent memory.
|
|
*
|
|
*
|
|
|
|
+ * @param key The ExtendedBlockId for a block.
|
|
|
|
+ *
|
|
* @param bytesCount The number of bytes to release.
|
|
* @param bytesCount The number of bytes to release.
|
|
*
|
|
*
|
|
* @return The new number of usedBytes.
|
|
* @return The new number of usedBytes.
|
|
*/
|
|
*/
|
|
- long release(long bytesCount) {
|
|
|
|
- return usedBytesCount.release(bytesCount);
|
|
|
|
|
|
+ long release(ExtendedBlockId key, long bytesCount) {
|
|
|
|
+ Byte index = blockKeyToVolume.remove(key);
|
|
|
|
+ return usedBytesCounts.get(index).release(bytesCount);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -155,46 +211,70 @@ public class PmemVolumeManager {
|
|
*
|
|
*
|
|
* @throws IOException If there is no available pmem volume.
|
|
* @throws IOException If there is no available pmem volume.
|
|
*/
|
|
*/
|
|
- private void loadVolumes(String[] volumes) throws IOException {
|
|
|
|
|
|
+ private void loadVolumes(String[] volumes)
|
|
|
|
+ throws IOException {
|
|
// Check whether the volume exists
|
|
// Check whether the volume exists
|
|
- for (String volume: volumes) {
|
|
|
|
|
|
+ for (byte n = 0; n < volumes.length; n++) {
|
|
try {
|
|
try {
|
|
- File pmemDir = new File(volume);
|
|
|
|
- verifyIfValidPmemVolume(pmemDir);
|
|
|
|
- // Remove all files under the volume.
|
|
|
|
- FileUtils.cleanDirectory(pmemDir);
|
|
|
|
|
|
+ File pmemDir = new File(volumes[n]);
|
|
|
|
+ File realPmemDir = verifyIfValidPmemVolume(pmemDir);
|
|
|
|
+ this.pmemVolumes.add(realPmemDir.getPath());
|
|
|
|
+ long maxBytes;
|
|
|
|
+ if (maxBytesPerPmem == -1) {
|
|
|
|
+ maxBytes = realPmemDir.getUsableSpace();
|
|
|
|
+ } else {
|
|
|
|
+ maxBytes = maxBytesPerPmem;
|
|
|
|
+ }
|
|
|
|
+ UsedBytesCount usedBytesCount = new UsedBytesCount(maxBytes);
|
|
|
|
+ this.usedBytesCounts.add(usedBytesCount);
|
|
|
|
+ LOG.info("Added persistent memory - {} with size={}",
|
|
|
|
+ volumes[n], maxBytes);
|
|
} catch (IllegalArgumentException e) {
|
|
} catch (IllegalArgumentException e) {
|
|
- LOG.error("Failed to parse persistent memory volume " + volume, e);
|
|
|
|
|
|
+ LOG.error("Failed to parse persistent memory volume " + volumes[n], e);
|
|
continue;
|
|
continue;
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
- LOG.error("Bad persistent memory volume: " + volume, e);
|
|
|
|
|
|
+ LOG.error("Bad persistent memory volume: " + volumes[n], e);
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
- pmemVolumes.add(volume);
|
|
|
|
- LOG.info("Added persistent memory - " + volume);
|
|
|
|
}
|
|
}
|
|
count = pmemVolumes.size();
|
|
count = pmemVolumes.size();
|
|
if (count == 0) {
|
|
if (count == 0) {
|
|
throw new IOException(
|
|
throw new IOException(
|
|
"At least one valid persistent memory volume is required!");
|
|
"At least one valid persistent memory volume is required!");
|
|
}
|
|
}
|
|
|
|
+ cleanup();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ void cleanup() {
|
|
|
|
+ // Remove all files under the volume.
|
|
|
|
+ for (String pmemDir: pmemVolumes) {
|
|
|
|
+ try {
|
|
|
|
+ FileUtils.cleanDirectory(new File(pmemDir));
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Failed to clean up " + pmemDir, e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
- static void verifyIfValidPmemVolume(File pmemDir)
|
|
|
|
|
|
+ static File verifyIfValidPmemVolume(File pmemDir)
|
|
throws IOException {
|
|
throws IOException {
|
|
if (!pmemDir.exists()) {
|
|
if (!pmemDir.exists()) {
|
|
final String message = pmemDir + " does not exist";
|
|
final String message = pmemDir + " does not exist";
|
|
throw new IOException(message);
|
|
throw new IOException(message);
|
|
}
|
|
}
|
|
-
|
|
|
|
if (!pmemDir.isDirectory()) {
|
|
if (!pmemDir.isDirectory()) {
|
|
final String message = pmemDir + " is not a directory";
|
|
final String message = pmemDir + " is not a directory";
|
|
throw new IllegalArgumentException(message);
|
|
throw new IllegalArgumentException(message);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ File realPmemDir = new File(getRealPmemDir(pmemDir.getPath()));
|
|
|
|
+ if (!realPmemDir.exists() && !realPmemDir.mkdir()) {
|
|
|
|
+ throw new IOException("Failed to create " + realPmemDir.getPath());
|
|
|
|
+ }
|
|
|
|
+
|
|
String uuidStr = UUID.randomUUID().toString();
|
|
String uuidStr = UUID.randomUUID().toString();
|
|
- String testFilePath = pmemDir.getPath() + "/.verify.pmem." + uuidStr;
|
|
|
|
|
|
+ String testFilePath = realPmemDir.getPath() + "/.verify.pmem." + uuidStr;
|
|
byte[] contents = uuidStr.getBytes("UTF-8");
|
|
byte[] contents = uuidStr.getBytes("UTF-8");
|
|
RandomAccessFile testFile = null;
|
|
RandomAccessFile testFile = null;
|
|
MappedByteBuffer out = null;
|
|
MappedByteBuffer out = null;
|
|
@@ -203,15 +283,17 @@ public class PmemVolumeManager {
|
|
out = testFile.getChannel().map(FileChannel.MapMode.READ_WRITE, 0,
|
|
out = testFile.getChannel().map(FileChannel.MapMode.READ_WRITE, 0,
|
|
contents.length);
|
|
contents.length);
|
|
if (out == null) {
|
|
if (out == null) {
|
|
- throw new IOException("Failed to map the test file under " + pmemDir);
|
|
|
|
|
|
+ throw new IOException(
|
|
|
|
+ "Failed to map the test file under " + realPmemDir);
|
|
}
|
|
}
|
|
out.put(contents);
|
|
out.put(contents);
|
|
// Forces to write data to storage device containing the mapped file
|
|
// Forces to write data to storage device containing the mapped file
|
|
out.force();
|
|
out.force();
|
|
|
|
+ return realPmemDir;
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
throw new IOException(
|
|
throw new IOException(
|
|
"Exception while writing data to persistent storage dir: " +
|
|
"Exception while writing data to persistent storage dir: " +
|
|
- pmemDir, e);
|
|
|
|
|
|
+ realPmemDir, e);
|
|
} finally {
|
|
} finally {
|
|
if (out != null) {
|
|
if (out != null) {
|
|
out.clear();
|
|
out.clear();
|
|
@@ -229,18 +311,38 @@ public class PmemVolumeManager {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public static String getRealPmemDir(String rawPmemDir) {
|
|
|
|
+ return new File(rawPmemDir, CACHE_DIR).getAbsolutePath();
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Choose a persistent memory volume based on a specific algorithm.
|
|
* Choose a persistent memory volume based on a specific algorithm.
|
|
* Currently it is a round-robin policy.
|
|
* Currently it is a round-robin policy.
|
|
*
|
|
*
|
|
* TODO: Refine volume selection policy by considering storage utilization.
|
|
* TODO: Refine volume selection policy by considering storage utilization.
|
|
*/
|
|
*/
|
|
- Byte getOneVolumeIndex() throws IOException {
|
|
|
|
- if (count != 0) {
|
|
|
|
- return (byte)(i++ % count);
|
|
|
|
- } else {
|
|
|
|
|
|
+ synchronized Byte chooseVolume(long bytesCount) throws IOException {
|
|
|
|
+ if (count == 0) {
|
|
throw new IOException("No usable persistent memory is found");
|
|
throw new IOException("No usable persistent memory is found");
|
|
}
|
|
}
|
|
|
|
+ int k = 0;
|
|
|
|
+ long maxAvailableSpace = 0L;
|
|
|
|
+ while (k++ != count) {
|
|
|
|
+ if (nextIndex == count) {
|
|
|
|
+ nextIndex = 0;
|
|
|
|
+ }
|
|
|
|
+ byte index = nextIndex++;
|
|
|
|
+ long availableBytes = usedBytesCounts.get(index).getAvailableBytes();
|
|
|
|
+ if (availableBytes >= bytesCount) {
|
|
|
|
+ return index;
|
|
|
|
+ }
|
|
|
|
+ if (availableBytes > maxAvailableSpace) {
|
|
|
|
+ maxAvailableSpace = availableBytes;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ throw new IOException("There is no enough persistent memory space " +
|
|
|
|
+ "for caching. The current max available space is " +
|
|
|
|
+ maxAvailableSpace + ", but " + bytesCount + "is required.");
|
|
}
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
@@ -276,7 +378,7 @@ public class PmemVolumeManager {
|
|
/**
|
|
/**
|
|
* The cache file path is pmemVolume/BlockPoolId-BlockId.
|
|
* The cache file path is pmemVolume/BlockPoolId-BlockId.
|
|
*/
|
|
*/
|
|
- public String getCacheFilePath(ExtendedBlockId key) {
|
|
|
|
|
|
+ public String getCachePath(ExtendedBlockId key) {
|
|
Byte volumeIndex = blockKeyToVolume.get(key);
|
|
Byte volumeIndex = blockKeyToVolume.get(key);
|
|
if (volumeIndex == null) {
|
|
if (volumeIndex == null) {
|
|
return null;
|
|
return null;
|
|
@@ -288,19 +390,4 @@ public class PmemVolumeManager {
|
|
Map<ExtendedBlockId, Byte> getBlockKeyToVolume() {
|
|
Map<ExtendedBlockId, Byte> getBlockKeyToVolume() {
|
|
return blockKeyToVolume;
|
|
return blockKeyToVolume;
|
|
}
|
|
}
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Add cached block's ExtendedBlockId and its cache volume index to a map
|
|
|
|
- * after cache.
|
|
|
|
- */
|
|
|
|
- public void afterCache(ExtendedBlockId key, Byte volumeIndex) {
|
|
|
|
- blockKeyToVolume.put(key, volumeIndex);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Remove the record in blockKeyToVolume for uncached block after uncache.
|
|
|
|
- */
|
|
|
|
- public void afterUncache(ExtendedBlockId key) {
|
|
|
|
- blockKeyToVolume.remove(key);
|
|
|
|
- }
|
|
|
|
}
|
|
}
|