HDFS-3703. Merging change 1384209 from trunk to branch-2
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1384210 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1415f8490d
commit
042f9102fe
|
@ -9,6 +9,10 @@ Release 2.0.3-alpha - Unreleased
|
||||||
HDFS-2656. Add libwebhdfs, a pure C client based on WebHDFS.
|
HDFS-2656. Add libwebhdfs, a pure C client based on WebHDFS.
|
||||||
(Jaimin D Jetly and Jing Zhao via szetszwo)
|
(Jaimin D Jetly and Jing Zhao via szetszwo)
|
||||||
|
|
||||||
|
HDFS-3703. Datanodes are marked stale if heartbeat is not received in
|
||||||
|
configured timeout and are selected as the last location to read from.
|
||||||
|
(Jing Zhao via suresh)
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
|
|
||||||
HDFS-3925. Prettify PipelineAck#toString() for printing to a log
|
HDFS-3925. Prettify PipelineAck#toString() for printing to a log
|
||||||
|
|
|
@ -175,6 +175,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
||||||
public static final String DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY = "dfs.datanode.socket.reuse.keepalive";
|
public static final String DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY = "dfs.datanode.socket.reuse.keepalive";
|
||||||
public static final int DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT = 1000;
|
public static final int DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT = 1000;
|
||||||
|
|
||||||
|
// Whether to enable datanode's stale state detection and usage
|
||||||
|
public static final String DFS_NAMENODE_CHECK_STALE_DATANODE_KEY = "dfs.namenode.check.stale.datanode";
|
||||||
|
public static final boolean DFS_NAMENODE_CHECK_STALE_DATANODE_DEFAULT = false;
|
||||||
|
// The default value of the time interval for marking datanodes as stale
|
||||||
|
public static final String DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY = "dfs.namenode.stale.datanode.interval";
|
||||||
|
public static final long DFS_NAMENODE_STALE_DATANODE_INTERVAL_MILLI_DEFAULT = 30 * 1000; // 30s
|
||||||
|
|
||||||
// Replication monitoring related keys
|
// Replication monitoring related keys
|
||||||
public static final String DFS_NAMENODE_INVALIDATE_WORK_PCT_PER_ITERATION =
|
public static final String DFS_NAMENODE_INVALIDATE_WORK_PCT_PER_ITERATION =
|
||||||
"dfs.namenode.invalidate.work.pct.per.iteration";
|
"dfs.namenode.invalidate.work.pct.per.iteration";
|
||||||
|
|
|
@ -133,6 +133,43 @@ public class DFSUtil {
|
||||||
a.isDecommissioned() ? 1 : -1;
|
a.isDecommissioned() ? 1 : -1;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Comparator for sorting DataNodeInfo[] based on decommissioned/stale states.
|
||||||
|
* Decommissioned/stale nodes are moved to the end of the array on sorting
|
||||||
|
* with this compartor.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public static class DecomStaleComparator implements Comparator<DatanodeInfo> {
|
||||||
|
private long staleInterval;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor of DecomStaleComparator
|
||||||
|
*
|
||||||
|
* @param interval
|
||||||
|
* The time invertal for marking datanodes as stale is passed from
|
||||||
|
* outside, since the interval may be changed dynamically
|
||||||
|
*/
|
||||||
|
public DecomStaleComparator(long interval) {
|
||||||
|
this.staleInterval = interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(DatanodeInfo a, DatanodeInfo b) {
|
||||||
|
// Decommissioned nodes will still be moved to the end of the list
|
||||||
|
if (a.isDecommissioned()) {
|
||||||
|
return b.isDecommissioned() ? 0 : 1;
|
||||||
|
} else if (b.isDecommissioned()) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
// Stale nodes will be moved behind the normal nodes
|
||||||
|
boolean aStale = a.isStale(staleInterval);
|
||||||
|
boolean bStale = b.isStale(staleInterval);
|
||||||
|
return aStale == bStale ? 0 : (aStale ? 1 : -1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Address matcher for matching an address to local address
|
* Address matcher for matching an address to local address
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.net.NetworkTopology;
|
||||||
import org.apache.hadoop.net.Node;
|
import org.apache.hadoop.net.Node;
|
||||||
import org.apache.hadoop.net.NodeBase;
|
import org.apache.hadoop.net.NodeBase;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
import org.apache.hadoop.util.Time;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class extends the primary identifier of a Datanode with ephemeral
|
* This class extends the primary identifier of a Datanode with ephemeral
|
||||||
|
@ -320,6 +321,23 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
||||||
return adminState;
|
return adminState;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if the datanode is in stale state. Here if
|
||||||
|
* the namenode has not received heartbeat msg from a
|
||||||
|
* datanode for more than staleInterval (default value is
|
||||||
|
* {@link DFSConfigKeys#DFS_NAMENODE_STALE_DATANODE_INTERVAL_MILLI_DEFAULT}),
|
||||||
|
* the datanode will be treated as stale node.
|
||||||
|
*
|
||||||
|
* @param staleInterval
|
||||||
|
* the time interval for marking the node as stale. If the last
|
||||||
|
* update time is beyond the given time interval, the node will be
|
||||||
|
* marked as stale.
|
||||||
|
* @return true if the node is stale
|
||||||
|
*/
|
||||||
|
public boolean isStale(long staleInterval) {
|
||||||
|
return (Time.now() - lastUpdate) >= staleInterval;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets the admin state of this node.
|
* Sets the admin state of this node.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -25,6 +25,7 @@ import java.net.InetAddress;
|
||||||
import java.net.UnknownHostException;
|
import java.net.UnknownHostException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Comparator;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -129,6 +130,11 @@ public class DatanodeManager {
|
||||||
*/
|
*/
|
||||||
private boolean hasClusterEverBeenMultiRack = false;
|
private boolean hasClusterEverBeenMultiRack = false;
|
||||||
|
|
||||||
|
/** Whether or not to check the stale datanodes */
|
||||||
|
private volatile boolean checkForStaleNodes;
|
||||||
|
/** The time interval for detecting stale datanodes */
|
||||||
|
private volatile long staleInterval;
|
||||||
|
|
||||||
DatanodeManager(final BlockManager blockManager,
|
DatanodeManager(final BlockManager blockManager,
|
||||||
final Namesystem namesystem, final Configuration conf
|
final Namesystem namesystem, final Configuration conf
|
||||||
) throws IOException {
|
) throws IOException {
|
||||||
|
@ -166,6 +172,21 @@ public class DatanodeManager {
|
||||||
DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY, blockInvalidateLimit);
|
DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY, blockInvalidateLimit);
|
||||||
LOG.info(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY
|
LOG.info(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY
|
||||||
+ "=" + this.blockInvalidateLimit);
|
+ "=" + this.blockInvalidateLimit);
|
||||||
|
// set the value of stale interval based on configuration
|
||||||
|
this.checkForStaleNodes = conf.getBoolean(
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY,
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_DEFAULT);
|
||||||
|
if (this.checkForStaleNodes) {
|
||||||
|
this.staleInterval = conf.getLong(
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_MILLI_DEFAULT);
|
||||||
|
if (this.staleInterval < DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_MILLI_DEFAULT) {
|
||||||
|
LOG.warn("The given interval for marking stale datanode = "
|
||||||
|
+ this.staleInterval + ", which is smaller than the default value "
|
||||||
|
+ DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_MILLI_DEFAULT
|
||||||
|
+ ".");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Daemon decommissionthread = null;
|
private Daemon decommissionthread = null;
|
||||||
|
@ -213,11 +234,14 @@ public class DatanodeManager {
|
||||||
final List<LocatedBlock> locatedblocks) {
|
final List<LocatedBlock> locatedblocks) {
|
||||||
//sort the blocks
|
//sort the blocks
|
||||||
final DatanodeDescriptor client = getDatanodeByHost(targethost);
|
final DatanodeDescriptor client = getDatanodeByHost(targethost);
|
||||||
|
|
||||||
|
Comparator<DatanodeInfo> comparator = checkForStaleNodes ?
|
||||||
|
new DFSUtil.DecomStaleComparator(staleInterval) :
|
||||||
|
DFSUtil.DECOM_COMPARATOR;
|
||||||
for (LocatedBlock b : locatedblocks) {
|
for (LocatedBlock b : locatedblocks) {
|
||||||
networktopology.pseudoSortByDistance(client, b.getLocations());
|
networktopology.pseudoSortByDistance(client, b.getLocations());
|
||||||
|
// Move decommissioned/stale datanodes to the bottom
|
||||||
// Move decommissioned datanodes to the bottom
|
Arrays.sort(b.getLocations(), comparator);
|
||||||
Arrays.sort(b.getLocations(), DFSUtil.DECOM_COMPARATOR);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1158,6 +1158,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
||||||
if (blocks != null) {
|
if (blocks != null) {
|
||||||
blockManager.getDatanodeManager().sortLocatedBlocks(
|
blockManager.getDatanodeManager().sortLocatedBlocks(
|
||||||
clientMachine, blocks.getLocatedBlocks());
|
clientMachine, blocks.getLocatedBlocks());
|
||||||
|
|
||||||
|
LocatedBlock lastBlock = blocks.getLastLocatedBlock();
|
||||||
|
if (lastBlock != null) {
|
||||||
|
ArrayList<LocatedBlock> lastBlockList = new ArrayList<LocatedBlock>();
|
||||||
|
lastBlockList.add(lastBlock);
|
||||||
|
blockManager.getDatanodeManager().sortLocatedBlocks(
|
||||||
|
clientMachine, lastBlockList);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return blocks;
|
return blocks;
|
||||||
}
|
}
|
||||||
|
|
|
@ -960,6 +960,30 @@
|
||||||
<value>${dfs.web.authentication.kerberos.principal}</value>
|
<value>${dfs.web.authentication.kerberos.principal}</value>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>dfs.namenode.check.stale.datanode</name>
|
||||||
|
<value>false</value>
|
||||||
|
<description>
|
||||||
|
Indicate whether or not to check "stale" datanodes whose
|
||||||
|
heartbeat messages have not been received by the namenode
|
||||||
|
for more than a specified time interval. If this configuration
|
||||||
|
parameter is set as true, the stale datanodes will be moved to
|
||||||
|
the end of the target node list for reading. The writing will
|
||||||
|
also try to avoid stale nodes.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>dfs.namenode.stale.datanode.interval</name>
|
||||||
|
<value>30000</value>
|
||||||
|
<description>
|
||||||
|
Default time interval for marking a datanode as "stale", i.e., if
|
||||||
|
the namenode has not received heartbeat msg from a datanode for
|
||||||
|
more than this time interval, the datanode will be marked and treated
|
||||||
|
as "stale" by default.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>dfs.namenode.invalidate.work.pct.per.iteration</name>
|
<name>dfs.namenode.invalidate.work.pct.per.iteration</name>
|
||||||
<value>0.32f</value>
|
<value>0.32f</value>
|
||||||
|
|
|
@ -17,8 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hdfs;
|
package org.apache.hadoop.hdfs;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.*;
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
@ -28,22 +27,152 @@ import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.protocol.Block;
|
import org.apache.hadoop.hdfs.protocol.Block;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
||||||
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||||
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
|
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
||||||
import org.apache.hadoop.ipc.RemoteException;
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
|
import org.apache.hadoop.util.Time;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class tests if block replacement request to data nodes work correctly.
|
* This class tests if getblocks request works correctly.
|
||||||
*/
|
*/
|
||||||
public class TestGetBlocks {
|
public class TestGetBlocks {
|
||||||
|
private static final int blockSize = 8192;
|
||||||
|
private static final String racks[] = new String[] { "/d1/r1", "/d1/r1",
|
||||||
|
"/d1/r2", "/d1/r2", "/d1/r2", "/d2/r3", "/d2/r3" };
|
||||||
|
private static final int numDatanodes = racks.length;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stop the heartbeat of a datanode in the MiniDFSCluster
|
||||||
|
*
|
||||||
|
* @param cluster
|
||||||
|
* The MiniDFSCluster
|
||||||
|
* @param hostName
|
||||||
|
* The hostName of the datanode to be stopped
|
||||||
|
* @return The DataNode whose heartbeat has been stopped
|
||||||
|
*/
|
||||||
|
private DataNode stopDataNodeHeartbeat(MiniDFSCluster cluster, String hostName) {
|
||||||
|
for (DataNode dn : cluster.getDataNodes()) {
|
||||||
|
if (dn.getDatanodeId().getHostName().equals(hostName)) {
|
||||||
|
DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
|
||||||
|
return dn;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test if the datanodes returned by
|
||||||
|
* {@link ClientProtocol#getBlockLocations(String, long, long)} is correct
|
||||||
|
* when stale nodes checking is enabled. Also test during the scenario when 1)
|
||||||
|
* stale nodes checking is enabled, 2) a writing is going on, 3) a datanode
|
||||||
|
* becomes stale happen simultaneously
|
||||||
|
*
|
||||||
|
* @throws Exception
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testReadSelectNonStaleDatanode() throws Exception {
|
||||||
|
HdfsConfiguration conf = new HdfsConfiguration();
|
||||||
|
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY, true);
|
||||||
|
long staleInterval = 30 * 1000 * 60;
|
||||||
|
conf.setLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
|
||||||
|
staleInterval);
|
||||||
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
||||||
|
.numDataNodes(numDatanodes).racks(racks).build();
|
||||||
|
|
||||||
|
cluster.waitActive();
|
||||||
|
InetSocketAddress addr = new InetSocketAddress("localhost",
|
||||||
|
cluster.getNameNodePort());
|
||||||
|
DFSClient client = new DFSClient(addr, conf);
|
||||||
|
List<DatanodeDescriptor> nodeInfoList = cluster.getNameNode()
|
||||||
|
.getNamesystem().getBlockManager().getDatanodeManager()
|
||||||
|
.getDatanodeListForReport(DatanodeReportType.LIVE);
|
||||||
|
assertEquals("Unexpected number of datanodes", numDatanodes,
|
||||||
|
nodeInfoList.size());
|
||||||
|
FileSystem fileSys = cluster.getFileSystem();
|
||||||
|
FSDataOutputStream stm = null;
|
||||||
|
try {
|
||||||
|
// do the writing but do not close the FSDataOutputStream
|
||||||
|
// in order to mimic the ongoing writing
|
||||||
|
final Path fileName = new Path("/file1");
|
||||||
|
stm = fileSys.create(
|
||||||
|
fileName,
|
||||||
|
true,
|
||||||
|
fileSys.getConf().getInt(
|
||||||
|
CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
|
||||||
|
(short) 3, blockSize);
|
||||||
|
stm.write(new byte[(blockSize * 3) / 2]);
|
||||||
|
// We do not close the stream so that
|
||||||
|
// the writing seems to be still ongoing
|
||||||
|
stm.hflush();
|
||||||
|
|
||||||
|
LocatedBlocks blocks = client.getNamenode().getBlockLocations(
|
||||||
|
fileName.toString(), 0, blockSize);
|
||||||
|
DatanodeInfo[] nodes = blocks.get(0).getLocations();
|
||||||
|
assertEquals(nodes.length, 3);
|
||||||
|
DataNode staleNode = null;
|
||||||
|
DatanodeDescriptor staleNodeInfo = null;
|
||||||
|
// stop the heartbeat of the first node
|
||||||
|
staleNode = this.stopDataNodeHeartbeat(cluster, nodes[0].getHostName());
|
||||||
|
assertNotNull(staleNode);
|
||||||
|
// set the first node as stale
|
||||||
|
staleNodeInfo = cluster.getNameNode().getNamesystem().getBlockManager()
|
||||||
|
.getDatanodeManager()
|
||||||
|
.getDatanode(staleNode.getDatanodeId());
|
||||||
|
staleNodeInfo.setLastUpdate(Time.now() - staleInterval - 1);
|
||||||
|
|
||||||
|
LocatedBlocks blocksAfterStale = client.getNamenode().getBlockLocations(
|
||||||
|
fileName.toString(), 0, blockSize);
|
||||||
|
DatanodeInfo[] nodesAfterStale = blocksAfterStale.get(0).getLocations();
|
||||||
|
assertEquals(nodesAfterStale.length, 3);
|
||||||
|
assertEquals(nodesAfterStale[2].getHostName(), nodes[0].getHostName());
|
||||||
|
|
||||||
|
// restart the staleNode's heartbeat
|
||||||
|
DataNodeTestUtils.setHeartbeatsDisabledForTests(staleNode, false);
|
||||||
|
// reset the first node as non-stale, so as to avoid two stale nodes
|
||||||
|
staleNodeInfo.setLastUpdate(Time.now());
|
||||||
|
|
||||||
|
LocatedBlock lastBlock = client.getLocatedBlocks(fileName.toString(), 0,
|
||||||
|
Long.MAX_VALUE).getLastLocatedBlock();
|
||||||
|
nodes = lastBlock.getLocations();
|
||||||
|
assertEquals(nodes.length, 3);
|
||||||
|
// stop the heartbeat of the first node for the last block
|
||||||
|
staleNode = this.stopDataNodeHeartbeat(cluster, nodes[0].getHostName());
|
||||||
|
assertNotNull(staleNode);
|
||||||
|
// set the node as stale
|
||||||
|
cluster.getNameNode().getNamesystem().getBlockManager()
|
||||||
|
.getDatanodeManager()
|
||||||
|
.getDatanode(staleNode.getDatanodeId())
|
||||||
|
.setLastUpdate(Time.now() - staleInterval - 1);
|
||||||
|
|
||||||
|
LocatedBlock lastBlockAfterStale = client.getLocatedBlocks(
|
||||||
|
fileName.toString(), 0, Long.MAX_VALUE).getLastLocatedBlock();
|
||||||
|
nodesAfterStale = lastBlockAfterStale.getLocations();
|
||||||
|
assertEquals(nodesAfterStale.length, 3);
|
||||||
|
assertEquals(nodesAfterStale[2].getHostName(), nodes[0].getHostName());
|
||||||
|
} finally {
|
||||||
|
if (stm != null) {
|
||||||
|
stm.close();
|
||||||
|
}
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/** test getBlocks */
|
/** test getBlocks */
|
||||||
@Test
|
@Test
|
||||||
public void testGetBlocks() throws Exception {
|
public void testGetBlocks() throws Exception {
|
||||||
|
@ -54,9 +183,8 @@ public class TestGetBlocks {
|
||||||
final Random r = new Random();
|
final Random r = new Random();
|
||||||
|
|
||||||
CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
|
CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
|
||||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF)
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(
|
||||||
.numDataNodes(REPLICATION_FACTOR)
|
REPLICATION_FACTOR).build();
|
||||||
.build();
|
|
||||||
try {
|
try {
|
||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
|
|
||||||
|
@ -69,7 +197,8 @@ public class TestGetBlocks {
|
||||||
long bytesToWrite = fileLen;
|
long bytesToWrite = fileLen;
|
||||||
while (bytesToWrite > 0) {
|
while (bytesToWrite > 0) {
|
||||||
r.nextBytes(data);
|
r.nextBytes(data);
|
||||||
int bytesToWriteNext = (1024<bytesToWrite)?1024:(int)bytesToWrite;
|
int bytesToWriteNext = (1024 < bytesToWrite) ? 1024
|
||||||
|
: (int) bytesToWrite;
|
||||||
out.write(data, 0, bytesToWriteNext);
|
out.write(data, 0, bytesToWriteNext);
|
||||||
bytesToWrite -= bytesToWriteNext;
|
bytesToWrite -= bytesToWriteNext;
|
||||||
}
|
}
|
||||||
|
@ -80,9 +209,10 @@ public class TestGetBlocks {
|
||||||
DatanodeInfo[] dataNodes = null;
|
DatanodeInfo[] dataNodes = null;
|
||||||
boolean notWritten;
|
boolean notWritten;
|
||||||
do {
|
do {
|
||||||
final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF), CONF);
|
final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF),
|
||||||
locatedBlocks = dfsclient.getNamenode().
|
CONF);
|
||||||
getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
|
locatedBlocks = dfsclient.getNamenode()
|
||||||
|
.getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
|
||||||
assertEquals(2, locatedBlocks.size());
|
assertEquals(2, locatedBlocks.size());
|
||||||
notWritten = false;
|
notWritten = false;
|
||||||
for (int i = 0; i < 2; i++) {
|
for (int i = 0; i < 2; i++) {
|
||||||
|
@ -136,8 +266,7 @@ public class TestGetBlocks {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void getBlocksWithException(NamenodeProtocol namenode,
|
private void getBlocksWithException(NamenodeProtocol namenode,
|
||||||
DatanodeInfo datanode,
|
DatanodeInfo datanode, long size) throws IOException {
|
||||||
long size) throws IOException {
|
|
||||||
boolean getException = false;
|
boolean getException = false;
|
||||||
try {
|
try {
|
||||||
namenode.getBlocks(DFSTestUtil.getLocalDatanodeInfo(), 2);
|
namenode.getBlocks(DFSTestUtil.getLocalDatanodeInfo(), 2);
|
||||||
|
@ -164,18 +293,12 @@ public class TestGetBlocks {
|
||||||
System.out.println("map=" + map.toString().replace(",", "\n "));
|
System.out.println("map=" + map.toString().replace(",", "\n "));
|
||||||
|
|
||||||
for (int i = 0; i < blkids.length; i++) {
|
for (int i = 0; i < blkids.length; i++) {
|
||||||
Block b = new Block(blkids[i], 0, GenerationStamp.GRANDFATHER_GENERATION_STAMP);
|
Block b = new Block(blkids[i], 0,
|
||||||
|
GenerationStamp.GRANDFATHER_GENERATION_STAMP);
|
||||||
Long v = map.get(b);
|
Long v = map.get(b);
|
||||||
System.out.println(b + " => " + v);
|
System.out.println(b + " => " + v);
|
||||||
assertEquals(blkids[i], v.longValue());
|
assertEquals(blkids[i], v.longValue());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @param args
|
|
||||||
*/
|
|
||||||
public static void main(String[] args) throws Exception {
|
|
||||||
(new TestGetBlocks()).testGetBlocks();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue