|
@@ -30,7 +30,6 @@ import java.nio.channels.FileChannel;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
-import java.util.Collections;
|
|
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
@@ -88,6 +87,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
|
|
|
|
+import static org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
|
|
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
|
|
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
@@ -159,7 +159,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
@Override // FsDatasetSpi
|
|
@Override // FsDatasetSpi
|
|
public synchronized Block getStoredBlock(String bpid, long blkid)
|
|
public synchronized Block getStoredBlock(String bpid, long blkid)
|
|
throws IOException {
|
|
throws IOException {
|
|
- File blockfile = getFile(bpid, blkid);
|
|
|
|
|
|
+ File blockfile = getFile(bpid, blkid, false);
|
|
if (blockfile == null) {
|
|
if (blockfile == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -219,7 +219,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
private volatile boolean fsRunning;
|
|
private volatile boolean fsRunning;
|
|
|
|
|
|
final ReplicaMap volumeMap;
|
|
final ReplicaMap volumeMap;
|
|
- final LazyWriteReplicaTracker lazyWriteReplicaTracker;
|
|
|
|
|
|
+ final RamDiskReplicaTracker ramDiskReplicaTracker;
|
|
|
|
|
|
private static final int MAX_BLOCK_EVICTIONS_PER_ITERATION = 3;
|
|
private static final int MAX_BLOCK_EVICTIONS_PER_ITERATION = 3;
|
|
|
|
|
|
@@ -263,7 +263,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
storageMap = new ConcurrentHashMap<String, DatanodeStorage>();
|
|
storageMap = new ConcurrentHashMap<String, DatanodeStorage>();
|
|
volumeMap = new ReplicaMap(this);
|
|
volumeMap = new ReplicaMap(this);
|
|
- lazyWriteReplicaTracker = new LazyWriteReplicaTracker(this);
|
|
|
|
|
|
+ ramDiskReplicaTracker = RamDiskReplicaTracker.getInstance(conf, this);
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
final VolumeChoosingPolicy<FsVolumeImpl> blockChooserImpl =
|
|
final VolumeChoosingPolicy<FsVolumeImpl> blockChooserImpl =
|
|
@@ -298,7 +298,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
FsVolumeImpl fsVolume = FsVolumeImplAllocator.createVolume(
|
|
FsVolumeImpl fsVolume = FsVolumeImplAllocator.createVolume(
|
|
this, sd.getStorageUuid(), dir, this.conf, storageType);
|
|
this, sd.getStorageUuid(), dir, this.conf, storageType);
|
|
ReplicaMap tempVolumeMap = new ReplicaMap(this);
|
|
ReplicaMap tempVolumeMap = new ReplicaMap(this);
|
|
- fsVolume.getVolumeMap(volumeMap, lazyWriteReplicaTracker);
|
|
|
|
|
|
+ fsVolume.getVolumeMap(tempVolumeMap, ramDiskReplicaTracker);
|
|
|
|
|
|
volumeMap.addAll(tempVolumeMap);
|
|
volumeMap.addAll(tempVolumeMap);
|
|
volumes.addVolume(fsVolume);
|
|
volumes.addVolume(fsVolume);
|
|
@@ -326,7 +326,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
for (final String bpid : bpids) {
|
|
for (final String bpid : bpids) {
|
|
try {
|
|
try {
|
|
fsVolume.addBlockPool(bpid, this.conf);
|
|
fsVolume.addBlockPool(bpid, this.conf);
|
|
- fsVolume.getVolumeMap(bpid, tempVolumeMap);
|
|
|
|
|
|
+ fsVolume.getVolumeMap(bpid, tempVolumeMap, ramDiskReplicaTracker);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
LOG.warn("Caught exception when adding " + fsVolume +
|
|
LOG.warn("Caught exception when adding " + fsVolume +
|
|
". Will throw later.", e);
|
|
". Will throw later.", e);
|
|
@@ -586,12 +586,16 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
* checking that it exists. This should be used when the
|
|
* checking that it exists. This should be used when the
|
|
* next operation is going to open the file for read anyway,
|
|
* next operation is going to open the file for read anyway,
|
|
* and thus the exists check is redundant.
|
|
* and thus the exists check is redundant.
|
|
|
|
+ *
|
|
|
|
+ * @param touch if true then update the last access timestamp of the
|
|
|
|
+ * block. Currently used for blocks on transient storage.
|
|
*/
|
|
*/
|
|
- private File getBlockFileNoExistsCheck(ExtendedBlock b)
|
|
|
|
|
|
+ private File getBlockFileNoExistsCheck(ExtendedBlock b,
|
|
|
|
+ boolean touch)
|
|
throws IOException {
|
|
throws IOException {
|
|
final File f;
|
|
final File f;
|
|
synchronized(this) {
|
|
synchronized(this) {
|
|
- f = getFile(b.getBlockPoolId(), b.getLocalBlock().getBlockId());
|
|
|
|
|
|
+ f = getFile(b.getBlockPoolId(), b.getLocalBlock().getBlockId(), touch);
|
|
}
|
|
}
|
|
if (f == null) {
|
|
if (f == null) {
|
|
throw new IOException("Block " + b + " is not valid");
|
|
throw new IOException("Block " + b + " is not valid");
|
|
@@ -602,7 +606,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
@Override // FsDatasetSpi
|
|
@Override // FsDatasetSpi
|
|
public InputStream getBlockInputStream(ExtendedBlock b,
|
|
public InputStream getBlockInputStream(ExtendedBlock b,
|
|
long seekOffset) throws IOException {
|
|
long seekOffset) throws IOException {
|
|
- File blockFile = getBlockFileNoExistsCheck(b);
|
|
|
|
|
|
+ File blockFile = getBlockFileNoExistsCheck(b, true);
|
|
if (isNativeIOAvailable) {
|
|
if (isNativeIOAvailable) {
|
|
return NativeIO.getShareDeleteFileInputStream(blockFile, seekOffset);
|
|
return NativeIO.getShareDeleteFileInputStream(blockFile, seekOffset);
|
|
} else {
|
|
} else {
|
|
@@ -1240,7 +1244,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
newReplicaInfo = new FinalizedReplica(replicaInfo, v, dest.getParentFile());
|
|
newReplicaInfo = new FinalizedReplica(replicaInfo, v, dest.getParentFile());
|
|
|
|
|
|
if (v.isTransientStorage()) {
|
|
if (v.isTransientStorage()) {
|
|
- lazyWriteReplicaTracker.addReplica(bpid, replicaInfo.getBlockId(), v);
|
|
|
|
|
|
+ ramDiskReplicaTracker.addReplica(bpid, replicaInfo.getBlockId(), v);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
volumeMap.add(bpid, newReplicaInfo);
|
|
volumeMap.add(bpid, newReplicaInfo);
|
|
@@ -1265,7 +1269,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
LOG.warn("Block " + b + " unfinalized and removed. " );
|
|
LOG.warn("Block " + b + " unfinalized and removed. " );
|
|
}
|
|
}
|
|
if (replicaInfo.getVolume().isTransientStorage()) {
|
|
if (replicaInfo.getVolume().isTransientStorage()) {
|
|
- lazyWriteReplicaTracker.discardReplica(b.getBlockPoolId(), b.getBlockId(), true);
|
|
|
|
|
|
+ ramDiskReplicaTracker.discardReplica(b.getBlockPoolId(), b.getBlockId(), true);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1411,7 +1415,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
//Should we check for metadata file too?
|
|
//Should we check for metadata file too?
|
|
final File f;
|
|
final File f;
|
|
synchronized(this) {
|
|
synchronized(this) {
|
|
- f = getFile(bpid, blockId);
|
|
|
|
|
|
+ f = getFile(bpid, blockId, false);
|
|
}
|
|
}
|
|
|
|
|
|
if(f != null ) {
|
|
if(f != null ) {
|
|
@@ -1496,7 +1500,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
}
|
|
}
|
|
|
|
|
|
if (v.isTransientStorage()) {
|
|
if (v.isTransientStorage()) {
|
|
- lazyWriteReplicaTracker.discardReplica(bpid, invalidBlks[i].getBlockId(), true);
|
|
|
|
|
|
+ ramDiskReplicaTracker.discardReplica(bpid, invalidBlks[i].getBlockId(), true);
|
|
}
|
|
}
|
|
|
|
|
|
// If a DFSClient has the replica in its cache of short-circuit file
|
|
// If a DFSClient has the replica in its cache of short-circuit file
|
|
@@ -1628,7 +1632,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
@Override // FsDatasetSpi
|
|
@Override // FsDatasetSpi
|
|
public synchronized boolean contains(final ExtendedBlock block) {
|
|
public synchronized boolean contains(final ExtendedBlock block) {
|
|
final long blockId = block.getLocalBlock().getBlockId();
|
|
final long blockId = block.getLocalBlock().getBlockId();
|
|
- return getFile(block.getBlockPoolId(), blockId) != null;
|
|
|
|
|
|
+ return getFile(block.getBlockPoolId(), blockId, false) != null;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1637,9 +1641,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
* @param blockId a block's id
|
|
* @param blockId a block's id
|
|
* @return on disk data file path; null if the replica does not exist
|
|
* @return on disk data file path; null if the replica does not exist
|
|
*/
|
|
*/
|
|
- File getFile(final String bpid, final long blockId) {
|
|
|
|
|
|
+ File getFile(final String bpid, final long blockId, boolean touch) {
|
|
ReplicaInfo info = volumeMap.get(bpid, blockId);
|
|
ReplicaInfo info = volumeMap.get(bpid, blockId);
|
|
if (info != null) {
|
|
if (info != null) {
|
|
|
|
+ if (touch && info.getVolume().isTransientStorage()) {
|
|
|
|
+ ramDiskReplicaTracker.touch(bpid, blockId);
|
|
|
|
+ }
|
|
return info.getBlockFile();
|
|
return info.getBlockFile();
|
|
}
|
|
}
|
|
return null;
|
|
return null;
|
|
@@ -1808,7 +1815,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
blockScanner.deleteBlock(bpid, new Block(blockId));
|
|
blockScanner.deleteBlock(bpid, new Block(blockId));
|
|
}
|
|
}
|
|
if (vol.isTransientStorage()) {
|
|
if (vol.isTransientStorage()) {
|
|
- lazyWriteReplicaTracker.discardReplica(bpid, blockId, true);
|
|
|
|
|
|
+ ramDiskReplicaTracker.discardReplica(bpid, blockId, true);
|
|
}
|
|
}
|
|
LOG.warn("Removed block " + blockId
|
|
LOG.warn("Removed block " + blockId
|
|
+ " from memory with missing block file on the disk");
|
|
+ " from memory with missing block file on the disk");
|
|
@@ -1830,11 +1837,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
diskFile.length(), diskGS, vol, diskFile.getParentFile());
|
|
diskFile.length(), diskGS, vol, diskFile.getParentFile());
|
|
volumeMap.add(bpid, diskBlockInfo);
|
|
volumeMap.add(bpid, diskBlockInfo);
|
|
final DataBlockScanner blockScanner = datanode.getBlockScanner();
|
|
final DataBlockScanner blockScanner = datanode.getBlockScanner();
|
|
- if (blockScanner != null) {
|
|
|
|
- blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo));
|
|
|
|
- }
|
|
|
|
- if (vol.isTransientStorage()) {
|
|
|
|
- lazyWriteReplicaTracker.addReplica(bpid, blockId, (FsVolumeImpl) vol);
|
|
|
|
|
|
+ if (!vol.isTransientStorage()) {
|
|
|
|
+ if (blockScanner != null) {
|
|
|
|
+ blockScanner.addBlock(new ExtendedBlock(bpid, diskBlockInfo));
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ ramDiskReplicaTracker.addReplica(bpid, blockId, (FsVolumeImpl) vol);
|
|
}
|
|
}
|
|
LOG.warn("Added missing block to memory " + diskBlockInfo);
|
|
LOG.warn("Added missing block to memory " + diskBlockInfo);
|
|
return;
|
|
return;
|
|
@@ -2117,7 +2125,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
volumes.addBlockPool(bpid, conf);
|
|
volumes.addBlockPool(bpid, conf);
|
|
volumeMap.initBlockPool(bpid);
|
|
volumeMap.initBlockPool(bpid);
|
|
}
|
|
}
|
|
- volumes.getAllVolumesMap(bpid, volumeMap, lazyWriteReplicaTracker);
|
|
|
|
|
|
+ volumes.getAllVolumesMap(bpid, volumeMap, ramDiskReplicaTracker);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -2347,7 +2355,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
LOG.debug("LazyWriter starting to save blockId=" + blockId + "; bpid=" + bpid);
|
|
LOG.debug("LazyWriter starting to save blockId=" + blockId + "; bpid=" + bpid);
|
|
}
|
|
}
|
|
|
|
|
|
- lazyWriteReplicaTracker.recordStartLazyPersist(bpid, blockId, targetVolume);
|
|
|
|
|
|
+ ramDiskReplicaTracker.recordStartLazyPersist(bpid, blockId, targetVolume);
|
|
bpSlice = targetVolume.getBlockPoolSlice(bpid);
|
|
bpSlice = targetVolume.getBlockPoolSlice(bpid);
|
|
srcMeta = replicaInfo.getMetaFile();
|
|
srcMeta = replicaInfo.getMetaFile();
|
|
srcFile = replicaInfo.getBlockFile();
|
|
srcFile = replicaInfo.getBlockFile();
|
|
@@ -2359,7 +2367,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
bpSlice.lazyPersistReplica(blockId, genStamp, srcMeta, srcFile);
|
|
bpSlice.lazyPersistReplica(blockId, genStamp, srcMeta, srcFile);
|
|
|
|
|
|
synchronized (FsDatasetImpl.this) {
|
|
synchronized (FsDatasetImpl.this) {
|
|
- lazyWriteReplicaTracker.recordEndLazyPersist(bpid, blockId, savedFiles);
|
|
|
|
|
|
+ ramDiskReplicaTracker.recordEndLazyPersist(bpid, blockId, savedFiles);
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("LazyWriter finished saving blockId=" + blockId + "; bpid=" + bpid +
|
|
LOG.debug("LazyWriter finished saving blockId=" + blockId + "; bpid=" + bpid +
|
|
@@ -2374,21 +2382,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
* @return true if there is more work to be done, false otherwise.
|
|
* @return true if there is more work to be done, false otherwise.
|
|
*/
|
|
*/
|
|
private boolean saveNextReplica() {
|
|
private boolean saveNextReplica() {
|
|
- LazyWriteReplicaTracker.ReplicaState replicaState = null;
|
|
|
|
|
|
+ RamDiskReplica block = null;
|
|
boolean succeeded = false;
|
|
boolean succeeded = false;
|
|
|
|
|
|
try {
|
|
try {
|
|
- replicaState = lazyWriteReplicaTracker.dequeueNextReplicaToPersist();
|
|
|
|
- if (replicaState != null) {
|
|
|
|
- moveReplicaToNewVolume(replicaState.bpid, replicaState.blockId);
|
|
|
|
|
|
+ block = ramDiskReplicaTracker.dequeueNextReplicaToPersist();
|
|
|
|
+ if (block != null) {
|
|
|
|
+ moveReplicaToNewVolume(block.getBlockPoolId(), block.getBlockId());
|
|
}
|
|
}
|
|
succeeded = true;
|
|
succeeded = true;
|
|
} catch(IOException ioe) {
|
|
} catch(IOException ioe) {
|
|
- LOG.warn("Exception saving replica " + replicaState, ioe);
|
|
|
|
|
|
+ LOG.warn("Exception saving replica " + block, ioe);
|
|
} finally {
|
|
} finally {
|
|
- if (!succeeded && replicaState != null) {
|
|
|
|
- LOG.warn("Failed to save replica " + replicaState + ". re-enqueueing it.");
|
|
|
|
- lazyWriteReplicaTracker.reenqueueReplicaNotPersisted(replicaState);
|
|
|
|
|
|
+ if (!succeeded && block != null) {
|
|
|
|
+ LOG.warn("Failed to save replica " + block + ". re-enqueueing it.");
|
|
|
|
+ ramDiskReplicaTracker.reenqueueReplicaNotPersisted(block);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2426,8 +2434,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
while (iterations++ < MAX_BLOCK_EVICTIONS_PER_ITERATION &&
|
|
while (iterations++ < MAX_BLOCK_EVICTIONS_PER_ITERATION &&
|
|
transientFreeSpaceBelowThreshold()) {
|
|
transientFreeSpaceBelowThreshold()) {
|
|
- LazyWriteReplicaTracker.ReplicaState replicaState =
|
|
|
|
- lazyWriteReplicaTracker.getNextCandidateForEviction();
|
|
|
|
|
|
+ RamDiskReplica replicaState = ramDiskReplicaTracker.getNextCandidateForEviction();
|
|
|
|
|
|
if (replicaState == null) {
|
|
if (replicaState == null) {
|
|
break;
|
|
break;
|
|
@@ -2440,46 +2447,48 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
ReplicaInfo replicaInfo, newReplicaInfo;
|
|
ReplicaInfo replicaInfo, newReplicaInfo;
|
|
File blockFile, metaFile;
|
|
File blockFile, metaFile;
|
|
long blockFileUsed, metaFileUsed;
|
|
long blockFileUsed, metaFileUsed;
|
|
|
|
+ final String bpid = replicaState.getBlockPoolId();
|
|
|
|
|
|
synchronized (FsDatasetImpl.this) {
|
|
synchronized (FsDatasetImpl.this) {
|
|
- replicaInfo = getReplicaInfo(replicaState.bpid, replicaState.blockId);
|
|
|
|
|
|
+ replicaInfo = getReplicaInfo(replicaState.getBlockPoolId(), replicaState.getBlockId());
|
|
Preconditions.checkState(replicaInfo.getVolume().isTransientStorage());
|
|
Preconditions.checkState(replicaInfo.getVolume().isTransientStorage());
|
|
blockFile = replicaInfo.getBlockFile();
|
|
blockFile = replicaInfo.getBlockFile();
|
|
metaFile = replicaInfo.getMetaFile();
|
|
metaFile = replicaInfo.getMetaFile();
|
|
blockFileUsed = blockFile.length();
|
|
blockFileUsed = blockFile.length();
|
|
metaFileUsed = metaFile.length();
|
|
metaFileUsed = metaFile.length();
|
|
- lazyWriteReplicaTracker.discardReplica(replicaState, false);
|
|
|
|
|
|
+ ramDiskReplicaTracker.discardReplica(replicaState, false);
|
|
|
|
|
|
// Move the replica from lazyPersist/ to finalized/ on target volume
|
|
// Move the replica from lazyPersist/ to finalized/ on target volume
|
|
BlockPoolSlice bpSlice =
|
|
BlockPoolSlice bpSlice =
|
|
- replicaState.lazyPersistVolume.getBlockPoolSlice(replicaState.bpid);
|
|
|
|
|
|
+ replicaState.getLazyPersistVolume().getBlockPoolSlice(bpid);
|
|
File newBlockFile = bpSlice.activateSavedReplica(
|
|
File newBlockFile = bpSlice.activateSavedReplica(
|
|
- replicaInfo, replicaState.savedBlockFile);
|
|
|
|
|
|
+ replicaInfo, replicaState.getSavedMetaFile(),
|
|
|
|
+ replicaState.getSavedBlockFile());
|
|
|
|
|
|
newReplicaInfo =
|
|
newReplicaInfo =
|
|
new FinalizedReplica(replicaInfo.getBlockId(),
|
|
new FinalizedReplica(replicaInfo.getBlockId(),
|
|
replicaInfo.getBytesOnDisk(),
|
|
replicaInfo.getBytesOnDisk(),
|
|
replicaInfo.getGenerationStamp(),
|
|
replicaInfo.getGenerationStamp(),
|
|
- replicaState.lazyPersistVolume,
|
|
|
|
|
|
+ replicaState.getLazyPersistVolume(),
|
|
newBlockFile.getParentFile());
|
|
newBlockFile.getParentFile());
|
|
|
|
|
|
// Update the volumeMap entry.
|
|
// Update the volumeMap entry.
|
|
- volumeMap.add(replicaState.bpid, newReplicaInfo);
|
|
|
|
|
|
+ volumeMap.add(bpid, newReplicaInfo);
|
|
}
|
|
}
|
|
|
|
|
|
// Before deleting the files from transient storage we must notify the
|
|
// Before deleting the files from transient storage we must notify the
|
|
// NN that the files are on the new storage. Else a blockReport from
|
|
// NN that the files are on the new storage. Else a blockReport from
|
|
// the transient storage might cause the NN to think the blocks are lost.
|
|
// the transient storage might cause the NN to think the blocks are lost.
|
|
ExtendedBlock extendedBlock =
|
|
ExtendedBlock extendedBlock =
|
|
- new ExtendedBlock(replicaState.bpid, newReplicaInfo);
|
|
|
|
|
|
+ new ExtendedBlock(bpid, newReplicaInfo);
|
|
datanode.notifyNamenodeReceivedBlock(
|
|
datanode.notifyNamenodeReceivedBlock(
|
|
extendedBlock, null, newReplicaInfo.getStorageUuid());
|
|
extendedBlock, null, newReplicaInfo.getStorageUuid());
|
|
|
|
|
|
// Remove the old replicas from transient storage.
|
|
// Remove the old replicas from transient storage.
|
|
if (blockFile.delete() || !blockFile.exists()) {
|
|
if (blockFile.delete() || !blockFile.exists()) {
|
|
- ((FsVolumeImpl) replicaInfo.getVolume()).decDfsUsed(replicaState.bpid, blockFileUsed);
|
|
|
|
|
|
+ ((FsVolumeImpl) replicaInfo.getVolume()).decDfsUsed(bpid, blockFileUsed);
|
|
if (metaFile.delete() || !metaFile.exists()) {
|
|
if (metaFile.delete() || !metaFile.exists()) {
|
|
- ((FsVolumeImpl) replicaInfo.getVolume()).decDfsUsed(replicaState.bpid, metaFileUsed);
|
|
|
|
|
|
+ ((FsVolumeImpl) replicaInfo.getVolume()).decDfsUsed(bpid, metaFileUsed);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2500,7 +2509,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
// Sleep if we have no more work to do or if it looks like we are not
|
|
// Sleep if we have no more work to do or if it looks like we are not
|
|
// making any forward progress. This is to ensure that if all persist
|
|
// making any forward progress. This is to ensure that if all persist
|
|
// operations are failing we don't keep retrying them in a tight loop.
|
|
// operations are failing we don't keep retrying them in a tight loop.
|
|
- if (numSuccessiveFailures >= lazyWriteReplicaTracker.numReplicasNotPersisted()) {
|
|
|
|
|
|
+ if (numSuccessiveFailures >= ramDiskReplicaTracker.numReplicasNotPersisted()) {
|
|
Thread.sleep(checkpointerInterval * 1000);
|
|
Thread.sleep(checkpointerInterval * 1000);
|
|
numSuccessiveFailures = 0;
|
|
numSuccessiveFailures = 0;
|
|
}
|
|
}
|