/** * Activate a volume to serve requests. * @throws IOException if the storage UUID already exists. */ private void activateVolume( ReplicaMap replicaMap, Storage.StorageDirectory sd, StorageType storageType, FsVolumeReference ref) throws IOException { try (AutoCloseableLock lock = datasetLock.acquire()) { DatanodeStorage dnStorage = storageMap.get(sd.getStorageUuid()); if (dnStorage != null) { final String errorMsg = String.format( "Found duplicated storage UUID: %s in %s.", sd.getStorageUuid(), sd.getVersionFile()); LOG.error(errorMsg); throw new IOException(errorMsg); } volumeMap.addAll(replicaMap); storageMap.put(sd.getStorageUuid(), new DatanodeStorage(sd.getStorageUuid(), DatanodeStorage.State.NORMAL, storageType)); asyncDiskService.addVolume((FsVolumeImpl) ref.getVolume()); volumes.addVolume(ref); } }
/** * Return the number of volume. */ public int getVolumeCount() { return volumes.getVolumes().size(); }
/** * Dynamically remove volume in the list. * @param storageLocation {@link StorageLocation} of the volume to be removed. * @param clearFailure set true to remove failure info for this volume. */ void removeVolume(StorageLocation storageLocation, boolean clearFailure) { for (FsVolumeImpl fsVolume : volumes) { StorageLocation baseLocation = fsVolume.getStorageLocation(); if (baseLocation.equals(storageLocation)) { removeVolume(fsVolume); } } if (clearFailure) { removeVolumeFailureInfo(storageLocation); } }
@Override public void addBlockPool(String bpid, Configuration conf) throws IOException { LOG.info("Adding block pool " + bpid); try (AutoCloseableLock lock = datasetLock.acquire()) { volumes.addBlockPool(bpid, conf); volumeMap.initBlockPool(bpid); } volumes.getAllVolumesMap(bpid, volumeMap, ramDiskReplicaTracker); }
/** * Get next volume. * * @param blockSize free space needed on the volume * @return next volume to store the block in. */ FsVolumeReference getNextTransientVolume(long blockSize) throws IOException { // Get a snapshot of currently available volumes. final List<FsVolumeImpl> curVolumes = getVolumes(); final List<FsVolumeImpl> list = new ArrayList<>(curVolumes.size()); for(FsVolumeImpl v : curVolumes) { if (v.isTransientStorage()) { list.add(v); } } return chooseVolume(list, blockSize, null); }
/** * Updates the failed volume info in the volumeFailureInfos Map * and calls {@link #removeVolume(FsVolumeImpl)} to remove the volume * from the volume list for each of the failed volumes. * * @param failedVolumes set of volumes marked failed. */ void handleVolumeFailures(Set<FsVolumeSpi> failedVolumes) { try (AutoCloseableLock lock = checkDirsLock.acquire()) { for(FsVolumeSpi vol : failedVolumes) { FsVolumeImpl fsv = (FsVolumeImpl) vol; try (FsVolumeReference ref = fsv.obtainReference()) { addVolumeFailureInfo(fsv); removeVolume(fsv); } catch (ClosedChannelException e) { FsDatasetImpl.LOG.debug("Caught exception when obtaining " + "reference count on closed volume", e); } catch (IOException e) { FsDatasetImpl.LOG.error("Unexpected IOException", e); } } waitVolumeRemoved(5000, checkDirsLockCondition); } }
final List<FsVolumeImpl> volumeList = getVolumes(); addVolumeFailureInfo(fsv); removeVolume(fsv); } catch (ClosedChannelException e) { FsDatasetImpl.LOG.debug("Caught exception when obtaining " +
conf.setLong(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); final BlockScanner blockScanner = new BlockScanner(datanode, conf); final FsVolumeList volumeList = new FsVolumeList( Collections.<VolumeFailureInfo>emptyList(), blockScanner, blockChooser); final List<FsVolumeImpl> oldVolumes = new ArrayList<>(); FsVolumeReference ref = mock(FsVolumeReference.class); when(ref.getVolume()).thenReturn(volume); volumeList.addVolume(ref); when(newRef.getVolume()).thenReturn(newVolume); when(newVolume.getBasePath()).thenReturn("data4"); FsVolumeImpl blockedVolume = volumeList.getVolumes().get(1); doAnswer(new Answer() { @Override FsVolumeImpl brokenVolume = volumeList.getVolumes().get(2); doThrow(new DiskChecker.DiskErrorException("broken")) .when(brokenVolume).checkDirs(); volumeList.checkDirs(); assertTrue(volumeList.getVolumes().contains(newVolume)); assertFalse(volumeList.getVolumes().contains(brokenVolume)); assertEquals(NUM_VOLUMES - 1, volumeList.getVolumes().size());
replicaInfo.getVolume().isTransientStorage()) { targetReference = volumes.getNextVolume( StorageType.DEFAULT, null, replicaInfo.getNumBytes()); targetVolume = (FsVolumeImpl) targetReference.getVolume();
@Test public void testGetNextVolumeWithClosedVolume() throws IOException { FsVolumeList volumeList = new FsVolumeList( Collections.<VolumeFailureInfo>emptyList(), blockScanner, blockChooser); List<FsVolumeImpl> volumes = new ArrayList<>(); for (int i = 0; i < 3; i++) { File curDir = new File(baseDir, "nextvolume-" + i); curDir.mkdirs(); FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", curDir, conf, StorageType.DEFAULT); volume.setCapacityForTesting(1024 * 1024 * 1024); volumes.add(volume); volumeList.addVolume(volume.obtainReference()); } // Close the second volume. volumes.get(1).closeAndWait(); for (int i = 0; i < 10; i++) { try (FsVolumeReference ref = volumeList.getNextVolume(StorageType.DEFAULT, 128)) { // volume No.2 will not be chosen. assertNotEquals(ref.getVolume(), volumes.get(1)); } } }
try { ref = volumes.getNextTransientVolume(b.getNumBytes()); datanode.getMetrics().incrRamDiskBlocksWrite(); } catch (DiskOutOfSpaceException de) { ref = volumes.getNextVolume(storageType, storageId, b.getNumBytes());
@Test public void testCheckDirsWithClosedVolume() throws IOException { FsVolumeList volumeList = new FsVolumeList( Collections.<VolumeFailureInfo>emptyList(), blockScanner, blockChooser); List<FsVolumeImpl> volumes = new ArrayList<>(); for (int i = 0; i < 3; i++) { File curDir = new File(baseDir, "volume-" + i); curDir.mkdirs(); FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", curDir, conf, StorageType.DEFAULT); volumes.add(volume); volumeList.addVolume(volume.obtainReference()); } // Close the 2nd volume. volumes.get(1).closeAndWait(); // checkDirs() should ignore the 2nd volume since it is closed. volumeList.checkDirs(); }
builder = dataStorage.prepareVolume(datanode, location.getFile(), nsInfos); } catch (IOException e) { volumes.addVolumeFailureInfo(new VolumeFailureInfo( location.getFile().getAbsolutePath(), Time.now())); throw e; storageType)); asyncDiskService.addVolume(sd.getCurrentDir()); volumes.addVolume(ref);
volumes.removeVolume(sdLocation, clearFailure); volumes.waitVolumeRemoved(5000, datasetLockCondition); volumes.removeVolumeFailureInfo(storageLocToRemove);
@Override public Object answer(InvocationOnMock invocationOnMock) throws Throwable { volumeList.removeVolume(new File("data4"), false); volumeList.addVolume(newRef); return null; } }).when(blockedVolume).checkDirs();
@Test public void testReleaseVolumeRefIfNoBlockScanner() throws IOException { FsVolumeList volumeList = new FsVolumeList( Collections.<VolumeFailureInfo>emptyList(), null, blockChooser); File volDir = new File(baseDir, "volume-0"); volDir.mkdirs(); FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", volDir, conf, StorageType.DEFAULT); FsVolumeReference ref = volume.obtainReference(); volumeList.addVolume(ref); try { ref.close(); fail("Should throw exception because the reference is closed in " + "VolumeList#addVolume()."); } catch (IllegalStateException e) { } }
RoundRobinVolumeChoosingPolicy.class, VolumeChoosingPolicy.class), conf); volumes = new FsVolumeList(volumeFailureInfos, datanode.getBlockScanner(), blockChooserImpl); asyncDiskService = new FsDatasetAsyncDiskService(datanode, this);
private void addVolumeFailureInfo(FsVolumeImpl vol) { addVolumeFailureInfo(new VolumeFailureInfo( vol.getStorageLocation(), Time.now(), vol.getCapacity())); }
volumes.removeVolume(absRoot, clearFailure);
/** * Get next volume. * * @param blockSize free space needed on the volume * @param storageType the desired {@link StorageType} * @param storageId the storage id which may or may not be used by * the VolumeChoosingPolicy. * @return next volume to store the block in. */ FsVolumeReference getNextVolume(StorageType storageType, String storageId, long blockSize) throws IOException { final List<FsVolumeImpl> list = new ArrayList<>(volumes.size()); for(FsVolumeImpl v : volumes) { if (v.getStorageType() == storageType) { list.add(v); } } return chooseVolume(list, blockSize, storageId); }