|
@@ -37,12 +37,12 @@ import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
import org.apache.commons.io.IOUtils;
|
|
|
-import org.apache.commons.lang.builder.HashCodeBuilder;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.fs.ChecksumException;
|
|
|
+import org.apache.hadoop.hdfs.ExtendedBlockId;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
@@ -56,43 +56,6 @@ import org.apache.hadoop.io.nativeio.NativeIO;
|
|
|
@InterfaceAudience.Private
|
|
|
@InterfaceStability.Unstable
|
|
|
public class FsDatasetCache {
|
|
|
- /**
|
|
|
- * Keys which identify MappableBlocks.
|
|
|
- */
|
|
|
- private static final class Key {
|
|
|
- /**
|
|
|
- * Block id.
|
|
|
- */
|
|
|
- final long id;
|
|
|
-
|
|
|
- /**
|
|
|
- * Block pool id.
|
|
|
- */
|
|
|
- final String bpid;
|
|
|
-
|
|
|
- Key(long id, String bpid) {
|
|
|
- this.id = id;
|
|
|
- this.bpid = bpid;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public boolean equals(Object o) {
|
|
|
- if (o == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- if (!(o.getClass() == getClass())) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- Key other = (Key)o;
|
|
|
- return ((other.id == this.id) && (other.bpid.equals(this.bpid)));
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int hashCode() {
|
|
|
- return new HashCodeBuilder().append(id).append(bpid).hashCode();
|
|
|
- }
|
|
|
- };
|
|
|
-
|
|
|
/**
|
|
|
* MappableBlocks that we know about.
|
|
|
*/
|
|
@@ -143,7 +106,8 @@ public class FsDatasetCache {
|
|
|
/**
|
|
|
* Stores MappableBlock objects and the states they're in.
|
|
|
*/
|
|
|
- private final HashMap<Key, Value> mappableBlockMap = new HashMap<Key, Value>();
|
|
|
+ private final HashMap<ExtendedBlockId, Value> mappableBlockMap =
|
|
|
+ new HashMap<ExtendedBlockId, Value>();
|
|
|
|
|
|
private final AtomicLong numBlocksCached = new AtomicLong(0);
|
|
|
|
|
@@ -260,12 +224,12 @@ public class FsDatasetCache {
|
|
|
*/
|
|
|
synchronized List<Long> getCachedBlocks(String bpid) {
|
|
|
List<Long> blocks = new ArrayList<Long>();
|
|
|
- for (Iterator<Entry<Key, Value>> iter =
|
|
|
+ for (Iterator<Entry<ExtendedBlockId, Value>> iter =
|
|
|
mappableBlockMap.entrySet().iterator(); iter.hasNext(); ) {
|
|
|
- Entry<Key, Value> entry = iter.next();
|
|
|
- if (entry.getKey().bpid.equals(bpid)) {
|
|
|
+ Entry<ExtendedBlockId, Value> entry = iter.next();
|
|
|
+ if (entry.getKey().getBlockPoolId().equals(bpid)) {
|
|
|
if (entry.getValue().state.shouldAdvertise()) {
|
|
|
- blocks.add(entry.getKey().id);
|
|
|
+ blocks.add(entry.getKey().getBlockId());
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -278,7 +242,7 @@ public class FsDatasetCache {
|
|
|
synchronized void cacheBlock(long blockId, String bpid,
|
|
|
String blockFileName, long length, long genstamp,
|
|
|
Executor volumeExecutor) {
|
|
|
- Key key = new Key(blockId, bpid);
|
|
|
+ ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
|
|
|
Value prevValue = mappableBlockMap.get(key);
|
|
|
if (prevValue != null) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -299,7 +263,7 @@ public class FsDatasetCache {
|
|
|
}
|
|
|
|
|
|
synchronized void uncacheBlock(String bpid, long blockId) {
|
|
|
- Key key = new Key(blockId, bpid);
|
|
|
+ ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
|
|
|
Value prevValue = mappableBlockMap.get(key);
|
|
|
|
|
|
if (prevValue == null) {
|
|
@@ -344,12 +308,12 @@ public class FsDatasetCache {
|
|
|
* Background worker that mmaps, mlocks, and checksums a block
|
|
|
*/
|
|
|
private class CachingTask implements Runnable {
|
|
|
- private final Key key;
|
|
|
+ private final ExtendedBlockId key;
|
|
|
private final String blockFileName;
|
|
|
private final long length;
|
|
|
private final long genstamp;
|
|
|
|
|
|
- CachingTask(Key key, String blockFileName, long length, long genstamp) {
|
|
|
+ CachingTask(ExtendedBlockId key, String blockFileName, long length, long genstamp) {
|
|
|
this.key = key;
|
|
|
this.blockFileName = blockFileName;
|
|
|
this.length = length;
|
|
@@ -361,13 +325,13 @@ public class FsDatasetCache {
|
|
|
boolean success = false;
|
|
|
FileInputStream blockIn = null, metaIn = null;
|
|
|
MappableBlock mappableBlock = null;
|
|
|
- ExtendedBlock extBlk =
|
|
|
- new ExtendedBlock(key.bpid, key.id, length, genstamp);
|
|
|
+ ExtendedBlock extBlk = new ExtendedBlock(key.getBlockPoolId(),
|
|
|
+ key.getBlockId(), length, genstamp);
|
|
|
long newUsedBytes = usedBytesCount.reserve(length);
|
|
|
if (newUsedBytes < 0) {
|
|
|
- LOG.warn("Failed to cache block id " + key.id + ", pool " + key.bpid +
|
|
|
- ": could not reserve " + length + " more bytes in the " +
|
|
|
- "cache: " + DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY +
|
|
|
+ LOG.warn("Failed to cache " + key + ": could not reserve " + length +
|
|
|
+ " more bytes in the cache: " +
|
|
|
+ DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY +
|
|
|
" of " + maxBytes + " exceeded.");
|
|
|
numBlocksFailedToCache.incrementAndGet();
|
|
|
return;
|
|
@@ -378,16 +342,15 @@ public class FsDatasetCache {
|
|
|
metaIn = (FileInputStream)dataset.getMetaDataInputStream(extBlk)
|
|
|
.getWrappedStream();
|
|
|
} catch (ClassCastException e) {
|
|
|
- LOG.warn("Failed to cache block with id " + key.id + ", pool " +
|
|
|
- key.bpid + ": Underlying blocks are not backed by files.", e);
|
|
|
+ LOG.warn("Failed to cache " + key +
|
|
|
+ ": Underlying blocks are not backed by files.", e);
|
|
|
return;
|
|
|
} catch (FileNotFoundException e) {
|
|
|
- LOG.info("Failed to cache block with id " + key.id + ", pool " +
|
|
|
- key.bpid + ": failed to find backing files.");
|
|
|
+ LOG.info("Failed to cache " + key + ": failed to find backing " +
|
|
|
+ "files.");
|
|
|
return;
|
|
|
} catch (IOException e) {
|
|
|
- LOG.warn("Failed to cache block with id " + key.id + ", pool " +
|
|
|
- key.bpid + ": failed to open file", e);
|
|
|
+ LOG.warn("Failed to cache " + key + ": failed to open file", e);
|
|
|
return;
|
|
|
}
|
|
|
try {
|
|
@@ -395,11 +358,10 @@ public class FsDatasetCache {
|
|
|
load(length, blockIn, metaIn, blockFileName);
|
|
|
} catch (ChecksumException e) {
|
|
|
// Exception message is bogus since this wasn't caused by a file read
|
|
|
- LOG.warn("Failed to cache block " + key.id + " in " + key.bpid + ": " +
|
|
|
- "checksum verification failed.");
|
|
|
+ LOG.warn("Failed to cache " + key + ": checksum verification failed.");
|
|
|
return;
|
|
|
} catch (IOException e) {
|
|
|
- LOG.warn("Failed to cache block " + key.id + " in " + key.bpid, e);
|
|
|
+ LOG.warn("Failed to cache " + key, e);
|
|
|
return;
|
|
|
}
|
|
|
synchronized (FsDatasetCache.this) {
|
|
@@ -409,15 +371,14 @@ public class FsDatasetCache {
|
|
|
value.state == State.CACHING_CANCELLED);
|
|
|
if (value.state == State.CACHING_CANCELLED) {
|
|
|
mappableBlockMap.remove(key);
|
|
|
- LOG.warn("Caching of block " + key.id + " in " + key.bpid +
|
|
|
- " was cancelled.");
|
|
|
+ LOG.warn("Caching of " + key + " was cancelled.");
|
|
|
return;
|
|
|
}
|
|
|
mappableBlockMap.put(key, new Value(mappableBlock, State.CACHED));
|
|
|
}
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Successfully cached block " + key.id + " in " + key.bpid +
|
|
|
- ". We are now caching " + newUsedBytes + " bytes in total.");
|
|
|
+ LOG.debug("Successfully cached " + key + ". We are now caching " +
|
|
|
+ newUsedBytes + " bytes in total.");
|
|
|
}
|
|
|
numBlocksCached.addAndGet(1);
|
|
|
success = true;
|
|
@@ -425,9 +386,8 @@ public class FsDatasetCache {
|
|
|
if (!success) {
|
|
|
newUsedBytes = usedBytesCount.release(length);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Caching of block " + key.id + " in " +
|
|
|
- key.bpid + " was aborted. We are now caching only " +
|
|
|
- newUsedBytes + " + bytes in total.");
|
|
|
+ LOG.debug("Caching of " + key + " was aborted. We are now " +
|
|
|
+ "caching only " + newUsedBytes + " + bytes in total.");
|
|
|
}
|
|
|
IOUtils.closeQuietly(blockIn);
|
|
|
IOUtils.closeQuietly(metaIn);
|
|
@@ -445,9 +405,9 @@ public class FsDatasetCache {
|
|
|
}
|
|
|
|
|
|
private class UncachingTask implements Runnable {
|
|
|
- private final Key key;
|
|
|
+ private final ExtendedBlockId key;
|
|
|
|
|
|
- UncachingTask(Key key) {
|
|
|
+ UncachingTask(ExtendedBlockId key) {
|
|
|
this.key = key;
|
|
|
}
|
|
|
|
|
@@ -470,8 +430,8 @@ public class FsDatasetCache {
|
|
|
usedBytesCount.release(value.mappableBlock.getLength());
|
|
|
numBlocksCached.addAndGet(-1);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Uncaching of block " + key.id + " in " + key.bpid +
|
|
|
- " completed. usedBytes = " + newUsedBytes);
|
|
|
+ LOG.debug("Uncaching of " + key + " completed. " +
|
|
|
+ "usedBytes = " + newUsedBytes);
|
|
|
}
|
|
|
}
|
|
|
}
|