|
@@ -361,20 +361,22 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
*/
|
|
*/
|
|
private static List<VolumeFailureInfo> getInitialVolumeFailureInfos(
|
|
private static List<VolumeFailureInfo> getInitialVolumeFailureInfos(
|
|
Collection<StorageLocation> dataLocations, DataStorage storage) {
|
|
Collection<StorageLocation> dataLocations, DataStorage storage) {
|
|
- Set<String> failedLocationSet = Sets.newHashSetWithExpectedSize(
|
|
|
|
|
|
+ Set<StorageLocation> failedLocationSet = Sets.newHashSetWithExpectedSize(
|
|
dataLocations.size());
|
|
dataLocations.size());
|
|
for (StorageLocation sl: dataLocations) {
|
|
for (StorageLocation sl: dataLocations) {
|
|
- failedLocationSet.add(sl.getFile().getAbsolutePath());
|
|
|
|
|
|
+ LOG.info("Adding to failedLocationSet " + sl);
|
|
|
|
+ failedLocationSet.add(sl);
|
|
}
|
|
}
|
|
for (Iterator<Storage.StorageDirectory> it = storage.dirIterator();
|
|
for (Iterator<Storage.StorageDirectory> it = storage.dirIterator();
|
|
it.hasNext(); ) {
|
|
it.hasNext(); ) {
|
|
Storage.StorageDirectory sd = it.next();
|
|
Storage.StorageDirectory sd = it.next();
|
|
- failedLocationSet.remove(sd.getRoot().getAbsolutePath());
|
|
|
|
|
|
+ failedLocationSet.remove(sd.getStorageLocation());
|
|
|
|
+ LOG.info("Removing from failedLocationSet " + sd.getStorageLocation());
|
|
}
|
|
}
|
|
List<VolumeFailureInfo> volumeFailureInfos = Lists.newArrayListWithCapacity(
|
|
List<VolumeFailureInfo> volumeFailureInfos = Lists.newArrayListWithCapacity(
|
|
failedLocationSet.size());
|
|
failedLocationSet.size());
|
|
long failureDate = Time.now();
|
|
long failureDate = Time.now();
|
|
- for (String failedStorageLocation: failedLocationSet) {
|
|
|
|
|
|
+ for (StorageLocation failedStorageLocation: failedLocationSet) {
|
|
volumeFailureInfos.add(new VolumeFailureInfo(failedStorageLocation,
|
|
volumeFailureInfos.add(new VolumeFailureInfo(failedStorageLocation,
|
|
failureDate));
|
|
failureDate));
|
|
}
|
|
}
|
|
@@ -403,49 +405,55 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
new DatanodeStorage(sd.getStorageUuid(),
|
|
new DatanodeStorage(sd.getStorageUuid(),
|
|
DatanodeStorage.State.NORMAL,
|
|
DatanodeStorage.State.NORMAL,
|
|
storageType));
|
|
storageType));
|
|
- asyncDiskService.addVolume(sd.getCurrentDir());
|
|
|
|
|
|
+ asyncDiskService.addVolume((FsVolumeImpl) ref.getVolume());
|
|
volumes.addVolume(ref);
|
|
volumes.addVolume(ref);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
private void addVolume(Collection<StorageLocation> dataLocations,
|
|
private void addVolume(Collection<StorageLocation> dataLocations,
|
|
Storage.StorageDirectory sd) throws IOException {
|
|
Storage.StorageDirectory sd) throws IOException {
|
|
- final File dir = sd.getCurrentDir();
|
|
|
|
- final StorageType storageType =
|
|
|
|
- getStorageTypeFromLocations(dataLocations, sd.getRoot());
|
|
|
|
|
|
+ final StorageLocation storageLocation = sd.getStorageLocation();
|
|
|
|
|
|
// If IOException raises from FsVolumeImpl() or getVolumeMap(), there is
|
|
// If IOException raises from FsVolumeImpl() or getVolumeMap(), there is
|
|
// nothing needed to be rolled back to make various data structures, e.g.,
|
|
// nothing needed to be rolled back to make various data structures, e.g.,
|
|
// storageMap and asyncDiskService, consistent.
|
|
// storageMap and asyncDiskService, consistent.
|
|
- FsVolumeImpl fsVolume = new FsVolumeImpl(
|
|
|
|
- this, sd.getStorageUuid(), dir, this.conf, storageType);
|
|
|
|
|
|
+ FsVolumeImpl fsVolume = new FsVolumeImplBuilder()
|
|
|
|
+ .setDataset(this)
|
|
|
|
+ .setStorageID(sd.getStorageUuid())
|
|
|
|
+ .setStorageDirectory(sd)
|
|
|
|
+ .setConf(this.conf)
|
|
|
|
+ .build();
|
|
FsVolumeReference ref = fsVolume.obtainReference();
|
|
FsVolumeReference ref = fsVolume.obtainReference();
|
|
ReplicaMap tempVolumeMap = new ReplicaMap(datasetLock);
|
|
ReplicaMap tempVolumeMap = new ReplicaMap(datasetLock);
|
|
fsVolume.getVolumeMap(tempVolumeMap, ramDiskReplicaTracker);
|
|
fsVolume.getVolumeMap(tempVolumeMap, ramDiskReplicaTracker);
|
|
|
|
|
|
- activateVolume(tempVolumeMap, sd, storageType, ref);
|
|
|
|
- LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
|
|
|
|
|
|
+ activateVolume(tempVolumeMap, sd, storageLocation.getStorageType(), ref);
|
|
|
|
+ LOG.info("Added volume - " + storageLocation + ", StorageType: " +
|
|
|
|
+ storageLocation.getStorageType());
|
|
}
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
- public FsVolumeImpl createFsVolume(String storageUuid, File currentDir,
|
|
|
|
- StorageType storageType) throws IOException {
|
|
|
|
- return new FsVolumeImpl(this, storageUuid, currentDir, conf, storageType);
|
|
|
|
|
|
+ public FsVolumeImpl createFsVolume(String storageUuid,
|
|
|
|
+ Storage.StorageDirectory sd,
|
|
|
|
+ final StorageLocation location) throws IOException {
|
|
|
|
+ return new FsVolumeImplBuilder()
|
|
|
|
+ .setDataset(this)
|
|
|
|
+ .setStorageID(storageUuid)
|
|
|
|
+ .setStorageDirectory(sd)
|
|
|
|
+ .setConf(conf)
|
|
|
|
+ .build();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void addVolume(final StorageLocation location,
|
|
public void addVolume(final StorageLocation location,
|
|
final List<NamespaceInfo> nsInfos)
|
|
final List<NamespaceInfo> nsInfos)
|
|
throws IOException {
|
|
throws IOException {
|
|
- final File dir = location.getFile();
|
|
|
|
-
|
|
|
|
// Prepare volume in DataStorage
|
|
// Prepare volume in DataStorage
|
|
final DataStorage.VolumeBuilder builder;
|
|
final DataStorage.VolumeBuilder builder;
|
|
try {
|
|
try {
|
|
- builder = dataStorage.prepareVolume(datanode, location.getFile(), nsInfos);
|
|
|
|
|
|
+ builder = dataStorage.prepareVolume(datanode, location, nsInfos);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
- volumes.addVolumeFailureInfo(new VolumeFailureInfo(
|
|
|
|
- location.getFile().getAbsolutePath(), Time.now()));
|
|
|
|
|
|
+ volumes.addVolumeFailureInfo(new VolumeFailureInfo(location, Time.now()));
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -453,7 +461,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
StorageType storageType = location.getStorageType();
|
|
StorageType storageType = location.getStorageType();
|
|
final FsVolumeImpl fsVolume =
|
|
final FsVolumeImpl fsVolume =
|
|
- createFsVolume(sd.getStorageUuid(), sd.getCurrentDir(), storageType);
|
|
|
|
|
|
+ createFsVolume(sd.getStorageUuid(), sd, location);
|
|
final ReplicaMap tempVolumeMap = new ReplicaMap(new AutoCloseableLock());
|
|
final ReplicaMap tempVolumeMap = new ReplicaMap(new AutoCloseableLock());
|
|
ArrayList<IOException> exceptions = Lists.newArrayList();
|
|
ArrayList<IOException> exceptions = Lists.newArrayList();
|
|
|
|
|
|
@@ -482,34 +490,33 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
builder.build();
|
|
builder.build();
|
|
activateVolume(tempVolumeMap, sd, storageType, ref);
|
|
activateVolume(tempVolumeMap, sd, storageType, ref);
|
|
- LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
|
|
|
|
|
|
+ LOG.info("Added volume - " + location + ", StorageType: " + storageType);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Removes a set of volumes from FsDataset.
|
|
* Removes a set of volumes from FsDataset.
|
|
- * @param volumesToRemove a set of absolute root path of each volume.
|
|
|
|
|
|
+ * @param storageLocationsToRemove a set of
|
|
|
|
+ * {@link StorageLocation}s for each volume.
|
|
* @param clearFailure set true to clear failure information.
|
|
* @param clearFailure set true to clear failure information.
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- public void removeVolumes(Set<File> volumesToRemove, boolean clearFailure) {
|
|
|
|
- // Make sure that all volumes are absolute path.
|
|
|
|
- for (File vol : volumesToRemove) {
|
|
|
|
- Preconditions.checkArgument(vol.isAbsolute(),
|
|
|
|
- String.format("%s is not absolute path.", vol.getPath()));
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ public void removeVolumes(
|
|
|
|
+ Collection<StorageLocation> storageLocationsToRemove,
|
|
|
|
+ boolean clearFailure) {
|
|
Map<String, List<ReplicaInfo>> blkToInvalidate = new HashMap<>();
|
|
Map<String, List<ReplicaInfo>> blkToInvalidate = new HashMap<>();
|
|
List<String> storageToRemove = new ArrayList<>();
|
|
List<String> storageToRemove = new ArrayList<>();
|
|
try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
|
|
for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
|
|
Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
|
|
Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
|
|
- final File absRoot = sd.getRoot().getAbsoluteFile();
|
|
|
|
- if (volumesToRemove.contains(absRoot)) {
|
|
|
|
- LOG.info("Removing " + absRoot + " from FsDataset.");
|
|
|
|
-
|
|
|
|
|
|
+ final StorageLocation sdLocation = sd.getStorageLocation();
|
|
|
|
+ LOG.info("Checking removing StorageLocation " +
|
|
|
|
+ sdLocation + " with id " + sd.getStorageUuid());
|
|
|
|
+ if (storageLocationsToRemove.contains(sdLocation)) {
|
|
|
|
+ LOG.info("Removing StorageLocation " + sdLocation + " with id " +
|
|
|
|
+ sd.getStorageUuid() + " from FsDataset.");
|
|
// Disable the volume from the service.
|
|
// Disable the volume from the service.
|
|
- asyncDiskService.removeVolume(sd.getCurrentDir());
|
|
|
|
- volumes.removeVolume(absRoot, clearFailure);
|
|
|
|
|
|
+ asyncDiskService.removeVolume(sd.getStorageUuid());
|
|
|
|
+ volumes.removeVolume(sdLocation, clearFailure);
|
|
volumes.waitVolumeRemoved(5000, datasetLockCondition);
|
|
volumes.waitVolumeRemoved(5000, datasetLockCondition);
|
|
|
|
|
|
// Removed all replica information for the blocks on the volume.
|
|
// Removed all replica information for the blocks on the volume.
|
|
@@ -517,12 +524,14 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
// not scan disks.
|
|
// not scan disks.
|
|
for (String bpid : volumeMap.getBlockPoolList()) {
|
|
for (String bpid : volumeMap.getBlockPoolList()) {
|
|
List<ReplicaInfo> blocks = new ArrayList<>();
|
|
List<ReplicaInfo> blocks = new ArrayList<>();
|
|
- for (Iterator<ReplicaInfo> it = volumeMap.replicas(bpid).iterator();
|
|
|
|
- it.hasNext(); ) {
|
|
|
|
|
|
+ for (Iterator<ReplicaInfo> it =
|
|
|
|
+ volumeMap.replicas(bpid).iterator(); it.hasNext();) {
|
|
ReplicaInfo block = it.next();
|
|
ReplicaInfo block = it.next();
|
|
- final File absBasePath =
|
|
|
|
- new File(block.getVolume().getBasePath()).getAbsoluteFile();
|
|
|
|
- if (absBasePath.equals(absRoot)) {
|
|
|
|
|
|
+ final StorageLocation blockStorageLocation =
|
|
|
|
+ block.getVolume().getStorageLocation();
|
|
|
|
+ LOG.info("checking for block " + block.getBlockId() +
|
|
|
|
+ " with storageLocation " + blockStorageLocation);
|
|
|
|
+ if (blockStorageLocation.equals(sdLocation)) {
|
|
blocks.add(block);
|
|
blocks.add(block);
|
|
it.remove();
|
|
it.remove();
|
|
}
|
|
}
|
|
@@ -625,7 +634,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
List<String> failedStorageLocations = Lists.newArrayListWithCapacity(
|
|
List<String> failedStorageLocations = Lists.newArrayListWithCapacity(
|
|
infos.length);
|
|
infos.length);
|
|
for (VolumeFailureInfo info: infos) {
|
|
for (VolumeFailureInfo info: infos) {
|
|
- failedStorageLocations.add(info.getFailedStorageLocation());
|
|
|
|
|
|
+ failedStorageLocations.add(
|
|
|
|
+ info.getFailedStorageLocation().getFile().getAbsolutePath());
|
|
}
|
|
}
|
|
return failedStorageLocations.toArray(
|
|
return failedStorageLocations.toArray(
|
|
new String[failedStorageLocations.size()]);
|
|
new String[failedStorageLocations.size()]);
|
|
@@ -663,7 +673,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
long lastVolumeFailureDate = 0;
|
|
long lastVolumeFailureDate = 0;
|
|
long estimatedCapacityLostTotal = 0;
|
|
long estimatedCapacityLostTotal = 0;
|
|
for (VolumeFailureInfo info: infos) {
|
|
for (VolumeFailureInfo info: infos) {
|
|
- failedStorageLocations.add(info.getFailedStorageLocation());
|
|
|
|
|
|
+ failedStorageLocations.add(
|
|
|
|
+ info.getFailedStorageLocation().getFile().getAbsolutePath());
|
|
long failureDate = info.getFailureDate();
|
|
long failureDate = info.getFailureDate();
|
|
if (failureDate > lastVolumeFailureDate) {
|
|
if (failureDate > lastVolumeFailureDate) {
|
|
lastVolumeFailureDate = failureDate;
|
|
lastVolumeFailureDate = failureDate;
|
|
@@ -960,25 +971,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
FsVolumeImpl targetVolume = (FsVolumeImpl) volumeRef.getVolume();
|
|
FsVolumeImpl targetVolume = (FsVolumeImpl) volumeRef.getVolume();
|
|
// Copy files to temp dir first
|
|
// Copy files to temp dir first
|
|
- File[] blockFiles = copyBlockFiles(block.getBlockId(),
|
|
|
|
- block.getGenerationStamp(), replicaInfo,
|
|
|
|
- targetVolume.getTmpDir(block.getBlockPoolId()),
|
|
|
|
- replicaInfo.isOnTransientStorage(), smallBufferSize, conf);
|
|
|
|
-
|
|
|
|
- ReplicaInfo newReplicaInfo = new ReplicaBuilder(ReplicaState.TEMPORARY)
|
|
|
|
- .setBlockId(replicaInfo.getBlockId())
|
|
|
|
- .setGenerationStamp(replicaInfo.getGenerationStamp())
|
|
|
|
- .setFsVolume(targetVolume)
|
|
|
|
- .setDirectoryToUse(blockFiles[0].getParentFile())
|
|
|
|
- .setBytesToReserve(0)
|
|
|
|
- .build();
|
|
|
|
- newReplicaInfo.setNumBytes(blockFiles[1].length());
|
|
|
|
|
|
+ ReplicaInfo newReplicaInfo = targetVolume.moveBlockToTmpLocation(block,
|
|
|
|
+ replicaInfo, smallBufferSize, conf);
|
|
|
|
+
|
|
// Finalize the copied files
|
|
// Finalize the copied files
|
|
newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
|
|
newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
|
|
try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
try (AutoCloseableLock lock = datasetLock.acquire()) {
|
|
// Increment numBlocks here as this block moved without knowing to BPS
|
|
// Increment numBlocks here as this block moved without knowing to BPS
|
|
FsVolumeImpl volume = (FsVolumeImpl) newReplicaInfo.getVolume();
|
|
FsVolumeImpl volume = (FsVolumeImpl) newReplicaInfo.getVolume();
|
|
- volume.getBlockPoolSlice(block.getBlockPoolId()).incrNumBlocks();
|
|
|
|
|
|
+ volume.incrNumBlocks(block.getBlockPoolId());
|
|
}
|
|
}
|
|
|
|
|
|
removeOldReplica(replicaInfo, newReplicaInfo, block.getBlockPoolId());
|
|
removeOldReplica(replicaInfo, newReplicaInfo, block.getBlockPoolId());
|
|
@@ -2072,7 +2073,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
* @return the failed volumes. Returns null if no volume failed.
|
|
* @return the failed volumes. Returns null if no volume failed.
|
|
*/
|
|
*/
|
|
@Override // FsDatasetSpi
|
|
@Override // FsDatasetSpi
|
|
- public Set<File> checkDataDir() {
|
|
|
|
|
|
+ public Set<StorageLocation> checkDataDir() {
|
|
return volumes.checkDirs();
|
|
return volumes.checkDirs();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2250,9 +2251,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
.setFsVolume(vol)
|
|
.setFsVolume(vol)
|
|
.setDirectoryToUse(diskFile.getParentFile())
|
|
.setDirectoryToUse(diskFile.getParentFile())
|
|
.build();
|
|
.build();
|
|
- ((FsVolumeImpl) vol).getBlockPoolSlice(bpid)
|
|
|
|
- .resolveDuplicateReplicas(
|
|
|
|
- memBlockInfo, diskBlockInfo, volumeMap);
|
|
|
|
|
|
+ ((FsVolumeImpl) vol).resolveDuplicateReplicas(bpid,
|
|
|
|
+ memBlockInfo, diskBlockInfo, volumeMap);
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
if (!diskFile.delete()) {
|
|
if (!diskFile.delete()) {
|
|
@@ -2803,15 +2803,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
// Add thread for DISK volume if RamDisk is configured
|
|
// Add thread for DISK volume if RamDisk is configured
|
|
if (ramDiskConfigured &&
|
|
if (ramDiskConfigured &&
|
|
asyncLazyPersistService != null &&
|
|
asyncLazyPersistService != null &&
|
|
- !asyncLazyPersistService.queryVolume(v.getCurrentDir())) {
|
|
|
|
- asyncLazyPersistService.addVolume(v.getCurrentDir());
|
|
|
|
|
|
+ !asyncLazyPersistService.queryVolume(v)) {
|
|
|
|
+ asyncLazyPersistService.addVolume(v);
|
|
}
|
|
}
|
|
|
|
|
|
// Remove thread for DISK volume if RamDisk is not configured
|
|
// Remove thread for DISK volume if RamDisk is not configured
|
|
if (!ramDiskConfigured &&
|
|
if (!ramDiskConfigured &&
|
|
asyncLazyPersistService != null &&
|
|
asyncLazyPersistService != null &&
|
|
- asyncLazyPersistService.queryVolume(v.getCurrentDir())) {
|
|
|
|
- asyncLazyPersistService.removeVolume(v.getCurrentDir());
|
|
|
|
|
|
+ asyncLazyPersistService.queryVolume(v)) {
|
|
|
|
+ asyncLazyPersistService.removeVolume(v);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2946,11 +2946,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
// Move the replica from lazyPersist/ to finalized/ on
|
|
// Move the replica from lazyPersist/ to finalized/ on
|
|
// the target volume
|
|
// the target volume
|
|
- BlockPoolSlice bpSlice =
|
|
|
|
- replicaState.getLazyPersistVolume().getBlockPoolSlice(bpid);
|
|
|
|
-
|
|
|
|
newReplicaInfo =
|
|
newReplicaInfo =
|
|
- bpSlice.activateSavedReplica(replicaInfo, replicaState);
|
|
|
|
|
|
+ replicaState.getLazyPersistVolume().activateSavedReplica(bpid,
|
|
|
|
+ replicaInfo, replicaState);
|
|
|
|
|
|
// Update the volumeMap entry.
|
|
// Update the volumeMap entry.
|
|
volumeMap.add(bpid, newReplicaInfo);
|
|
volumeMap.add(bpid, newReplicaInfo);
|