@Override public FsDatasetImpl newInstance(DataNode datanode, DataStorage storage, Configuration conf) throws IOException { return new FsDatasetImpl(datanode, storage, conf); } }
List<VolumeFailureInfo> volumeFailureInfos = getInitialVolumeFailureInfos( dataLocations, storage); addVolume(storage.getStorageDir(idx)); setupAsyncLazyPersistThreads(); registerMBean(datanode.getDatanodeUuid());
" should be greater than the replica " + b + "'s generation stamp"); ReplicaInfo replicaInfo = getReplicaInfo(b); LOG.info("Appending to " + replicaInfo); if (replicaInfo.getState() != ReplicaState.FINALIZED) { ReplicaInPipeline replica = null; try { replica = append(b.getBlockPoolId(), replicaInfo, newGS, b.getNumBytes()); } catch (IOException e) {
replicaInfo = getReplicaInfo(b); if (replicaInfo.getState() == ReplicaState.FINALIZED) { finalizedReplicaInfo = finalizeReplica(b.getBlockPoolId(), replicaInfo);
/** * Moves a given block from one volume to another volume. This is used by disk * balancer. * * @param block - ExtendedBlock * @param destination - Destination volume * @return Old replica info */ @Override public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block, FsVolumeSpi destination) throws IOException { ReplicaInfo replicaInfo = getReplicaInfo(block); if (replicaInfo.getState() != ReplicaState.FINALIZED) { throw new ReplicaNotFoundException( ReplicaNotFoundException.UNFINALIZED_REPLICA + block); } FsVolumeReference volumeRef = null; try (AutoCloseableLock lock = datasetLock.acquire()) { volumeRef = destination.obtainReference(); } try { moveBlock(block, replicaInfo, volumeRef); } finally { if (volumeRef != null) { volumeRef.close(); } } return replicaInfo; }
setupAsyncLazyPersistThreads(); List<ReplicaInfo> blocks = entry.getValue(); for (ReplicaInfo block : blocks) { invalidate(bpid, block);
public ReplicaInfo moveBlockAcrossStorage(ExtendedBlock block, StorageType targetStorageType) throws IOException { ReplicaInfo replicaInfo = getReplicaInfo(block); if (replicaInfo.getState() != ReplicaState.FINALIZED) { throw new ReplicaNotFoundException( FsVolumeImpl targetVolume = (FsVolumeImpl) volumeRef.getVolume(); File[] blockFiles = copyBlockFiles(block.getBlockId(), block.getGenerationStamp(), oldMetaFile, oldBlockFile, targetVolume.getTmpDir(block.getBlockPoolId()), newReplicaInfo.setNumBytes(blockFiles[1].length()); newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo); removeOldReplica(replicaInfo, newReplicaInfo, oldBlockFile, oldMetaFile, oldBlockFile.length(), oldMetaFile.length(), block.getBlockPoolId()); } finally {
@Override // FsDatasetSpi public synchronized ReplicaHandler recoverAppend( ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException { LOG.info("Recover failed append to " + b); ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen); FsVolumeReference ref = replicaInfo.getVolume().obtainReference(); ReplicaBeingWritten replica; try { // change the replica's state/gs etc. if (replicaInfo.getState() == ReplicaState.FINALIZED) { replica = append(b.getBlockPoolId(), (FinalizedReplica) replicaInfo, newGS, b.getNumBytes()); } else { //RBW bumpReplicaGS(replicaInfo, newGS); replica = (ReplicaBeingWritten) replicaInfo; } } catch (IOException e) { IOUtils.cleanup(null, ref); throw e; } return new ReplicaHandler(replica, ref); }
checkReplicaFiles(replica); final ReplicaInfo finalized = updateReplicaUnderRecovery(oldBlock .getBlockPoolId(), replica, recoveryId, newBlockId, newlength); checkReplicaFiles(finalized);
@Override // FsDatasetSpi public long getReplicaVisibleLength(final ExtendedBlock block) throws IOException { try (AutoCloseableLock lock = datasetLock.acquire()) { final Replica replica = getReplicaInfo(block.getBlockPoolId(), block.getBlockId()); if (replica.getGenerationStamp() < block.getGenerationStamp()) { throw new IOException( "replica.getGenerationStamp() < block.getGenerationStamp(), block=" + block + ", replica=" + replica); } return replica.getVisibleLength(); } }
LOG.info("Recover RBW replica " + b); ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId()); truncateBlock(replicafile, rbw.getMetaFile(), bytesOnDisk, bytesAcked); rbw.setNumBytes(bytesAcked); rbw.setLastChecksumAndDataLen(bytesAcked, null); bumpReplicaGS(rbw, newGS); } catch (IOException e) { IOUtils.cleanup(null, ref);
createFsVolume(sd.getStorageUuid(), sd, location); final ReplicaMap tempVolumeMap = new ReplicaMap(new AutoCloseableLock()); ArrayList<IOException> exceptions = Lists.newArrayList(); setupAsyncLazyPersistThread(fsVolume); activateVolume(tempVolumeMap, sd, storageType, ref); LOG.info("Added volume - " + location + ", StorageType: " + storageType);
@Before public void setUp() throws IOException { datanode = mock(DataNode.class); storage = mock(DataStorage.class); this.conf = new Configuration(); this.conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0); final DNConf dnConf = new DNConf(conf); when(datanode.getConf()).thenReturn(conf); when(datanode.getDnConf()).thenReturn(dnConf); final BlockScanner disabledBlockScanner = new BlockScanner(datanode, conf); when(datanode.getBlockScanner()).thenReturn(disabledBlockScanner); final ShortCircuitRegistry shortCircuitRegistry = new ShortCircuitRegistry(conf); when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry); createStorageDirs(storage, conf, NUM_INIT_VOLUMES); dataset = new FsDatasetImpl(datanode, storage, conf); for (String bpid : BLOCK_POOL_IDS) { dataset.addBlockPool(bpid, conf); } assertEquals(NUM_INIT_VOLUMES, dataset.getVolumes().size()); assertEquals(0, dataset.getNumFailedVolumes()); }
createFsVolume(sd.getStorageUuid(), sd.getCurrentDir(), storageType); final ReplicaMap tempVolumeMap = new ReplicaMap(fsVolume); ArrayList<IOException> exceptions = Lists.newArrayList(); setupAsyncLazyPersistThread(fsVolume);
replicaInfo = getReplicaInfo(b); if (replicaInfo.getState() == ReplicaState.FINALIZED) { finalizedReplicaInfo = finalizeReplica(b.getBlockPoolId(), replicaInfo); File f = replicaInfo.getBlockFile(); File dest = finalizedReplicaInfo.getBlockFile(); fsyncDirectory(dest.getParentFile(), f.getParentFile());
try { try (AutoCloseableLock lock = datasetLock.acquire()) { ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen); FsVolumeReference ref = replicaInfo.getVolume().obtainReference(); ReplicaInPipeline replica; replica = append(b.getBlockPoolId(), replicaInfo, newGS, b.getNumBytes()); } else { //RBW
/** * We're informed that a block is no longer valid. Delete it. */ @Override // FsDatasetSpi public void invalidate(String bpid, Block invalidBlks[]) throws IOException { invalidate(bpid, invalidBlks, true); }
/** * Returns handles to the block file and its metadata file */ @Override // FsDatasetSpi public synchronized ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkOffset, long metaOffset) throws IOException { ReplicaInfo info = getReplicaInfo(b); FsVolumeReference ref = info.getVolume().obtainReference(); try { InputStream blockInStream = openAndSeek(info.getBlockFile(), blkOffset); try { InputStream metaInStream = openAndSeek(info.getMetaFile(), metaOffset); return new ReplicaInputStreams(blockInStream, metaInStream, ref); } catch (IOException e) { IOUtils.cleanup(null, blockInStream); throw e; } } catch (IOException e) { IOUtils.cleanup(null, ref); throw e; } }
@Test(timeout = 5000) public void testRemoveNewlyAddedVolume() throws IOException { final int numExistingVolumes = dataset.getVolumes().size(); List<NamespaceInfo> nsInfos = new ArrayList<>(); for (String bpid : BLOCK_POOL_IDS) { nsInfos.add(new NamespaceInfo(0, CLUSTER_ID, bpid, 1)); } String newVolumePath = BASE_DIR + "/newVolumeToRemoveLater"; StorageLocation loc = StorageLocation.parse(newVolumePath); Storage.StorageDirectory sd = createStorageDirectory(new File(newVolumePath)); DataStorage.VolumeBuilder builder = new DataStorage.VolumeBuilder(storage, sd); when(storage.prepareVolume(eq(datanode), eq(loc.getFile()), anyListOf(NamespaceInfo.class))) .thenReturn(builder); dataset.addVolume(loc, nsInfos); assertEquals(numExistingVolumes + 1, dataset.getVolumes().size()); when(storage.getNumStorageDirs()).thenReturn(numExistingVolumes + 1); when(storage.getStorageDir(numExistingVolumes)).thenReturn(sd); Set<File> volumesToRemove = new HashSet<>(); volumesToRemove.add(loc.getFile()); dataset.removeVolumes(volumesToRemove, true); assertEquals(numExistingVolumes, dataset.getVolumes().size()); }
badDir.mkdirs(); doReturn(mockVolume).when(spyDataset) .createFsVolume(anyString(), any(File.class), any(StorageType.class)); doThrow(new IOException("Failed to getVolumeMap()")) .when(mockVolume).getVolumeMap( spyDataset.addVolume(location, nsInfos); fail("Expect to throw MultipleIOException"); } catch (MultipleIOException e) {