private long getRemainingReserved() throws IOException { long actualNonDfsUsed = getActualNonDfsUsed(); long actualReserved = getReserved(); if (actualNonDfsUsed < actualReserved) { return actualReserved - actualNonDfsUsed; } return 0L; }
void getVolumeMap(String bpid, ReplicaMap volumeMap, final RamDiskReplicaTracker ramDiskReplicaMap) throws IOException { getBlockPoolSlice(bpid).getVolumeMap(volumeMap, ramDiskReplicaMap); }
@Override // FsDatasetSpi public StorageReport[] getStorageReports(String bpid) throws IOException { List<StorageReport> reports; // Volumes are the references from a copy-on-write snapshot, so the // access on the volume metrics doesn't require an additional lock. List<FsVolumeImpl> curVolumes = volumes.getVolumes(); reports = new ArrayList<>(curVolumes.size()); for (FsVolumeImpl volume : curVolumes) { try (FsVolumeReference ref = volume.obtainReference()) { StorageReport sr = new StorageReport(volume.toDatanodeStorage(), false, volume.getCapacity(), volume.getDfsUsed(), volume.getAvailable(), volume.getBlockPoolUsed(bpid), volume.getNonDfsUsed()); reports.add(sr); } catch (ClosedChannelException e) { continue; } } return reports.toArray(new StorageReport[reports.size()]); }
VolumeInfo(FsVolumeImpl v, long usedSpace, long freeSpace) { this.directory = v.toString(); this.usedSpace = usedSpace; this.freeSpace = freeSpace; this.reservedSpace = v.getReserved(); this.reservedSpaceForReplicas = v.getReservedForReplicas(); this.numBlocks = v.getNumBlocks(); this.storageType = v.getStorageType(); } }
long getCapacity() { long capacity = 0L; for (FsVolumeImpl v : volumes) { try (FsVolumeReference ref = v.obtainReference()) { capacity += v.getCapacity(); } catch (IOException e) { // ignore. } } return capacity; }
long reservedForReplicas = 50L; conf.setLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY, duReserved); FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", volDir, conf, StorageType.DEFAULT); FsVolumeImpl spyVolume = Mockito.spy(volume); spyVolume.setCapacityForTesting(testCapacity); Mockito.doReturn(dfAvailable).when(spyVolume).getDfAvailable(); Mockito.doReturn(dfsUsage).when(spyVolume).getDfsUsed(); Mockito.doReturn(reservedForReplicas).when(spyVolume).getReservedForRbw(); Mockito.doReturn(actualNonDfsUsage).when(spyVolume).getActualNonDfsUsed(); long expectedNonDfsUsage = actualNonDfsUsage - duReserved; assertEquals(expectedNonDfsUsage, spyVolume.getNonDfsUsed());
@Override public long getAvailable() throws IOException { long remaining = getCapacity() - getDfsUsed() - getReservedForRbw(); long available = usage.getAvailable() - getRemainingReserved() - getReservedForRbw(); if (remaining > available) { remaining = available; } return (remaining > 0) ? remaining : 0; }
private boolean transientFreeSpaceBelowThreshold() throws IOException { long free = 0; long capacity = 0; float percentFree = 0.0f; // Don't worry about fragmentation for now. We don't expect more than one // transient volume per DN. for (FsVolumeImpl v : getVolumes()) { try (FsVolumeReference ref = v.obtainReference()) { if (v.isTransientStorage()) { capacity += v.getCapacity(); free += v.getAvailable(); } } catch (ClosedChannelException e) { // ignore. } } if (capacity == 0) { return false; } percentFree = (float) ((double)free * 100 / capacity); return (percentFree < lowWatermarkFreeSpacePercentage) || (free < lowWatermarkFreeSpaceBytes); }
private Collection<VolumeInfo> getVolumeInfo() { Collection<VolumeInfo> info = new ArrayList<VolumeInfo>(); for (FsVolumeImpl volume : volumes.getVolumes()) { long used = 0; long free = 0; try (FsVolumeReference ref = volume.obtainReference()) { used = volume.getDfsUsed(); free = volume.getAvailable(); } catch (ClosedChannelException e) { continue; } catch (IOException e) { LOG.warn(e.getMessage()); used = 0; free = 0; } info.add(new VolumeInfo(volume, used, free)); } return info; }
/** * RBW files. They get moved to the finalized block directory when * the block is finalized. */ File createRbwFile(String bpid, Block b) throws IOException { checkReference(); reserveSpaceForRbw(b.getNumBytes()); try { return getBlockPoolSlice(bpid).createRbwFile(b); } catch (IOException exception) { releaseReservedSpace(b.getNumBytes()); throw exception; } }
private void addVolume(Collection<StorageLocation> dataLocations, Storage.StorageDirectory sd) throws IOException { final File dir = sd.getCurrentDir(); final StorageType storageType = getStorageTypeFromLocations(dataLocations, sd.getRoot()); // If IOException raises from FsVolumeImpl() or getVolumeMap(), there is // nothing needed to be rolled back to make various data structures, e.g., // storageMap and asyncDiskService, consistent. FsVolumeImpl fsVolume = new FsVolumeImpl( this, sd.getStorageUuid(), dir, this.conf, storageType); FsVolumeReference ref = fsVolume.obtainReference(); ReplicaMap tempVolumeMap = new ReplicaMap(this); fsVolume.getVolumeMap(tempVolumeMap, ramDiskReplicaTracker); synchronized (this) { volumeMap.addAll(tempVolumeMap); storageMap.put(sd.getStorageUuid(), new DatanodeStorage(sd.getStorageUuid(), DatanodeStorage.State.NORMAL, storageType)); asyncDiskService.addVolume(sd.getCurrentDir()); volumes.addVolume(ref); } LOG.info("Added volume - " + dir + ", StorageType: " + storageType); }
@Override public long getAvailable() throws IOException { long remaining = getCapacity() - getDfsUsed() - getReservedForReplicas(); long available = usage.getAvailable() - getRemainingReserved() - getReservedForReplicas(); if (remaining > available) { remaining = available; } return (remaining > 0) ? remaining : 0; }
/** * Temporary files. They get moved to the finalized block directory when * the block is finalized. */ File createTmpFile(String bpid, Block b) throws IOException { checkReference(); reserveSpaceForReplica(b.getNumBytes()); try { return getBlockPoolSlice(bpid).createTmpFile(b); } catch (IOException exception) { releaseReservedSpace(b.getNumBytes()); throw exception; } }
long getDfsUsed() throws IOException { long dfsUsed = 0L; for (FsVolumeImpl v : volumes) { try(FsVolumeReference ref = v.obtainReference()) { dfsUsed += v.getDfsUsed(); } catch (ClosedChannelException e) { // ignore. } } return dfsUsed; }
boolean isBPDirEmpty(String bpid) throws IOException { File volumeCurrentDir = this.getCurrentDir(); File bpDir = new File(volumeCurrentDir, bpid); File bpCurrentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT); File finalizedDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_FINALIZED); File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW); if (fileIoProvider.exists(this, finalizedDir) && !DatanodeUtil.dirNoFilesRecursive(this, finalizedDir, fileIoProvider)) { return false; } if (fileIoProvider.exists(this, rbwDir) && fileIoProvider.list(this, rbwDir).length != 0) { return false; } return true; }
/** * * @param bytesReserved Space that was reserved during * block creation. Now that the block is being finalized we * can free up this space. * @return * @throws IOException */ ReplicaInfo addFinalizedBlock(String bpid, Block b, ReplicaInfo replicaInfo, long bytesReserved) throws IOException { releaseReservedSpace(bytesReserved); File dest = getBlockPoolSlice(bpid).addFinalizedBlock(b, replicaInfo); byte[] checksum = null; // copy the last partial checksum if the replica is originally // in finalized or rbw state. if (replicaInfo.getState() == ReplicaState.FINALIZED) { FinalizedReplica finalized = (FinalizedReplica)replicaInfo; checksum = finalized.getLastPartialChunkChecksum(); } else if (replicaInfo.getState() == ReplicaState.RBW) { ReplicaBeingWritten rbw = (ReplicaBeingWritten)replicaInfo; checksum = rbw.getLastChecksumAndDataLen().getChecksum(); } return new ReplicaBuilder(ReplicaState.FINALIZED) .setBlock(replicaInfo) .setFsVolume(this) .setDirectoryToUse(dest.getParentFile()) .setLastPartialChunkChecksum(checksum) .build(); }
@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)); } } }
.getNextVolume(StorageType.DEFAULT, 0).getVolume(); ReplicaInfo replicaInfo = new FinalizedReplica( blocks[FINALIZED].getLocalBlock(), vol, vol.getCurrentDir().getParentFile()); replicasMap.add(bpid, replicaInfo); replicaInfo.getBlockFile().createNewFile(); blocks[TEMPORARY].getBlockId(), blocks[TEMPORARY].getGenerationStamp(), vol, vol.createTmpFile(bpid, blocks[TEMPORARY].getLocalBlock()).getParentFile(), 0)); vol.createRbwFile(bpid, blocks[RBW].getLocalBlock()).getParentFile(), null); replicasMap.add(bpid, replicaInfo); replicaInfo.getBlockFile().createNewFile(); blocks[RWR].getLocalBlock(), vol, vol.createRbwFile(bpid, blocks[RWR].getLocalBlock()).getParentFile())); replicasMap.add(bpid, new ReplicaUnderRecovery(new FinalizedReplica(blocks[RUR] .getLocalBlock(), vol, vol.getCurrentDir().getParentFile()), 2007));
@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(); }
final FsVolumeImpl v = (FsVolumeImpl)dataSet.volumeMap.get( bpid, blocks[FINALIZED].getLocalBlock()).getVolume(); long available = v.getCapacity()-v.getDfsUsed(); long expectedLen = blocks[FINALIZED].getNumBytes(); try { v.decDfsUsed(bpid, -available); blocks[FINALIZED].setNumBytes(expectedLen+100); dataSet.append(blocks[FINALIZED], newGS, expectedLen); "Insufficient space for appending to ")); v.decDfsUsed(bpid, available); blocks[FINALIZED].setNumBytes(expectedLen);