HDFS-9579. Provide bytes-read-by-network-distance metrics at FileSystem.Statistics level (Ming Ma via sjlee)

This commit is contained in:
Sangjin Lee 2016-03-21 10:56:32 -07:00
parent 478a25b929
commit d956e0a0bb
21 changed files with 366 additions and 83 deletions

View File

@ -3009,11 +3009,15 @@ public static final class Statistics {
* need.
*/
public static class StatisticsData {
volatile long bytesRead;
volatile long bytesWritten;
volatile int readOps;
volatile int largeReadOps;
volatile int writeOps;
private volatile long bytesRead;
private volatile long bytesWritten;
private volatile int readOps;
private volatile int largeReadOps;
private volatile int writeOps;
private volatile long bytesReadLocalHost;
private volatile long bytesReadDistanceOfOneOrTwo;
private volatile long bytesReadDistanceOfThreeOrFour;
private volatile long bytesReadDistanceOfFiveOrLarger;
/**
* Add another StatisticsData object to this one.
@ -3024,6 +3028,12 @@ void add(StatisticsData other) {
this.readOps += other.readOps;
this.largeReadOps += other.largeReadOps;
this.writeOps += other.writeOps;
this.bytesReadLocalHost += other.bytesReadLocalHost;
this.bytesReadDistanceOfOneOrTwo += other.bytesReadDistanceOfOneOrTwo;
this.bytesReadDistanceOfThreeOrFour +=
other.bytesReadDistanceOfThreeOrFour;
this.bytesReadDistanceOfFiveOrLarger +=
other.bytesReadDistanceOfFiveOrLarger;
}
/**
@ -3035,6 +3045,12 @@ void negate() {
this.readOps = -this.readOps;
this.largeReadOps = -this.largeReadOps;
this.writeOps = -this.writeOps;
this.bytesReadLocalHost = -this.bytesReadLocalHost;
this.bytesReadDistanceOfOneOrTwo = -this.bytesReadDistanceOfOneOrTwo;
this.bytesReadDistanceOfThreeOrFour =
-this.bytesReadDistanceOfThreeOrFour;
this.bytesReadDistanceOfFiveOrLarger =
-this.bytesReadDistanceOfFiveOrLarger;
}
@Override
@ -3063,6 +3079,22 @@ public int getLargeReadOps() {
public int getWriteOps() {
return writeOps;
}
public long getBytesReadLocalHost() {
return bytesReadLocalHost;
}
public long getBytesReadDistanceOfOneOrTwo() {
return bytesReadDistanceOfOneOrTwo;
}
public long getBytesReadDistanceOfThreeOrFour() {
return bytesReadDistanceOfThreeOrFour;
}
public long getBytesReadDistanceOfFiveOrLarger() {
return bytesReadDistanceOfFiveOrLarger;
}
}
private interface StatisticsAggregator<T> {
@ -3253,6 +3285,33 @@ public void incrementWriteOps(int count) {
getThreadStatistics().writeOps += count;
}
/**
* Increment the bytes read by the network distance in the statistics
* In the common network topology setup, distance value should be an even
* number such as 0, 2, 4, 6. To make it more general, we group distance
* by {1, 2}, {3, 4} and {5 and beyond} for accounting.
* @param distance the network distance
* @param newBytes the additional bytes read
*/
public void incrementBytesReadByDistance(int distance, long newBytes) {
switch (distance) {
case 0:
getThreadStatistics().bytesReadLocalHost += newBytes;
break;
case 1:
case 2:
getThreadStatistics().bytesReadDistanceOfOneOrTwo += newBytes;
break;
case 3:
case 4:
getThreadStatistics().bytesReadDistanceOfThreeOrFour += newBytes;
break;
default:
getThreadStatistics().bytesReadDistanceOfFiveOrLarger += newBytes;
break;
}
}
/**
* Apply the given aggregator to all StatisticsData objects associated with
* this Statistics object.
@ -3370,6 +3429,55 @@ public Integer aggregate() {
});
}
/**
* In the common network topology setup, distance value should be an even
* number such as 0, 2, 4, 6. To make it more general, we group distance
* by {1, 2}, {3, 4} and {5 and beyond} for accounting. So if the caller
* ask for bytes read for distance 2, the function will return the value
* for group {1, 2}.
* @param distance the network distance
* @return the total number of bytes read by the network distance
*/
public long getBytesReadByDistance(int distance) {
long bytesRead;
switch (distance) {
case 0:
bytesRead = getData().getBytesReadLocalHost();
break;
case 1:
case 2:
bytesRead = getData().getBytesReadDistanceOfOneOrTwo();
break;
case 3:
case 4:
bytesRead = getData().getBytesReadDistanceOfThreeOrFour();
break;
default:
bytesRead = getData().getBytesReadDistanceOfFiveOrLarger();
break;
}
return bytesRead;
}
/**
* Get all statistics data
* MR or other frameworks can use the method to get all statistics at once.
* @return the StatisticsData
*/
public StatisticsData getData() {
return visitAll(new StatisticsAggregator<StatisticsData>() {
private StatisticsData all = new StatisticsData();
@Override
public void accept(StatisticsData data) {
all.add(data);
}
public StatisticsData aggregate() {
return all;
}
});
}
@Override
public String toString() {

View File

@ -638,6 +638,20 @@ public static String getHostNameOfIP(String ipPort) {
/**
* Return hostname without throwing exception.
* The returned hostname String format is "hostname".
* @return hostname
*/
public static String getLocalHostname() {
try {
return InetAddress.getLocalHost().getHostName();
} catch(UnknownHostException uhe) {
return "" + uhe;
}
}
/**
* Return hostname without throwing exception.
* The returned hostname String format is "hostname/ip address".
* @return hostname
*/
public static String getHostname() {

View File

@ -369,6 +369,16 @@ protected boolean areChildrenLeaves() {
int getNumOfLeaves() {
return numOfLeaves;
}
@Override
public int hashCode() {
return super.hashCode();
}
@Override
public boolean equals(Object to) {
return super.equals(to);
}
} // end of InnerNode
/**
@ -607,9 +617,14 @@ public int getNumOfLeaves() {
* or {@link Integer#MAX_VALUE} if node1 or node2 do not belong to the cluster
*/
public int getDistance(Node node1, Node node2) {
if (node1 == node2) {
if ((node1 != null && node1.equals(node2)) ||
(node1 == null && node2 == null)) {
return 0;
}
if (node1 == null || node2 == null) {
LOG.warn("One of the nodes is a null pointer");
return Integer.MAX_VALUE;
}
Node n1=node1, n2=node2;
int dis = 0;
netlock.readLock().lock();

View File

@ -113,6 +113,22 @@ public static String getPath(Node node) {
return node.getNetworkLocation() + PATH_SEPARATOR_STR + node.getName();
}
@Override
public boolean equals(Object to) {
if (this == to) {
return true;
}
if (!(to instanceof NodeBase)) {
return false;
}
return getPath(this).equals(getPath((NodeBase)to));
}
@Override
public int hashCode() {
return getPath(this).hashCode();
}
/** @return this node's path as its string representation */
@Override
public String toString() {

View File

@ -79,11 +79,6 @@ public interface BlockReader extends ByteBufferReadable {
*/
int readAll(byte[] buf, int offset, int len) throws IOException;
/**
* @return true only if this is a local read.
*/
boolean isLocal();
/**
* @return true only if this is a short-circuit read.
* All short-circuit reads are also local.
@ -98,4 +93,9 @@ public interface BlockReader extends ByteBufferReadable {
* supported.
*/
ClientMmap getClientMmap(EnumSet<ReadOption> opts);
/**
* Return the network distance between local machine and the remote machine.
*/
int getNetworkDistance();
}

View File

@ -833,16 +833,19 @@ private static boolean isSecurityException(IOException ioe) {
@SuppressWarnings("deprecation")
private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
int networkDistance = clientContext.getNetworkDistance(datanode);
if (conf.getShortCircuitConf().isUseLegacyBlockReader()) {
return RemoteBlockReader.newBlockReader(fileName,
block, token, startOffset, length, conf.getIoBufferSize(),
verifyChecksum, clientName, peer, datanode,
clientContext.getPeerCache(), cachingStrategy, tracer);
clientContext.getPeerCache(), cachingStrategy, tracer,
networkDistance);
} else {
return RemoteBlockReader2.newBlockReader(
fileName, block, token, startOffset, length,
verifyChecksum, clientName, peer, datanode,
clientContext.getPeerCache(), cachingStrategy, tracer);
clientContext.getPeerCache(), cachingStrategy, tracer,
networkDistance);
}
}

View File

@ -640,11 +640,6 @@ public synchronized int readAll(byte[] buf, int off, int len)
return BlockReaderUtil.readAll(this, buf, off, len);
}
@Override
public boolean isLocal() {
return true;
}
@Override
public boolean isShortCircuit() {
return true;
@ -716,4 +711,9 @@ void forceAnchorable() {
void forceUnanchorable() {
replica.getSlot().makeUnanchorable();
}
@Override
public int getNetworkDistance() {
return 0;
}
}

View File

@ -722,11 +722,6 @@ public int available() {
return Integer.MAX_VALUE;
}
@Override
public boolean isLocal() {
return true;
}
@Override
public boolean isShortCircuit() {
return true;
@ -736,4 +731,9 @@ public boolean isShortCircuit() {
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
return null;
}
@Override
public int getNetworkDistance() {
return 0;
}
}

View File

@ -17,16 +17,28 @@
*/
package org.apache.hadoop.hdfs;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.shortcircuit.DomainSocketFactory;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
import org.apache.hadoop.hdfs.util.ByteArrayManager;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.net.ScriptBasedMapping;
import org.apache.hadoop.util.ReflectionUtils;
import com.google.common.annotations.VisibleForTesting;
@ -101,7 +113,12 @@ public class ClientContext {
*/
private boolean printedConfWarning = false;
private ClientContext(String name, DfsClientConf conf) {
private final NetworkTopology topology;
private final NodeBase clientNode;
private final Map<NodeBase, Integer> nodeToDistance;
private ClientContext(String name, DfsClientConf conf,
Configuration config) {
final ShortCircuitConf scConf = conf.getShortCircuitConf();
this.name = name;
@ -116,14 +133,28 @@ private ClientContext(String name, DfsClientConf conf) {
this.byteArrayManager = ByteArrayManager.newInstance(
conf.getWriteByteArrayManagerConf());
DNSToSwitchMapping dnsToSwitchMapping = ReflectionUtils.newInstance(
config.getClass(
CommonConfigurationKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
ScriptBasedMapping.class, DNSToSwitchMapping.class), config);
List<String> nodes = new ArrayList<>();
String clientHostName = NetUtils.getLocalHostname();
nodes.add(clientHostName);
clientNode = new NodeBase(clientHostName,
dnsToSwitchMapping.resolve(nodes).get(0));
this.topology = NetworkTopology.getInstance(config);
this.topology.add(clientNode);
this.nodeToDistance = new ConcurrentHashMap<>();
}
public static ClientContext get(String name, DfsClientConf conf) {
public static ClientContext get(String name, DfsClientConf conf,
Configuration config) {
ClientContext context;
synchronized(ClientContext.class) {
context = CACHES.get(name);
if (context == null) {
context = new ClientContext(name, conf);
context = new ClientContext(name, conf, config);
CACHES.put(name, context);
} else {
context.printConfWarningIfNeeded(conf);
@ -132,6 +163,10 @@ public static ClientContext get(String name, DfsClientConf conf) {
return context;
}
public static ClientContext get(String name, Configuration config) {
return get(name, new DfsClientConf(config), config);
}
/**
* Get a client context, from a Configuration object.
*
@ -141,8 +176,7 @@ public static ClientContext get(String name, DfsClientConf conf) {
@VisibleForTesting
public static ClientContext getFromConf(Configuration conf) {
return get(conf.get(HdfsClientConfigKeys.DFS_CLIENT_CONTEXT,
HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT),
new DfsClientConf(conf));
HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT), conf);
}
private void printConfWarningIfNeeded(DfsClientConf conf) {
@ -193,4 +227,16 @@ public DomainSocketFactory getDomainSocketFactory() {
public ByteArrayManager getByteArrayManager() {
return byteArrayManager;
}
public int getNetworkDistance(DatanodeInfo datanodeInfo) {
NodeBase node = new NodeBase(datanodeInfo.getHostName(),
datanodeInfo.getNetworkLocation());
Integer distance = nodeToDistance.get(node);
if (distance == null) {
topology.add(node);
distance = topology.getDistance(clientNode, node);
nodeToDistance.put(node, distance);
}
return distance;
}
}

View File

@ -225,7 +225,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
final String clientName;
final SocketFactory socketFactory;
final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
final FileSystem.Statistics stats;
private final FileSystem.Statistics stats;
private final String authority;
private final Random r = new Random();
private SocketAddress[] localInterfaceAddrs;
@ -369,7 +369,7 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
new CachingStrategy(writeDropBehind, readahead);
this.clientContext = ClientContext.get(
conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
dfsClientConf);
dfsClientConf, conf);
if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
@ -2939,6 +2939,13 @@ public Peer newConnectedPeer(InetSocketAddress addr,
}
}
void updateFileSystemReadStats(int distance, int nRead) {
if (stats != null) {
stats.incrementBytesRead(nRead);
stats.incrementBytesReadByDistance(distance, nRead);
}
}
/**
* Create hedged reads thread pool, HEDGED_READ_THREAD_POOL, if
* it does not already exist.

View File

@ -778,7 +778,7 @@ protected void updateReadStatistics(ReadStatistics readStatistics,
synchronized(infoLock) {
if (blockReader.isShortCircuit()) {
readStatistics.addShortCircuitBytes(nRead);
} else if (blockReader.isLocal()) {
} else if (blockReader.getNetworkDistance() == 0) {
readStatistics.addLocalBytes(nRead);
} else {
readStatistics.addRemoteBytes(nRead);
@ -801,6 +801,8 @@ public int doRead(BlockReader blockReader, int off, int len)
throws IOException {
int nRead = blockReader.read(buf, off, len);
updateReadStatistics(readStatistics, nRead, blockReader);
dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(),
nRead);
return nRead;
}
@ -831,6 +833,8 @@ public int doRead(BlockReader blockReader, int off, int len)
int ret = blockReader.read(buf);
success = true;
updateReadStatistics(readStatistics, ret, blockReader);
dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(),
ret);
if (ret == 0) {
DFSClient.LOG.warn("zero");
}
@ -941,9 +945,6 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, in
// got a EOS from reader though we expect more data on it.
throw new IOException("Unexpected EOS from the reader");
}
if (dfsClient.stats != null) {
dfsClient.stats.incrementBytesRead(result);
}
return result;
} catch (ChecksumException ce) {
throw ce;
@ -1223,6 +1224,8 @@ void actualGetFromOneDataNode(final DNAddrPair datanode,
for (int i = 0; i < offsets.length; i++) {
int nread = reader.readAll(buf, offsets[i], lengths[i]);
updateReadStatistics(readStatistics, nread, reader);
dfsClient.updateFileSystemReadStats(
reader.getNetworkDistance(), nread);
if (nread != lengths[i]) {
throw new IOException("truncated return from reader.read(): " +
"excpected " + lengths[i] + ", got " + nread);
@ -1528,9 +1531,6 @@ private int pread(long position, byte[] buffer, int offset, int length)
offset += bytesToRead;
}
assert remaining == 0 : "Wrong number of bytes read.";
if (dfsClient.stats != null) {
dfsClient.stats.incrementBytesRead(realLen);
}
return realLen;
}

View File

@ -108,11 +108,6 @@ public int readAll(byte[] buf, int offset, int len) throws IOException {
return BlockReaderUtil.readAll(this, buf, offset, len);
}
@Override
public boolean isLocal() {
return accessor.isLocal();
}
@Override
public boolean isShortCircuit() {
return accessor.isShortCircuit();
@ -123,4 +118,9 @@ public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
// For now, pluggable ReplicaAccessors do not support zero-copy.
return null;
}
@Override
public int getNetworkDistance() {
return accessor.getNetworkDistance();
}
}

View File

@ -44,7 +44,6 @@
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.core.TraceScope;
@ -93,11 +92,6 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
*/
private final long bytesNeededToFinish;
/**
* True if we are reading from a local DataNode.
*/
private final boolean isLocal;
private boolean eos = false;
private boolean sentStatusCode = false;
@ -109,6 +103,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
private final Tracer tracer;
private final int networkDistance;
/* FSInputChecker interface */
/* same interface as inputStream java.io.InputStream#read()
@ -342,7 +338,8 @@ private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
private RemoteBlockReader(String file, String bpid, long blockId,
DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer,
int networkDistance) {
// Path is used only for printing block and file information in debug
super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
":" + bpid + ":of:"+ file)/*too non path-like?*/,
@ -351,9 +348,6 @@ private RemoteBlockReader(String file, String bpid, long blockId,
checksum.getBytesPerChecksum(),
checksum.getChecksumSize());
this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
createSocketAddr(datanodeID.getXferAddr()));
this.peer = peer;
this.datanodeID = datanodeID;
this.in = in;
@ -375,6 +369,7 @@ private RemoteBlockReader(String file, String bpid, long blockId,
checksumSize = this.checksum.getChecksumSize();
this.peerCache = peerCache;
this.tracer = tracer;
this.networkDistance = networkDistance;
}
/**
@ -400,7 +395,7 @@ public static RemoteBlockReader newBlockReader(String file,
DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer)
Tracer tracer, int networkDistance)
throws IOException {
// in and out will be closed when sock is closed (by the caller)
final DataOutputStream out =
@ -436,7 +431,7 @@ public static RemoteBlockReader newBlockReader(String file,
return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
peer, datanodeID, peerCache, tracer);
peer, datanodeID, peerCache, tracer, networkDistance);
}
@Override
@ -493,11 +488,6 @@ public int available() {
return RemoteBlockReader2.TCP_WINDOW_SIZE;
}
@Override
public boolean isLocal() {
return isLocal;
}
@Override
public boolean isShortCircuit() {
return false;
@ -507,4 +497,9 @@ public boolean isShortCircuit() {
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
return null;
}
@Override
public int getNetworkDistance() {
return networkDistance;
}
}

View File

@ -45,7 +45,6 @@
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum;
import org.apache.htrace.core.TraceScope;
@ -116,17 +115,14 @@ public class RemoteBlockReader2 implements BlockReader {
*/
private long bytesNeededToFinish;
/**
* True if we are reading from a local DataNode.
*/
private final boolean isLocal;
private final boolean verifyChecksum;
private boolean sentStatusCode = false;
private final Tracer tracer;
private final int networkDistance;
@VisibleForTesting
public Peer getPeer() {
return peer;
@ -280,9 +276,8 @@ private void readTrailingEmptyPacket() throws IOException {
protected RemoteBlockReader2(String file, long blockId,
DataChecksum checksum, boolean verifyChecksum,
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
createSocketAddr(datanodeID.getXferAddr()));
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer,
int networkDistance) {
// Path is used only for printing block and file information in debug
this.peer = peer;
this.datanodeID = datanodeID;
@ -302,6 +297,7 @@ protected RemoteBlockReader2(String file, long blockId,
bytesPerChecksum = this.checksum.getBytesPerChecksum();
checksumSize = this.checksum.getChecksumSize();
this.tracer = tracer;
this.networkDistance = networkDistance;
}
@ -397,7 +393,8 @@ public static BlockReader newBlockReader(String file,
Peer peer, DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer) throws IOException {
Tracer tracer,
int networkDistance) throws IOException {
// in and out will be closed when sock is closed (by the caller)
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
peer.getOutputStream()));
@ -430,7 +427,7 @@ public static BlockReader newBlockReader(String file,
return new RemoteBlockReader2(file, block.getBlockId(), checksum,
verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
peerCache, tracer);
peerCache, tracer, networkDistance);
}
static void checkSuccess(
@ -453,11 +450,6 @@ public int available() {
return TCP_WINDOW_SIZE;
}
@Override
public boolean isLocal() {
return isLocal;
}
@Override
public boolean isShortCircuit() {
return false;
@ -467,4 +459,9 @@ public boolean isShortCircuit() {
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
return null;
}
@Override
public int getNetworkDistance() {
return networkDistance;
}
}

View File

@ -87,4 +87,11 @@ public abstract int read(long pos, byte[] buf, int off, int len)
* short-circuit byte count statistics.
*/
public abstract boolean isShortCircuit();
/**
* Return the network distance between local machine and the remote machine.
*/
public int getNetworkDistance() {
return isLocal() ? 0 : Integer.MAX_VALUE;
}
}

View File

@ -358,7 +358,7 @@ public void testZeroCopyMmapCache() throws Exception {
fsIn.close();
fsIn = fs.open(TEST_PATH);
final ShortCircuitCache cache = ClientContext.get(
CONTEXT, new DfsClientConf(conf)). getShortCircuitCache();
CONTEXT, conf).getShortCircuitCache();
cache.accept(new CountingVisitor(0, 5, 5, 0));
results[0] = fsIn.read(null, BLOCK_SIZE,
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
@ -661,7 +661,7 @@ public void testZeroCopyReadOfCachedData() throws Exception {
final ExtendedBlock firstBlock =
DFSTestUtil.getFirstBlock(fs, TEST_PATH);
final ShortCircuitCache cache = ClientContext.get(
CONTEXT, new DfsClientConf(conf)). getShortCircuitCache();
CONTEXT, conf).getShortCircuitCache();
waitForReplicaAnchorStatus(cache, firstBlock, true, true, 1);
// Uncache the replica
fs.removeCacheDirective(directiveId);

View File

@ -43,6 +43,7 @@
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
import org.apache.hadoop.fs.FsTracer;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
import org.apache.hadoop.util.Time;
@ -736,7 +737,8 @@ private void testStatistics(boolean isShortCircuit) throws Exception {
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
FileSystem fs = null;
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster = new MiniDFSCluster.Builder(conf).
hosts(new String[] {NetUtils.getLocalHostname()}).build();
cluster.waitActive();
fs = cluster.getFileSystem();
DFSTestUtil.createFile(fs, TEST_PATH,

View File

@ -99,8 +99,6 @@ public void testReadFromOneDN() throws Exception {
DFSClient client = new DFSClient(
new InetSocketAddress("localhost",
util.getCluster().getNameNodePort()), util.getConf());
ClientContext cacheContext =
ClientContext.get(contextName, client.getConf());
DFSInputStream in = client.open(testFile.toString());
LOG.info("opened " + testFile.toString());
byte[] dataBuf = new byte[BLOCK_SIZE];

View File

@ -77,6 +77,7 @@
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.hdfs.web.HftpFileSystem;
import org.apache.hadoop.hdfs.web.WebHdfsConstants;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils;
@ -513,6 +514,67 @@ private void checkStatistics(FileSystem fs, int readOps, int writeOps,
assertEquals(largeReadOps, DFSTestUtil.getStatistics(fs).getLargeReadOps());
}
/** Checks read statistics. */
private void checkReadStatistics(FileSystem fs, int distance, long expectedReadBytes) {
long bytesRead = DFSTestUtil.getStatistics(fs).
getBytesReadByDistance(distance);
assertEquals(expectedReadBytes, bytesRead);
}
@Test
public void testLocalHostReadStatistics() throws Exception {
testReadFileSystemStatistics(0);
}
@Test
public void testLocalRackReadStatistics() throws Exception {
testReadFileSystemStatistics(2);
}
@Test
public void testRemoteRackOfFirstDegreeReadStatistics() throws Exception {
testReadFileSystemStatistics(4);
}
/** expectedDistance is the expected distance between client and dn.
* 0 means local host.
* 2 means same rack.
* 4 means remote rack of first degree.
*/
private void testReadFileSystemStatistics(int expectedDistance)
throws IOException {
MiniDFSCluster cluster = null;
final Configuration conf = getTestConfiguration();
// create a cluster with a dn with the expected distance.
if (expectedDistance == 0) {
cluster = new MiniDFSCluster.Builder(conf).
hosts(new String[] {NetUtils.getLocalHostname()}).build();
} else if (expectedDistance == 2) {
cluster = new MiniDFSCluster.Builder(conf).
hosts(new String[] {"hostFoo"}).build();
} else if (expectedDistance == 4) {
cluster = new MiniDFSCluster.Builder(conf).
racks(new String[] {"/rackFoo"}).build();
}
// create a file, read the file and verify the metrics
try {
final FileSystem fs = cluster.getFileSystem();
DFSTestUtil.getStatistics(fs).reset();
Path dir = new Path("/test");
Path file = new Path(dir, "file");
String input = "hello world";
DFSTestUtil.writeFile(fs, file, input);
FSDataInputStream stm = fs.open(file);
byte[] actual = new byte[4096];
stm.read(actual);
checkReadStatistics(fs, expectedDistance, input.length());
} finally {
if (cluster != null) cluster.shutdown();
}
}
@Test
public void testFileChecksum() throws Exception {
GenericTestUtils.setLogLevel(HftpFileSystem.LOG, Level.ALL);

View File

@ -28,6 +28,7 @@
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.junit.Assert;
import org.junit.Test;
@ -246,6 +247,11 @@ public boolean isShortCircuit() {
return true;
}
@Override
public int getNetworkDistance() {
return 0;
}
synchronized String getError() {
return error;
}
@ -271,7 +277,7 @@ public void testExternalBlockReader() throws Exception {
String uuid = UUID.randomUUID().toString();
conf.set(SYNTHETIC_BLOCK_READER_TEST_UUID_KEY, uuid);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(1)
.hosts(new String[] {NetUtils.getLocalHostname()})
.build();
final int TEST_LENGTH = 2047;
DistributedFileSystem dfs = cluster.getFileSystem();

View File

@ -129,6 +129,13 @@ public void testGetDistance() throws Exception {
assertEquals(cluster.getDistance(dataNodes[0], dataNodes[1]), 2);
assertEquals(cluster.getDistance(dataNodes[0], dataNodes[3]), 4);
assertEquals(cluster.getDistance(dataNodes[0], dataNodes[6]), 6);
// verify the distance is zero as long as two nodes have the same path.
// They don't need to refer to the same object.
NodeBase node1 = new NodeBase(dataNodes[0].getHostName(),
dataNodes[0].getNetworkLocation());
NodeBase node2 = new NodeBase(dataNodes[0].getHostName(),
dataNodes[0].getNetworkLocation());
assertEquals(0, cluster.getDistance(node1, node2));
}
@Test