/** {@inheritDoc} */ public void updateBlock(Block oldblock, Block newblock) throws IOException { if (oldblock.getBlockId() != newblock.getBlockId()) { throw new IOException( "Cannot update oldblock (=" + oldblock + ") to newblock (=" + newblock + ")."); } for (; ; ) { final List<Thread> threads = tryUpdateBlock(oldblock, newblock); if (threads == null) { return; } // interrupt and wait for all ongoing create threads for (Thread t : threads) { t.interrupt(); } for (Thread t : threads) { try { t.join(); } catch (InterruptedException e) { DataNode.LOG.warn("interruptOngoingCreates: t=" + t, e); } } } }
/** * Remove a block from disk * * @param blockFile block file * @param metaFile block meta file * @param b a block * @return true if on-disk files are deleted; false otherwise */ private boolean delBlockFromDisk(File blockFile, File metaFile, Block b) { if (blockFile == null) { DataNode.LOG.warn("No file exists for block: " + b); return true; } if (!blockFile.delete()) { DataNode.LOG.warn("Not able to delete the block file: " + blockFile); return false; } else { // remove the meta file if (metaFile != null && !metaFile.delete()) { DataNode.LOG.warn("Not able to delete the meta block file: " + metaFile); return false; } } return true; }
/** Remove the temporary block file (if any) */ public synchronized void unfinalizeBlock(Block b) throws IOException { // remove the block from in-memory data structure ActiveFile activefile = ongoingCreates.remove(b); if (activefile == null) { return; } volumeMap.remove(b); // delete the on-disk temp file if (delBlockFromDisk(activefile.file, getMetaFile(activefile.file, b), b)) { DataNode.LOG.warn("Block " + b + " unfinalized and removed. "); } }
/** * Find the metadata file for the specified block file. Return the generation stamp from the * name of the metafile. */ long getGenerationStampFromFile(File[] listdir, File blockFile) { String blockName = blockFile.getName(); for (int j = 0; j < listdir.length; j++) { String path = listdir[j].getName(); if (!path.startsWith(blockName)) { continue; } String[] vals = path.split("_"); if (vals.length != 3) { // blk, blkid, genstamp.meta continue; } String[] str = vals[2].split("\\."); if (str.length != 2) { continue; } return Long.parseLong(str[0]); } DataNode.LOG.warn("Block " + blockFile + " does not have a metafile!"); return Block.GRANDFATHER_GENERATION_STAMP; }
/** * Start writing to a block file If isRecovery is true and the block pre-exists, then we kill all * volumeMap.put(b, v); volumeMap.put(b, v); other threads that might be writing to this block, * and then reopen the file. */ public BlockWriteStreams writeToBlock(Block b, boolean isRecovery) throws IOException { // // Make sure the block isn't a valid one - we're still creating it! // if (isValidBlock(b)) { if (!isRecovery) { throw new BlockAlreadyExistsException( "Block " + b + " is valid, and cannot be written to."); } // If the block was successfully finalized because all packets // were successfully processed at the Datanode but the ack for // some of the packets were not received by the client. The client // re-opens the connection and retries sending those packets. // The other reason is that an "append" is occurring to this block. detachBlock(b, 1); } long blockSize = b.getNumBytes(); // // Serialize access to /tmp, and check if file already there. // File f = null; List<Thread> threads = null; synchronized (this) { // // Is it already in the create process? // ActiveFile activeFile = ongoingCreates.get(b); if (activeFile != null) { f = activeFile.file; threads = activeFile.threads; if (!isRecovery) { throw new BlockAlreadyExistsException( "Block " + b + " has already been started (though not completed), and thus cannot be created."); } else { for (Thread thread : threads) { thread.interrupt(); } } ongoingCreates.remove(b); } FSVolume v = null; if (!isRecovery) { v = volumes.getNextVolume(blockSize); // create temporary file to hold block in the designated volume f = createTmpFile(v, b); volumeMap.put(b, new DatanodeBlockInfo(v)); } else if (f != null) { DataNode.LOG.info("Reopen already-open Block for append " + b); // create or reuse temporary file to hold block in the // designated volume v = volumeMap.get(b).getVolume(); volumeMap.put(b, new DatanodeBlockInfo(v)); } else { // reopening block for appending to it. DataNode.LOG.info("Reopen Block for append " + b); v = volumeMap.get(b).getVolume(); f = createTmpFile(v, b); File blkfile = getBlockFile(b); File oldmeta = getMetaFile(b); File newmeta = getMetaFile(f, b); // rename meta file to tmp directory DataNode.LOG.debug("Renaming " + oldmeta + " to " + newmeta); if (!oldmeta.renameTo(newmeta)) { throw new IOException( "Block " + b + " reopen failed. " + " Unable to move meta file " + oldmeta + " to tmp dir " + newmeta); } // rename block file to tmp directory DataNode.LOG.debug("Renaming " + blkfile + " to " + f); if (!blkfile.renameTo(f)) { if (!f.delete()) { throw new IOException( "Block " + b + " reopen failed. " + " Unable to remove file " + f); } if (!blkfile.renameTo(f)) { throw new IOException( "Block " + b + " reopen failed. " + " Unable to move block file " + blkfile + " to tmp dir " + f); } } volumeMap.put(b, new DatanodeBlockInfo(v)); } if (f == null) { DataNode.LOG.warn("Block " + b + " reopen failed " + " Unable to locate tmp file."); throw new IOException("Block " + b + " reopen failed " + " Unable to locate tmp file."); } ongoingCreates.put(b, new ActiveFile(f, threads)); } try { if (threads != null) { for (Thread thread : threads) { thread.join(); } } } catch (InterruptedException e) { throw new IOException("Recovery waiting for thread interrupted."); } // // Finally, allow a writer to the block file // REMIND - mjc - make this a filter stream that enforces a max // block size, so clients can't go crazy // File metafile = getMetaFile(f, b); DataNode.LOG.debug("writeTo blockfile is " + f + " of size " + f.length()); DataNode.LOG.debug("writeTo metafile is " + metafile + " of size " + metafile.length()); return createBlockWriteStreams(f, metafile); }
/** * We're informed that a block is no longer valid. We could lazily garbage-collect the block, but * why bother? just get rid of it. */ public void invalidate(Block invalidBlks[]) throws IOException { boolean error = false; for (int i = 0; i < invalidBlks.length; i++) { File f = null; FSVolume v; synchronized (this) { f = getFile(invalidBlks[i]); DatanodeBlockInfo dinfo = volumeMap.get(invalidBlks[i]); if (dinfo == null) { DataNode.LOG.warn( "Unexpected error trying to delete block " + invalidBlks[i] + ". BlockInfo not found in volumeMap."); error = true; continue; } v = dinfo.getVolume(); if (f == null) { DataNode.LOG.warn( "Unexpected error trying to delete block " + invalidBlks[i] + ". Block not found in blockMap." + ((v == null) ? " " : " Block found in volumeMap.")); error = true; continue; } if (v == null) { DataNode.LOG.warn( "Unexpected error trying to delete block " + invalidBlks[i] + ". No volume for this block." + " Block found in blockMap. " + f + "."); error = true; continue; } File parent = f.getParentFile(); if (parent == null) { DataNode.LOG.warn( "Unexpected error trying to delete block " + invalidBlks[i] + ". Parent not found for file " + f + "."); error = true; continue; } v.clearPath(parent); volumeMap.remove(invalidBlks[i]); } File metaFile = getMetaFile(f, invalidBlks[i]); long blockSize = f.length() + metaFile.length(); if (!f.delete() || (!metaFile.delete() && metaFile.exists())) { DataNode.LOG.warn( "Unexpected error trying to delete block " + invalidBlks[i] + " at file " + f); error = true; continue; } v.decDfsUsed(blockSize); DataNode.LOG.info("Deleting block " + invalidBlks[i] + " file " + f); if (f.exists()) { // // This is a temporary check especially for hadoop-1220. // This will go away in the future. // DataNode.LOG.info("File " + f + " was deleted but still exists!"); } } if (error) { throw new IOException("Error in deleting blocks."); } }