/** * Make sure at least one non-transient volume has a saved copy of the replica. An infinite loop * is used to ensure the async lazy persist tasks are completely done before verification. Caller * of ensureLazyPersistBlocksAreSaved expects either a successful pass or timeout failure. */ protected final void ensureLazyPersistBlocksAreSaved(LocatedBlocks locatedBlocks) throws IOException, InterruptedException { final String bpid = cluster.getNamesystem().getBlockPoolId(); List<? extends FsVolumeSpi> volumes = cluster.getDataNodes().get(0).getFSDataset().getVolumes(); final Set<Long> persistedBlockIds = new HashSet<Long>(); while (persistedBlockIds.size() < locatedBlocks.getLocatedBlocks().size()) { // Take 1 second sleep before each verification iteration Thread.sleep(1000); for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) { for (FsVolumeSpi v : volumes) { if (v.isTransientStorage()) { continue; } FsVolumeImpl volume = (FsVolumeImpl) v; File lazyPersistDir = volume.getBlockPoolSlice(bpid).getLazypersistDir(); long blockId = lb.getBlock().getBlockId(); File targetDir = DatanodeUtil.idToBlockDir(lazyPersistDir, blockId); File blockFile = new File(targetDir, lb.getBlock().getBlockName()); if (blockFile.exists()) { // Found a persisted copy for this block and added to the Set persistedBlockIds.add(blockId); } } } } // We should have found a persisted copy for each located block. assertThat(persistedBlockIds.size(), is(locatedBlocks.getLocatedBlocks().size())); }
/* * Since NameNode will not persist any locations of the block, addBlock() * retry call after restart NN should re-select the locations and return to * client. refer HDFS-5257 */ @Test public void testAddBlockRetryShouldReturnBlockWithLocations() throws Exception { final String src = "/testAddBlockRetryShouldReturnBlockWithLocations"; NamenodeProtocols nameNodeRpc = cluster.getNameNodeRpc(); // create file nameNodeRpc.create( src, FsPermission.getFileDefault(), "clientName", new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true, (short) 3, 1024, null); // start first addBlock() LOG.info("Starting first addBlock for " + src); LocatedBlock lb1 = nameNodeRpc.addBlock(src, "clientName", null, null, INodeId.GRANDFATHER_INODE_ID, null); assertTrue("Block locations should be present", lb1.getLocations().length > 0); cluster.restartNameNode(); nameNodeRpc = cluster.getNameNodeRpc(); LocatedBlock lb2 = nameNodeRpc.addBlock(src, "clientName", null, null, INodeId.GRANDFATHER_INODE_ID, null); assertEquals("Blocks are not equal", lb1.getBlock(), lb2.getBlock()); assertTrue("Wrong locations with retry", lb2.getLocations().length > 0); }
private DNAddrPair chooseDataNode(LocatedBlock block) throws IOException { while (true) { DatanodeInfo[] nodes = block.getLocations(); try { DatanodeInfo chosenNode = bestNode(nodes, deadNodes); InetSocketAddress targetAddr = NetUtils.createSocketAddr(chosenNode.getXferAddr()); return new DNAddrPair(chosenNode, targetAddr); } catch (IOException ie) { String blockInfo = block.getBlock() + " file=" + src; if (failures >= dfsClient.getMaxBlockAcquireFailures()) { throw new BlockMissingException( src, "Could not obtain block: " + blockInfo, block.getStartOffset()); } if (nodes == null || nodes.length == 0) { DFSClient.LOG.info("No node available for block: " + blockInfo); } DFSClient.LOG.info( "Could not obtain block " + block.getBlock() + " from any node: " + ie + ". Will get new block locations from namenode and retry..."); try { // Introducing a random factor to the wait time before another retry. // The wait time is dependent on # of failures and a random factor. // At the first time of getting a BlockMissingException, the wait time // is a random number between 0..3000 ms. If the first retry // still fails, we will wait 3000 ms grace period before the 2nd retry. // Also at the second retry, the waiting window is expanded to 6000 ms // alleviating the request rate from the server. Similarly the 3rd retry // will wait 6000ms grace period before retry and the waiting window is // expanded to 9000ms. double waitTime = timeWindow * failures + // grace period for the last round of attempt timeWindow * (failures + 1) * DFSUtil.getRandom().nextDouble(); // expanding time window for each failure DFSClient.LOG.warn( "DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec."); Thread.sleep((long) waitTime); } catch (InterruptedException iex) { } deadNodes.clear(); // 2nd option is to remove only nodes[blockId] openInfo(); block = getBlockAt(block.getStartOffset(), false); failures++; continue; } } }
private void addBlocks(String fileName, String clientName) throws IOException { for (int jdx = 0; jdx < blocksPerFile; jdx++) { LocatedBlock loc = nameNode.addBlock(fileName, clientName); for (DatanodeInfo dnInfo : loc.getLocations()) { int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getName()); datanodes[dnIdx].addBlock(loc.getBlock()); nameNode.blockReceived( datanodes[dnIdx].dnRegistration, new Block[] {loc.getBlock()}, new String[] {""}); } } }
/** Read the block length from one of the datanodes. */ private long readBlockLength(LocatedBlock locatedblock) throws IOException { assert locatedblock != null : "LocatedBlock cannot be null"; int replicaNotFoundCount = locatedblock.getLocations().length; for (DatanodeInfo datanode : locatedblock.getLocations()) { ClientDatanodeProtocol cdp = null; try { cdp = DFSUtil.createClientDatanodeProtocolProxy( datanode, dfsClient.conf, dfsClient.getConf().socketTimeout, locatedblock); final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock()); if (n >= 0) { return n; } } catch (IOException ioe) { if (ioe instanceof RemoteException && (((RemoteException) ioe).unwrapRemoteException() instanceof ReplicaNotFoundException)) { // special case : replica might not be on the DN, treat as 0 length replicaNotFoundCount--; } if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug( "Failed to getReplicaVisibleLength from datanode " + datanode + " for block " + locatedblock.getBlock(), ioe); } } finally { if (cdp != null) { RPC.stopProxy(cdp); } } } // Namenode told us about these locations, but none know about the replica // means that we hit the race between pipeline creation start and end. // we require all 3 because some other exception could have happened // on a DN that has it. we want to report that error if (replicaNotFoundCount == 0) { return 0; } throw new IOException("Cannot obtain block length for " + locatedblock); }
/** Test that the {@link PlacementMonitor.BlockAndDatanodeResolver} works correctly. */ @Test public void testBlockAndDatanodeResolver() throws Exception { setupCluster(); try { Path src = new Path("/dir/file"); Path parity = new Path("/raid/dir/file"); DFSTestUtil.createFile(fs, src, 20, (short) 2, 0L); DFSTestUtil.createFile(fs, parity, 11, (short) 2, 0L); DFSTestUtil.waitReplication(fs, src, (short) 2); DFSTestUtil.waitReplication(fs, parity, (short) 2); LocatedBlocks srcLbs, parityLbs; List<BlockInfo> srcInfos, parityInfos; srcLbs = namenode.getBlockLocations(src.toString(), 4, 10); srcInfos = placementMonitor.getBlockInfos(fs, src, 4, 10); parityLbs = namenode.getBlockLocations(parity.toString(), 3, 7); parityInfos = placementMonitor.getBlockInfos(fs, parity, 3, 7); Assert.assertEquals(10, srcLbs.getLocatedBlocks().size()); Assert.assertEquals(7, parityLbs.getLocatedBlocks().size()); Assert.assertEquals(10, srcInfos.size()); Assert.assertEquals(7, parityInfos.size()); BlockAndDatanodeResolver resolver = new BlockAndDatanodeResolver(src, fs, parity, fs); for (int i = 0; i < srcInfos.size(); ++i) { LocatedBlock lb = resolver.getLocatedBlock(srcInfos.get(i)); Assert.assertEquals(srcLbs.get(i).getBlock(), lb.getBlock()); for (String nodeName : srcInfos.get(i).getNames()) { DatanodeInfo node = resolver.getDatanodeInfo(nodeName); Assert.assertEquals(node.getName(), nodeName); } } for (int i = 0; i < parityInfos.size(); ++i) { LocatedBlock lb = resolver.getLocatedBlock(parityInfos.get(i)); Assert.assertEquals(parityLbs.get(i).getBlock(), lb.getBlock()); for (String nodeName : parityInfos.get(i).getNames()) { DatanodeInfo node = resolver.getDatanodeInfo(nodeName); Assert.assertEquals(node.getName(), nodeName); } } } finally { if (cluster != null) { cluster.shutdown(); } if (placementMonitor != null) { placementMonitor.stop(); } } }
static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy( DatanodeID datanodeid, Configuration conf, int socketTimeout, boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException { final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname); InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr); if (LOG.isDebugEnabled()) { LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr); } // Since we're creating a new UserGroupInformation here, we know that no // future RPC proxies will be able to re-use the same connection. And // usages of this proxy tend to be one-off calls. // // This is a temporary fix: callers should really achieve this by using // RPC.stopProxy() on the resulting object, but this is currently not // working in trunk. See the discussion on HDFS-1965. Configuration confWithNoIpcIdle = new Configuration(conf); confWithNoIpcIdle.setInt( CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0); UserGroupInformation ticket = UserGroupInformation.createRemoteUser(locatedBlock.getBlock().getLocalBlock().toString()); ticket.addToken(locatedBlock.getBlockToken()); return createClientDatanodeProtocolProxy( addr, ticket, confWithNoIpcIdle, NetUtils.getDefaultSocketFactory(conf), socketTimeout); }
/** removes a specified block from MiniDFS storage and reports it as corrupt */ private void removeAndReportBlock( DistributedFileSystem blockDfs, Path filePath, LocatedBlock block) throws IOException { TestRaidDfs.corruptBlock(filePath, block.getBlock(), NUM_DATANODES, true); // report deleted block to the name node LocatedBlock[] toReport = {block}; blockDfs.getClient().namenode.reportBadBlocks(toReport); }
/** * Setup a {@link MiniDFSCluster}. Create a block with both {@link State#NORMAL} and {@link * State#READ_ONLY_SHARED} replicas. */ @Before public void setup() throws IOException, InterruptedException { conf = new HdfsConfiguration(); SimulatedFSDataset.setFactory(conf); Configuration[] overlays = new Configuration[NUM_DATANODES]; for (int i = 0; i < overlays.length; i++) { overlays[i] = new Configuration(); if (i == RO_NODE_INDEX) { overlays[i].setEnum( SimulatedFSDataset.CONFIG_PROPERTY_STATE, i == RO_NODE_INDEX ? READ_ONLY_SHARED : NORMAL); } } cluster = new MiniDFSCluster.Builder(conf) .numDataNodes(NUM_DATANODES) .dataNodeConfOverlays(overlays) .build(); fs = cluster.getFileSystem(); blockManager = cluster.getNameNode().getNamesystem().getBlockManager(); datanodeManager = blockManager.getDatanodeManager(); client = new DFSClient( new InetSocketAddress("localhost", cluster.getNameNodePort()), cluster.getConfiguration(0)); for (int i = 0; i < NUM_DATANODES; i++) { DataNode dataNode = cluster.getDataNodes().get(i); validateStorageState( BlockManagerTestUtil.getStorageReportsForDatanode( datanodeManager.getDatanode(dataNode.getDatanodeId())), i == RO_NODE_INDEX ? READ_ONLY_SHARED : NORMAL); } // Create a 1 block file DFSTestUtil.createFile(fs, PATH, BLOCK_SIZE, BLOCK_SIZE, BLOCK_SIZE, (short) 1, seed); LocatedBlock locatedBlock = getLocatedBlock(); extendedBlock = locatedBlock.getBlock(); block = extendedBlock.getLocalBlock(); assertThat(locatedBlock.getLocations().length, is(1)); normalDataNode = locatedBlock.getLocations()[0]; readOnlyDataNode = datanodeManager.getDatanode(cluster.getDataNodes().get(RO_NODE_INDEX).getDatanodeId()); assertThat(normalDataNode, is(not(readOnlyDataNode))); validateNumberReplicas(1); // Inject the block into the datanode with READ_ONLY_SHARED storage cluster.injectBlocks(0, RO_NODE_INDEX, Collections.singleton(block)); // There should now be 2 *locations* for the block // Must wait until the NameNode has processed the block report for the injected blocks waitForLocations(2); }
/** * The following test first creates a file. It verifies the block information from a datanode. * Then, it updates the block with new information and verifies again. */ @Test public void testBlockMetaDataInfo() throws Exception { MiniDFSCluster cluster = null; try { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); cluster.waitActive(); // create a file DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem(); String filestr = "/foo"; Path filepath = new Path(filestr); DFSTestUtil.createFile(dfs, filepath, 1024L, (short) 3, 0L); assertTrue(dfs.exists(filepath)); // get block info LocatedBlock locatedblock = getLastLocatedBlock(DFSClientAdapter.getDFSClient(dfs).getNamenode(), filestr); DatanodeInfo[] datanodeinfo = locatedblock.getLocations(); assertTrue(datanodeinfo.length > 0); // connect to a data node DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort()); InterDatanodeProtocol idp = DataNodeTestUtils.createInterDatanodeProtocolProxy(datanode, datanodeinfo[0], conf); // stop block scanner, so we could compare lastScanTime DataNodeTestUtils.shutdownBlockScanner(datanode); // verify BlockMetaDataInfo ExtendedBlock b = locatedblock.getBlock(); InterDatanodeProtocol.LOG.info("b=" + b + ", " + b.getClass()); checkMetaInfo(b, datanode); long recoveryId = b.getGenerationStamp() + 1; idp.initReplicaRecovery(new RecoveringBlock(b, locatedblock.getLocations(), recoveryId)); // verify updateBlock ExtendedBlock newblock = new ExtendedBlock( b.getBlockPoolId(), b.getBlockId(), b.getNumBytes() / 2, b.getGenerationStamp() + 1); idp.updateReplicaUnderRecovery(b, recoveryId, newblock.getNumBytes()); checkMetaInfo(newblock, datanode); // Verify correct null response trying to init recovery for a missing block ExtendedBlock badBlock = new ExtendedBlock("fake-pool", b.getBlockId(), 0, 0); assertNull( idp.initReplicaRecovery( new RecoveringBlock(badBlock, locatedblock.getLocations(), recoveryId))); } finally { if (cluster != null) { cluster.shutdown(); } } }
/** * TC11: Racing rename * * @throws IOException an exception might be thrown */ public void testTC11() throws Exception { final Path p = new Path("/TC11/foo"); System.out.println("p=" + p); // a. Create file and write one block of data. Close file. final int len1 = (int) BLOCK_SIZE; { FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION, BLOCK_SIZE); AppendTestUtil.write(out, 0, len1); out.close(); } // b. Reopen file in "append" mode. Append half block of data. FSDataOutputStream out = fs.append(p); final int len2 = (int) BLOCK_SIZE / 2; AppendTestUtil.write(out, len1, len2); out.hflush(); // c. Rename file to file.new. final Path pnew = new Path(p + ".new"); assertTrue(fs.rename(p, pnew)); // d. Close file handle that was opened in (b). try { out.close(); fail("close() should throw an exception"); } catch (Exception e) { AppendTestUtil.LOG.info("GOOD!", e); } // wait for the lease recovery cluster.setLeasePeriod(1000, 1000); AppendTestUtil.sleep(5000); // check block sizes final long len = fs.getFileStatus(pnew).getLen(); final LocatedBlocks locatedblocks = fs.dfs.getNamenode().getBlockLocations(pnew.toString(), 0L, len); final int numblock = locatedblocks.locatedBlockCount(); for (int i = 0; i < numblock; i++) { final LocatedBlock lb = locatedblocks.get(i); final Block blk = lb.getBlock(); final long size = lb.getBlockSize(); if (i < numblock - 1) { assertEquals(BLOCK_SIZE, size); } for (DatanodeInfo datanodeinfo : lb.getLocations()) { final DataNode dn = cluster.getDataNode(datanodeinfo.getIpcPort()); final Block metainfo = dn.data.getStoredBlock(blk.getBlockId()); assertEquals(size, metainfo.getNumBytes()); } } }
protected final boolean verifyBlockDeletedFromDir(File dir, LocatedBlocks locatedBlocks) { for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) { File targetDir = DatanodeUtil.idToBlockDir(dir, lb.getBlock().getBlockId()); File blockFile = new File(targetDir, lb.getBlock().getBlockName()); if (blockFile.exists()) { LOG.warn("blockFile: " + blockFile.getAbsolutePath() + " exists after deletion."); return false; } File metaFile = new File( targetDir, DatanodeUtil.getMetaName( lb.getBlock().getBlockName(), lb.getBlock().getGenerationStamp())); if (metaFile.exists()) { LOG.warn("metaFile: " + metaFile.getAbsolutePath() + " exists after deletion."); return false; } } return true; }
/** Convert a LocatedBlock to a Json map. */ private static Map<String, Object> toJsonMap(final LocatedBlock locatedblock) throws IOException { if (locatedblock == null) { return null; } final Map<String, Object> m = new TreeMap<String, Object>(); m.put("blockToken", toJsonMap(locatedblock.getBlockToken())); m.put("isCorrupt", locatedblock.isCorrupt()); m.put("startOffset", locatedblock.getStartOffset()); m.put("block", toJsonMap(locatedblock.getBlock())); m.put("locations", toJsonArray(locatedblock.getLocations())); return m; }
/** * try to access a block on a data node. If fails - throws exception * * @param datanode * @param lblock * @throws IOException */ private void accessBlock(DatanodeInfo datanode, LocatedBlock lblock) throws IOException { InetSocketAddress targetAddr = null; ExtendedBlock block = lblock.getBlock(); targetAddr = NetUtils.createSocketAddr(datanode.getXferAddr()); BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)) .setInetSocketAddress(targetAddr) .setBlock(block) .setFileName( BlockReaderFactory.getFileName(targetAddr, "test-blockpoolid", block.getBlockId())) .setBlockToken(lblock.getBlockToken()) .setStartOffset(0) .setLength(-1) .setVerifyChecksum(true) .setClientName("TestDataNodeVolumeFailure") .setDatanodeInfo(datanode) .setCachingStrategy(CachingStrategy.newDefaultStrategy()) .setClientCacheContext(ClientContext.getFromConf(conf)) .setConfiguration(conf) .setTracer(FsTracer.get(conf)) .setRemotePeerFactory( new RemotePeerFactory() { @Override public Peer newConnectedPeer( InetSocketAddress addr, Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId) throws IOException { Peer peer = null; Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket(); try { sock.connect(addr, HdfsConstants.READ_TIMEOUT); sock.setSoTimeout(HdfsConstants.READ_TIMEOUT); peer = DFSUtilClient.peerFromSocket(sock); } finally { if (peer == null) { IOUtils.closeSocket(sock); } } return peer; } }) .build(); blockReader.close(); }
/** * go to each block on the 2nd DataNode until it fails... * * @param path * @param size * @throws IOException */ private void triggerFailure(String path, long size) throws IOException { NamenodeProtocols nn = cluster.getNameNodeRpc(); List<LocatedBlock> locatedBlocks = nn.getBlockLocations(path, 0, size).getLocatedBlocks(); for (LocatedBlock lb : locatedBlocks) { DatanodeInfo dinfo = lb.getLocations()[1]; ExtendedBlock b = lb.getBlock(); try { accessBlock(dinfo, lb); } catch (IOException e) { System.out.println( "Failure triggered, on block: " + b.getBlockId() + "; corresponding volume should be removed by now"); break; } } }
/** * TC7: Corrupted replicas are present. * * @throws IOException an exception might be thrown */ public void testTC7() throws Exception { final short repl = 2; final Path p = new Path("/TC7/foo"); System.out.println("p=" + p); // a. Create file with replication factor of 2. Write half block of data. Close file. final int len1 = (int) (BLOCK_SIZE / 2); { FSDataOutputStream out = fs.create(p, false, buffersize, repl, BLOCK_SIZE); AppendTestUtil.write(out, 0, len1); out.close(); } DFSTestUtil.waitReplication(fs, p, repl); // b. Log into one datanode that has one replica of this block. // Find the block file on this datanode and truncate it to zero size. final LocatedBlocks locatedblocks = fs.dfs.getNamenode().getBlockLocations(p.toString(), 0L, len1); assertEquals(1, locatedblocks.locatedBlockCount()); final LocatedBlock lb = locatedblocks.get(0); final Block blk = lb.getBlock(); assertEquals(len1, lb.getBlockSize()); DatanodeInfo[] datanodeinfos = lb.getLocations(); assertEquals(repl, datanodeinfos.length); final DataNode dn = cluster.getDataNode(datanodeinfos[0].getIpcPort()); final FSDataset data = (FSDataset) dn.getFSDataset(); final RandomAccessFile raf = new RandomAccessFile(data.getBlockFile(blk), "rw"); AppendTestUtil.LOG.info("dn=" + dn + ", blk=" + blk + " (length=" + blk.getNumBytes() + ")"); assertEquals(len1, raf.length()); raf.setLength(0); raf.close(); // c. Open file in "append mode". Append a new block worth of data. Close file. final int len2 = (int) BLOCK_SIZE; { FSDataOutputStream out = fs.append(p); AppendTestUtil.write(out, len1, len2); out.close(); } // d. Reopen file and read two blocks worth of data. AppendTestUtil.check(fs, p, len1 + len2); }
private int printLocatedBlocks(Path filePath) throws Exception { LocatedBlocks lbs = dfs.getLocatedBlocks(filePath, 0L, Integer.MAX_VALUE); StringBuilder sb = new StringBuilder(); sb.append("Path " + filePath + ":"); int maxRepl = 0; for (LocatedBlock lb : lbs.getLocatedBlocks()) { sb.append(lb.getBlock()); sb.append(":"); for (DatanodeInfo loc : lb.getLocations()) { sb.append(loc.getHostName()); sb.append(" "); } if (lb.getLocations().length > maxRepl) { maxRepl = lb.getLocations().length; } } LOG.info(sb.toString()); return maxRepl; }
public void testAbandonBlock() throws IOException { MiniDFSCluster cluster = new MiniDFSCluster(CONF, 2, true, null); FileSystem fs = cluster.getFileSystem(); String src = FILE_NAME_PREFIX + "foo"; FSDataOutputStream fout = null; try { // start writing a a file but not close it fout = fs.create(new Path(src), true, 4096, (short) 1, 512L); for (int i = 0; i < 1024; i++) { fout.write(123); } fout.sync(); // try reading the block by someone final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF), CONF); LocatedBlocks blocks = dfsclient.namenode.getBlockLocations(src, 0, 1); LocatedBlock b = blocks.get(0); try { dfsclient.namenode.abandonBlock(b.getBlock(), src, "someone"); // previous line should throw an exception. assertTrue(false); } catch (IOException ioe) { LOG.info("GREAT! " + StringUtils.stringifyException(ioe)); } } finally { try { fout.close(); } catch (Exception e) { } try { fs.close(); } catch (Exception e) { } try { cluster.shutdown(); } catch (Exception e) { } } }
/** Create a file with one block and corrupt some/all of the block replicas. */ private void createAFileWithCorruptedBlockReplicas( Path filePath, short repl, int corruptBlockCount) throws IOException, AccessControlException, FileNotFoundException, UnresolvedLinkException, InterruptedException, TimeoutException { DFSTestUtil.createFile(dfs, filePath, BLOCK_SIZE, repl, 0); DFSTestUtil.waitReplication(dfs, filePath, repl); // Locate the file blocks by asking name node final LocatedBlocks locatedblocks = dfs.dfs.getNamenode().getBlockLocations(filePath.toString(), 0L, BLOCK_SIZE); Assert.assertEquals(repl, locatedblocks.get(0).getLocations().length); // The file only has one block LocatedBlock lblock = locatedblocks.get(0); DatanodeInfo[] datanodeinfos = lblock.getLocations(); ExtendedBlock block = lblock.getBlock(); // corrupt some /all of the block replicas for (int i = 0; i < corruptBlockCount; i++) { DatanodeInfo dninfo = datanodeinfos[i]; final DataNode dn = cluster.getDataNode(dninfo.getIpcPort()); corruptBlock(block, dn); LOG.debug("Corrupted block " + block.getBlockName() + " on data node " + dninfo); } }
/** * Count datanodes that have copies of the blocks for a file put it into the map * * @param map * @param path * @param size * @return * @throws IOException */ private int countNNBlocks(Map<String, BlockLocs> map, String path, long size) throws IOException { int total = 0; NamenodeProtocols nn = cluster.getNameNodeRpc(); List<LocatedBlock> locatedBlocks = nn.getBlockLocations(path, 0, size).getLocatedBlocks(); // System.out.println("Number of blocks: " + locatedBlocks.size()); for (LocatedBlock lb : locatedBlocks) { String blockId = "" + lb.getBlock().getBlockId(); // System.out.print(blockId + ": "); DatanodeInfo[] dn_locs = lb.getLocations(); BlockLocs bl = map.get(blockId); if (bl == null) { bl = new BlockLocs(); } // System.out.print(dn_info.name+","); total += dn_locs.length; bl.num_locs += dn_locs.length; map.put(blockId, bl); // System.out.println(); } return total; }
/** Get a BlockReader for the given block. */ public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead) throws IOException { InetSocketAddress targetAddr = null; Socket sock = null; ExtendedBlock block = testBlock.getBlock(); DatanodeInfo[] nodes = testBlock.getLocations(); targetAddr = NetUtils.createSocketAddr(nodes[0].getName()); sock = NetUtils.getDefaultSocketFactory(conf).createSocket(); sock.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT); sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); return BlockReaderFactory.newBlockReader( new DFSClient.Conf(conf), sock, targetAddr.toString() + ":" + block.getBlockId(), block, testBlock.getBlockToken(), offset, lenToRead, conf.getInt("io.file.buffer.size", 4096), true, ""); }
private long fetchLocatedBlocksAndGetLastBlockLength() throws IOException { LocatedBlocks newInfo = DFSClient.callGetBlockLocations(dfsClient.namenode, src, 0, prefetchSize); if (DFSClient.LOG.isDebugEnabled()) { DFSClient.LOG.debug("newInfo = " + newInfo); } if (newInfo == null) { throw new IOException("Cannot open filename " + src); } if (locatedBlocks != null) { Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator(); Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator(); while (oldIter.hasNext() && newIter.hasNext()) { if (!oldIter.next().getBlock().equals(newIter.next().getBlock())) { throw new IOException("Blocklist for " + src + " has changed!"); } } } locatedBlocks = newInfo; long lastBlockBeingWrittenLength = 0; if (!locatedBlocks.isLastBlockComplete()) { final LocatedBlock last = locatedBlocks.getLastLocatedBlock(); if (last != null) { if (last.getLocations().length == 0) { return -1; } final long len = readBlockLength(last); last.getBlock().setNumBytes(len); lastBlockBeingWrittenLength = len; } } currentNode = null; return lastBlockBeingWrittenLength; }
// try reading a block using a BlockReader directly protected void tryRead(final Configuration conf, LocatedBlock lblock, boolean shouldSucceed) { InetSocketAddress targetAddr = null; IOException ioe = null; BlockReader blockReader = null; ExtendedBlock block = lblock.getBlock(); try { DatanodeInfo[] nodes = lblock.getLocations(); targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr()); blockReader = new BlockReaderFactory(new DfsClientConf(conf)) .setFileName( BlockReaderFactory.getFileName( targetAddr, "test-blockpoolid", block.getBlockId())) .setBlock(block) .setBlockToken(lblock.getBlockToken()) .setInetSocketAddress(targetAddr) .setStartOffset(0) .setLength(-1) .setVerifyChecksum(true) .setClientName("TestBlockTokenWithDFS") .setDatanodeInfo(nodes[0]) .setCachingStrategy(CachingStrategy.newDefaultStrategy()) .setClientCacheContext(ClientContext.getFromConf(conf)) .setConfiguration(conf) .setTracer(FsTracer.get(conf)) .setRemotePeerFactory( new RemotePeerFactory() { @Override public Peer newConnectedPeer( InetSocketAddress addr, Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId) throws IOException { Peer peer = null; Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket(); try { sock.connect(addr, HdfsConstants.READ_TIMEOUT); sock.setSoTimeout(HdfsConstants.READ_TIMEOUT); peer = DFSUtilClient.peerFromSocket(sock); } finally { if (peer == null) { IOUtils.closeSocket(sock); } } return peer; } }) .build(); } catch (IOException ex) { ioe = ex; } finally { if (blockReader != null) { try { blockReader.close(); } catch (IOException e) { throw new RuntimeException(e); } } } if (shouldSucceed) { Assert.assertNotNull( "OP_READ_BLOCK: access token is invalid, " + "when it is expected to be valid", blockReader); } else { Assert.assertNotNull( "OP_READ_BLOCK: access token is valid, " + "when it is expected to be invalid", ioe); Assert.assertTrue( "OP_READ_BLOCK failed due to reasons other than access token: ", ioe instanceof InvalidBlockTokenException); } }
/** * The following test first creates a file with a few blocks. It randomly truncates the replica of * the last block stored in each datanode. Finally, it triggers block synchronization to * synchronize all stored block. */ public void testBlockSynchronization() throws Exception { final int ORG_FILE_SIZE = 3000; Configuration conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); MiniDFSCluster cluster = null; try { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build(); cluster.waitActive(); // create a file DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem(); String filestr = "/foo"; Path filepath = new Path(filestr); DFSTestUtil.createFile(dfs, filepath, ORG_FILE_SIZE, REPLICATION_NUM, 0L); assertTrue(dfs.exists(filepath)); DFSTestUtil.waitReplication(dfs, filepath, REPLICATION_NUM); // get block info for the last block LocatedBlock locatedblock = TestInterDatanodeProtocol.getLastLocatedBlock(dfs.dfs.getNamenode(), filestr); DatanodeInfo[] datanodeinfos = locatedblock.getLocations(); assertEquals(REPLICATION_NUM, datanodeinfos.length); // connect to data nodes DataNode[] datanodes = new DataNode[REPLICATION_NUM]; for (int i = 0; i < REPLICATION_NUM; i++) { datanodes[i] = cluster.getDataNode(datanodeinfos[i].getIpcPort()); assertTrue(datanodes[i] != null); } // verify Block Info ExtendedBlock lastblock = locatedblock.getBlock(); DataNode.LOG.info("newblocks=" + lastblock); for (int i = 0; i < REPLICATION_NUM; i++) { checkMetaInfo(lastblock, datanodes[i]); } DataNode.LOG.info("dfs.dfs.clientName=" + dfs.dfs.clientName); cluster.getNameNodeRpc().append(filestr, dfs.dfs.clientName); // expire lease to trigger block recovery. waitLeaseRecovery(cluster); Block[] updatedmetainfo = new Block[REPLICATION_NUM]; long oldSize = lastblock.getNumBytes(); lastblock = TestInterDatanodeProtocol.getLastLocatedBlock(dfs.dfs.getNamenode(), filestr).getBlock(); long currentGS = lastblock.getGenerationStamp(); for (int i = 0; i < REPLICATION_NUM; i++) { updatedmetainfo[i] = DataNodeTestUtils.getFSDataset(datanodes[i]) .getStoredBlock(lastblock.getBlockPoolId(), lastblock.getBlockId()); assertEquals(lastblock.getBlockId(), updatedmetainfo[i].getBlockId()); assertEquals(oldSize, updatedmetainfo[i].getNumBytes()); assertEquals(currentGS, updatedmetainfo[i].getGenerationStamp()); } // verify that lease recovery does not occur when namenode is in safemode System.out.println("Testing that lease recovery cannot happen during safemode."); filestr = "/foo.safemode"; filepath = new Path(filestr); dfs.create(filepath, (short) 1); cluster.getNameNodeRpc().setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER); assertTrue(dfs.dfs.exists(filestr)); DFSTestUtil.waitReplication(dfs, filepath, (short) 1); waitLeaseRecovery(cluster); // verify that we still cannot recover the lease LeaseManager lm = NameNodeAdapter.getLeaseManager(cluster.getNamesystem()); assertTrue("Found " + lm.countLease() + " lease, expected 1", lm.countLease() == 1); cluster.getNameNodeRpc().setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE); } finally { if (cluster != null) { cluster.shutdown(); } } }
/* * XXX (ab) Bulk of this method is copied verbatim from {@link DFSClient}, which is * bad. Both places should be refactored to provide a method to copy blocks * around. */ private void copyBlock(DFSClient dfs, LocatedBlock lblock, OutputStream fos) throws Exception { int failures = 0; InetSocketAddress targetAddr = null; TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>(); BlockReader blockReader = null; ExtendedBlock block = lblock.getBlock(); while (blockReader == null) { DatanodeInfo chosenNode; try { chosenNode = bestNode(dfs, lblock.getLocations(), deadNodes); targetAddr = NetUtils.createSocketAddr(chosenNode.getXferAddr()); } catch (IOException ie) { if (failures >= DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT) { throw new IOException("Could not obtain block " + lblock, ie); } LOG.info("Could not obtain block from any node: " + ie); try { Thread.sleep(10000); } catch (InterruptedException iex) { } deadNodes.clear(); failures++; continue; } try { String file = BlockReaderFactory.getFileName(targetAddr, block.getBlockPoolId(), block.getBlockId()); blockReader = new BlockReaderFactory(dfs.getConf()) .setFileName(file) .setBlock(block) .setBlockToken(lblock.getBlockToken()) .setStartOffset(0) .setLength(-1) .setVerifyChecksum(true) .setClientName("fsck") .setDatanodeInfo(chosenNode) .setInetSocketAddress(targetAddr) .setCachingStrategy(CachingStrategy.newDropBehind()) .setClientCacheContext(dfs.getClientContext()) .setConfiguration(namenode.conf) .setRemotePeerFactory( new RemotePeerFactory() { @Override public Peer newConnectedPeer(InetSocketAddress addr) throws IOException { Peer peer = null; Socket s = NetUtils.getDefaultSocketFactory(conf).createSocket(); try { s.connect(addr, HdfsServerConstants.READ_TIMEOUT); s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); peer = TcpPeerServer.peerFromSocketAndKey( s, namenode.getRpcServer().getDataEncryptionKey()); } finally { if (peer == null) { IOUtils.closeQuietly(s); } } return peer; } }) .build(); } catch (IOException ex) { // Put chosen node into dead list, continue LOG.info("Failed to connect to " + targetAddr + ":" + ex); deadNodes.add(chosenNode); } } byte[] buf = new byte[1024]; int cnt = 0; boolean success = true; long bytesRead = 0; try { while ((cnt = blockReader.read(buf, 0, buf.length)) > 0) { fos.write(buf, 0, cnt); bytesRead += cnt; } if (bytesRead != block.getNumBytes()) { throw new IOException( "Recorded block size is " + block.getNumBytes() + ", but datanode returned " + bytesRead + " bytes"); } } catch (Exception e) { LOG.error("Error reading block", e); success = false; } finally { blockReader.close(); } if (!success) { throw new Exception("Could not copy block data for " + lblock.getBlock()); } }
/** * Create a file, write something, fsync but not close. Then change lease period and wait for * lease recovery. Finally, read the block directly from each Datanode and verify the content. */ public void testLeaseExpireHardLimit() throws Exception { System.out.println("testLeaseExpireHardLimit start"); final long leasePeriod = 1000; final int DATANODE_NUM = 3; Configuration conf = new Configuration(); conf.setInt("heartbeat.recheck.interval", 1000); conf.setInt("dfs.heartbeat.interval", 1); // create cluster MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null); DistributedFileSystem dfs = null; try { cluster.waitActive(); dfs = (DistributedFileSystem) cluster.getFileSystem(); // create a new file. final String f = DIR + "foo"; final Path fpath = new Path(f); FSDataOutputStream out = TestFileCreation.createFile(dfs, fpath, DATANODE_NUM); out.write("something".getBytes()); out.sync(); int actualRepl = ((DFSClient.DFSOutputStream) (out.getWrappedStream())).getNumCurrentReplicas(); assertTrue( f + " should be replicated to " + DATANODE_NUM + " datanodes.", actualRepl == DATANODE_NUM); // set the soft and hard limit to be 1 second so that the // namenode triggers lease recovery cluster.setLeasePeriod(leasePeriod, leasePeriod); // wait for the lease to expire try { Thread.sleep(5 * leasePeriod); } catch (InterruptedException e) { } LocatedBlocks locations = dfs.dfs.namenode.getBlockLocations(f, 0, Long.MAX_VALUE); assertEquals(1, locations.locatedBlockCount()); LocatedBlock locatedblock = locations.getLocatedBlocks().get(0); int successcount = 0; for (DatanodeInfo datanodeinfo : locatedblock.getLocations()) { DataNode datanode = cluster.getDataNode(datanodeinfo.ipcPort); FSDataset dataset = (FSDataset) datanode.data; Block b = dataset.getStoredBlock(locatedblock.getBlock().getBlockId()); File blockfile = dataset.findBlockFile(b.getBlockId()); System.out.println("blockfile=" + blockfile); if (blockfile != null) { BufferedReader in = new BufferedReader(new FileReader(blockfile)); assertEquals("something", in.readLine()); in.close(); successcount++; } } System.out.println("successcount=" + successcount); assertTrue(successcount > 0); } finally { IOUtils.closeStream(dfs); cluster.shutdown(); } System.out.println("testLeaseExpireHardLimit successful"); }
/** Test that the filesystem removes the last block from a file if its lease expires. */ public void testFileCreationError2() throws IOException { long leasePeriod = 1000; System.out.println("testFileCreationError2 start"); Configuration conf = new Configuration(); conf.setInt("heartbeat.recheck.interval", 1000); conf.setInt("dfs.heartbeat.interval", 1); if (simulatedStorage) { conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true); } // create cluster MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null); DistributedFileSystem dfs = null; try { cluster.waitActive(); dfs = (DistributedFileSystem) cluster.getFileSystem(); DFSClient client = dfs.dfs; // create a new file. // Path file1 = new Path("/filestatus.dat"); createFile(dfs, file1, 1); System.out.println( "testFileCreationError2: " + "Created file filestatus.dat with one replicas."); LocatedBlocks locations = client.namenode.getBlockLocations(file1.toString(), 0, Long.MAX_VALUE); System.out.println( "testFileCreationError2: " + "The file has " + locations.locatedBlockCount() + " blocks."); // add another block to the file LocatedBlock location = client.namenode.addBlock(file1.toString(), client.clientName); System.out.println("testFileCreationError2: " + "Added block " + location.getBlock()); locations = client.namenode.getBlockLocations(file1.toString(), 0, Long.MAX_VALUE); int count = locations.locatedBlockCount(); System.out.println("testFileCreationError2: " + "The file now has " + count + " blocks."); // set the soft and hard limit to be 1 second so that the // namenode triggers lease recovery cluster.setLeasePeriod(leasePeriod, leasePeriod); // wait for the lease to expire try { Thread.sleep(5 * leasePeriod); } catch (InterruptedException e) { } // verify that the last block was synchronized. locations = client.namenode.getBlockLocations(file1.toString(), 0, Long.MAX_VALUE); System.out.println( "testFileCreationError2: " + "locations = " + locations.locatedBlockCount()); assertEquals(0, locations.locatedBlockCount()); System.out.println("testFileCreationError2 successful"); } finally { IOUtils.closeStream(dfs); cluster.shutdown(); } }
/** Retry addBlock() while another thread is in chooseTarget(). See HDFS-4452. */ @Test public void testRetryAddBlockWhileInChooseTarget() throws Exception { final String src = "/testRetryAddBlockWhileInChooseTarget"; FSNamesystem ns = cluster.getNamesystem(); BlockManager spyBM = spy(ns.getBlockManager()); final NamenodeProtocols nn = cluster.getNameNodeRpc(); // substitute mocked BlockManager into FSNamesystem Class<? extends FSNamesystem> nsClass = ns.getClass(); Field bmField = nsClass.getDeclaredField("blockManager"); bmField.setAccessible(true); bmField.set(ns, spyBM); doAnswer( new Answer<DatanodeStorageInfo[]>() { @Override public DatanodeStorageInfo[] answer(InvocationOnMock invocation) throws Throwable { LOG.info("chooseTarget for " + src); DatanodeStorageInfo[] ret = (DatanodeStorageInfo[]) invocation.callRealMethod(); count++; if (count == 1) { // run second addBlock() LOG.info("Starting second addBlock for " + src); nn.addBlock(src, "clientName", null, null, INodeId.GRANDFATHER_INODE_ID, null); LocatedBlocks lbs = nn.getBlockLocations(src, 0, Long.MAX_VALUE); assertEquals("Must be one block", 1, lbs.getLocatedBlocks().size()); lb2 = lbs.get(0); assertEquals("Wrong replication", REPLICATION, lb2.getLocations().length); } return ret; } }) .when(spyBM) .chooseTarget4NewBlock( Mockito.anyString(), Mockito.anyInt(), Mockito.<DatanodeDescriptor>any(), Mockito.<HashSet<Node>>any(), Mockito.anyLong(), Mockito.<List<String>>any(), Mockito.anyByte()); // create file nn.create( src, FsPermission.getFileDefault(), "clientName", new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true, (short) 3, 1024, null); // start first addBlock() LOG.info("Starting first addBlock for " + src); nn.addBlock(src, "clientName", null, null, INodeId.GRANDFATHER_INODE_ID, null); // check locations LocatedBlocks lbs = nn.getBlockLocations(src, 0, Long.MAX_VALUE); assertEquals("Must be one block", 1, lbs.getLocatedBlocks().size()); lb1 = lbs.get(0); assertEquals("Wrong replication", REPLICATION, lb1.getLocations().length); assertEquals("Blocks are not equal", lb1.getBlock(), lb2.getBlock()); }
/** * Test that when there is a failure replicating a block the temporary and meta files are cleaned * up and subsequent replication succeeds. */ @Test public void testReplicationError() throws Exception { // create a file of replication factor of 1 final Path fileName = new Path("/test.txt"); final int fileLen = 1; DFSTestUtil.createFile(fs, fileName, 1, (short) 1, 1L); DFSTestUtil.waitReplication(fs, fileName, (short) 1); // get the block belonged to the created file LocatedBlocks blocks = NameNodeAdapter.getBlockLocations( cluster.getNameNode(), fileName.toString(), 0, (long) fileLen); assertEquals("Should only find 1 block", blocks.locatedBlockCount(), 1); LocatedBlock block = blocks.get(0); // bring up a second datanode cluster.startDataNodes(conf, 1, true, null, null); cluster.waitActive(); final int sndNode = 1; DataNode datanode = cluster.getDataNodes().get(sndNode); // replicate the block to the second datanode InetSocketAddress target = datanode.getXferAddress(); Socket s = new Socket(target.getAddress(), target.getPort()); // write the header. DataOutputStream out = new DataOutputStream(s.getOutputStream()); DataChecksum checksum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512); new Sender(out) .writeBlock( block.getBlock(), StorageType.DEFAULT, BlockTokenSecretManager.DUMMY_TOKEN, "", new DatanodeInfo[0], new StorageType[0], null, BlockConstructionStage.PIPELINE_SETUP_CREATE, 1, 0L, 0L, 0L, checksum, CachingStrategy.newDefaultStrategy(), false); out.flush(); // close the connection before sending the content of the block out.close(); // the temporary block & meta files should be deleted String bpid = cluster.getNamesystem().getBlockPoolId(); File storageDir = cluster.getInstanceStorageDir(sndNode, 0); File dir1 = MiniDFSCluster.getRbwDir(storageDir, bpid); storageDir = cluster.getInstanceStorageDir(sndNode, 1); File dir2 = MiniDFSCluster.getRbwDir(storageDir, bpid); while (dir1.listFiles().length != 0 || dir2.listFiles().length != 0) { Thread.sleep(100); } // then increase the file's replication factor fs.setReplication(fileName, (short) 2); // replication should succeed DFSTestUtil.waitReplication(fs, fileName, (short) 1); // clean up the file fs.delete(fileName, false); }
protected void doTestRead(Configuration conf, MiniDFSCluster cluster, boolean isStriped) throws Exception { final int numDataNodes = cluster.getDataNodes().size(); final NameNode nn = cluster.getNameNode(); final NamenodeProtocols nnProto = nn.getRpcServer(); final BlockManager bm = nn.getNamesystem().getBlockManager(); final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager(); // set a short token lifetime (1 second) initially SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); Path fileToRead = new Path(FILE_TO_READ); FileSystem fs = cluster.getFileSystem(); byte[] expected = generateBytes(FILE_SIZE); createFile(fs, fileToRead, expected); /* * setup for testing expiration handling of cached tokens */ // read using blockSeekTo(). Acquired tokens are cached in in1 FSDataInputStream in1 = fs.open(fileToRead); assertTrue(checkFile1(in1, expected)); // read using blockSeekTo(). Acquired tokens are cached in in2 FSDataInputStream in2 = fs.open(fileToRead); assertTrue(checkFile1(in2, expected)); // read using fetchBlockByteRange(). Acquired tokens are cached in in3 FSDataInputStream in3 = fs.open(fileToRead); assertTrue(checkFile2(in3, expected)); /* * testing READ interface on DN using a BlockReader */ DFSClient client = null; try { client = new DFSClient(new InetSocketAddress("localhost", cluster.getNameNodePort()), conf); } finally { if (client != null) client.close(); } List<LocatedBlock> locatedBlocks = nnProto.getBlockLocations(FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks(); LocatedBlock lblock = locatedBlocks.get(0); // first block // verify token is not expired assertFalse(isBlockTokenExpired(lblock)); // read with valid token, should succeed tryRead(conf, lblock, true); /* * wait till myToken and all cached tokens in in1, in2 and in3 expire */ while (!isBlockTokenExpired(lblock)) { try { Thread.sleep(10); } catch (InterruptedException ignored) { } } /* * continue testing READ interface on DN using a BlockReader */ // verify token is expired assertTrue(isBlockTokenExpired(lblock)); // read should fail tryRead(conf, lblock, false); // use a valid new token bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ); // read should succeed tryRead(conf, lblock, true); // use a token with wrong blockID long rightId = lblock.getBlock().getBlockId(); long wrongId = rightId + 1; lblock.getBlock().setBlockId(wrongId); bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ); lblock.getBlock().setBlockId(rightId); // read should fail tryRead(conf, lblock, false); // use a token with wrong access modes bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.WRITE); // read should fail tryRead(conf, lblock, false); // set a long token lifetime for future tokens SecurityTestUtil.setBlockTokenLifetime(sm, 600 * 1000L); /* * testing that when cached tokens are expired, DFSClient will re-fetch * tokens transparently for READ. */ // confirm all tokens cached in in1 are expired by now List<LocatedBlock> lblocks = DFSTestUtil.getAllBlocks(in1); for (LocatedBlock blk : lblocks) { assertTrue(isBlockTokenExpired(blk)); } // verify blockSeekTo() is able to re-fetch token transparently in1.seek(0); assertTrue(checkFile1(in1, expected)); // confirm all tokens cached in in2 are expired by now List<LocatedBlock> lblocks2 = DFSTestUtil.getAllBlocks(in2); for (LocatedBlock blk : lblocks2) { assertTrue(isBlockTokenExpired(blk)); } // verify blockSeekTo() is able to re-fetch token transparently (testing // via another interface method) if (isStriped) { // striped block doesn't support seekToNewSource in2.seek(0); } else { assertTrue(in2.seekToNewSource(0)); } assertTrue(checkFile1(in2, expected)); // confirm all tokens cached in in3 are expired by now List<LocatedBlock> lblocks3 = DFSTestUtil.getAllBlocks(in3); for (LocatedBlock blk : lblocks3) { assertTrue(isBlockTokenExpired(blk)); } // verify fetchBlockByteRange() is able to re-fetch token transparently assertTrue(checkFile2(in3, expected)); /* * testing that after datanodes are restarted on the same ports, cached * tokens should still work and there is no need to fetch new tokens from * namenode. This test should run while namenode is down (to make sure no * new tokens can be fetched from namenode). */ // restart datanodes on the same ports that they currently use assertTrue(cluster.restartDataNodes(true)); cluster.waitActive(); assertEquals(numDataNodes, cluster.getDataNodes().size()); cluster.shutdownNameNode(0); // confirm tokens cached in in1 are still valid lblocks = DFSTestUtil.getAllBlocks(in1); for (LocatedBlock blk : lblocks) { assertFalse(isBlockTokenExpired(blk)); } // verify blockSeekTo() still works (forced to use cached tokens) in1.seek(0); assertTrue(checkFile1(in1, expected)); // confirm tokens cached in in2 are still valid lblocks2 = DFSTestUtil.getAllBlocks(in2); for (LocatedBlock blk : lblocks2) { assertFalse(isBlockTokenExpired(blk)); } // verify blockSeekTo() still works (forced to use cached tokens) if (isStriped) { in2.seek(0); } else { in2.seekToNewSource(0); } assertTrue(checkFile1(in2, expected)); // confirm tokens cached in in3 are still valid lblocks3 = DFSTestUtil.getAllBlocks(in3); for (LocatedBlock blk : lblocks3) { assertFalse(isBlockTokenExpired(blk)); } // verify fetchBlockByteRange() still works (forced to use cached tokens) assertTrue(checkFile2(in3, expected)); /* * testing that when namenode is restarted, cached tokens should still * work and there is no need to fetch new tokens from namenode. Like the * previous test, this test should also run while namenode is down. The * setup for this test depends on the previous test. */ // restart the namenode and then shut it down for test cluster.restartNameNode(0); cluster.shutdownNameNode(0); // verify blockSeekTo() still works (forced to use cached tokens) in1.seek(0); assertTrue(checkFile1(in1, expected)); // verify again blockSeekTo() still works (forced to use cached tokens) if (isStriped) { in2.seek(0); } else { in2.seekToNewSource(0); } assertTrue(checkFile1(in2, expected)); // verify fetchBlockByteRange() still works (forced to use cached tokens) assertTrue(checkFile2(in3, expected)); /* * testing that after both namenode and datanodes got restarted (namenode * first, followed by datanodes), DFSClient can't access DN without * re-fetching tokens and is able to re-fetch tokens transparently. The * setup of this test depends on the previous test. */ // restore the cluster and restart the datanodes for test cluster.restartNameNode(0); assertTrue(cluster.restartDataNodes(true)); cluster.waitActive(); assertEquals(numDataNodes, cluster.getDataNodes().size()); // shutdown namenode so that DFSClient can't get new tokens from namenode cluster.shutdownNameNode(0); // verify blockSeekTo() fails (cached tokens become invalid) in1.seek(0); assertFalse(checkFile1(in1, expected)); // verify fetchBlockByteRange() fails (cached tokens become invalid) assertFalse(checkFile2(in3, expected)); // restart the namenode to allow DFSClient to re-fetch tokens cluster.restartNameNode(0); // verify blockSeekTo() works again (by transparently re-fetching // tokens from namenode) in1.seek(0); assertTrue(checkFile1(in1, expected)); if (isStriped) { in2.seek(0); } else { in2.seekToNewSource(0); } assertTrue(checkFile1(in2, expected)); // verify fetchBlockByteRange() works again (by transparently // re-fetching tokens from namenode) assertTrue(checkFile2(in3, expected)); /* * testing that when datanodes are restarted on different ports, DFSClient * is able to re-fetch tokens transparently to connect to them */ // restart datanodes on newly assigned ports assertTrue(cluster.restartDataNodes(false)); cluster.waitActive(); assertEquals(numDataNodes, cluster.getDataNodes().size()); // verify blockSeekTo() is able to re-fetch token transparently in1.seek(0); assertTrue(checkFile1(in1, expected)); // verify blockSeekTo() is able to re-fetch token transparently if (isStriped) { in2.seek(0); } else { in2.seekToNewSource(0); } assertTrue(checkFile1(in2, expected)); // verify fetchBlockByteRange() is able to re-fetch token transparently assertTrue(checkFile2(in3, expected)); }