From fbe3e86970cda0390eaf3c4820539a6dcb957926 Mon Sep 17 00:00:00 2001 From: Colin Patrick Mccabe Date: Fri, 18 Mar 2016 12:17:32 -0700 Subject: [PATCH 01/43] HADOOP-12626. Intel ISA-L libraries should be added to the Dockerfile (Kai Zheng via cmccabe) --- dev-support/docker/Dockerfile | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile index e80df296a9a..5c8bdfa88fc 100644 --- a/dev-support/docker/Dockerfile +++ b/dev-support/docker/Dockerfile @@ -45,6 +45,14 @@ RUN apt-get update && apt-get install --no-install-recommends -y \ # See http://wiki.apache.org/commons/VfsProblems RUN cd /usr/share/maven/lib && ln -s ../../java/commons-lang.jar . +###### +# Install ISA-L library +###### +RUN curl -L http://http.us.debian.org/debian/pool/main/libi/libisal/libisal2_2.15.0-2_amd64.deb \ + -o /opt/libisal2_2.15.0-2_amd64.deb && \ + dpkg -i /opt/libisal2_2.15.0-2_amd64.deb + + ####### # Oracle Java ####### From 33239c99257dac08f7644be3701770daa75044cd Mon Sep 17 00:00:00 2001 From: Chris Nauroth Date: Fri, 18 Mar 2016 16:33:01 -0700 Subject: [PATCH 02/43] HADOOP-12926. lz4.c does not detect 64-bit mode properly. Contributed by Alan Burlison. --- .../src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4.c index 39f176fafc3..5c51afe98ec 100644 --- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4.c +++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4.c @@ -52,7 +52,7 @@ || defined(__ppc64__) || defined(__ppc64le__) \ || defined(__PPC64__) || defined(__PPC64LE__) \ || defined(__ia64) || defined(__itanium__) || defined(_M_IA64) \ - || defined(__s390x__) ) /* Detects 64 bits mode */ + || defined(__s390x__) || defined(_LP64)) /* Detects 64 bits mode */ # define LZ4_ARCH64 1 #else # define LZ4_ARCH64 0 From cd8b6889a74a949e37f4b2eb664cdf3b59bfb93b Mon Sep 17 00:00:00 2001 From: Sangjin Lee Date: Sat, 19 Mar 2016 14:02:04 -0700 Subject: [PATCH 03/43] HDFS-9579. Provide bytes-read-by-network-distance metrics at FileSystem.Statistics level (Ming Ma via sjlee) --- .../java/org/apache/hadoop/fs/FileSystem.java | 118 +++++++++++++++++- .../java/org/apache/hadoop/net/NetUtils.java | 16 ++- .../apache/hadoop/net/NetworkTopology.java | 17 ++- .../java/org/apache/hadoop/net/NodeBase.java | 18 ++- .../org/apache/hadoop/hdfs/BlockReader.java | 10 +- .../hadoop/hdfs/BlockReaderFactory.java | 7 +- .../apache/hadoop/hdfs/BlockReaderLocal.java | 10 +- .../hadoop/hdfs/BlockReaderLocalLegacy.java | 10 +- .../org/apache/hadoop/hdfs/ClientContext.java | 56 ++++++++- .../org/apache/hadoop/hdfs/DFSClient.java | 11 +- .../apache/hadoop/hdfs/DFSInputStream.java | 14 +-- .../hadoop/hdfs/DFSStripedInputStream.java | 3 - .../hadoop/hdfs/ExternalBlockReader.java | 10 +- .../apache/hadoop/hdfs/RemoteBlockReader.java | 29 ++--- .../hadoop/hdfs/RemoteBlockReader2.java | 29 ++--- .../apache/hadoop/hdfs/ReplicaAccessor.java | 7 ++ .../erasurecode/ErasureCodingWorker.java | 3 +- .../fs/TestEnhancedByteBufferAccess.java | 4 +- .../hadoop/hdfs/TestBlockReaderLocal.java | 4 +- .../org/apache/hadoop/hdfs/TestConnCache.java | 2 - .../hdfs/TestDistributedFileSystem.java | 62 +++++++++ .../hadoop/hdfs/TestExternalBlockReader.java | 8 +- .../hadoop/net/TestNetworkTopology.java | 7 ++ 23 files changed, 368 insertions(+), 87 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index a96ea403a79..a8a5c6d2dc7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -3023,11 +3023,15 @@ public abstract class FileSystem extends Configured implements Closeable { * 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. @@ -3038,6 +3042,12 @@ public abstract class FileSystem extends Configured implements Closeable { 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; } /** @@ -3049,6 +3059,12 @@ public abstract class FileSystem extends Configured implements Closeable { 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 @@ -3077,6 +3093,22 @@ public abstract class FileSystem extends Configured implements Closeable { 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 { @@ -3267,6 +3299,33 @@ public abstract class FileSystem extends Configured implements Closeable { 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. @@ -3384,6 +3443,55 @@ public abstract class FileSystem extends Configured implements Closeable { }); } + /** + * 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() { + private StatisticsData all = new StatisticsData(); + + @Override + public void accept(StatisticsData data) { + all.add(data); + } + + public StatisticsData aggregate() { + return all; + } + }); + } @Override public String toString() { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java index e475149aaa4..2c3661aaf2c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java @@ -638,13 +638,27 @@ public class NetUtils { /** * 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() { try {return "" + InetAddress.getLocalHost();} catch(UnknownHostException uhe) {return "" + uhe;} } - + /** * Compose a "host:port" string from the address. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java index b637da11f22..e1d29689aec 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java @@ -369,6 +369,16 @@ public class NetworkTopology { 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 class NetworkTopology { * 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(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NodeBase.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NodeBase.java index b136297711c..b465098cb4d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NodeBase.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NodeBase.java @@ -112,7 +112,23 @@ public class NodeBase implements Node { 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() { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java index 150cf2339c5..63acaa7c2e2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java @@ -83,11 +83,6 @@ public interface BlockReader extends ByteBufferReadable, Closeable { */ 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. @@ -107,4 +102,9 @@ public interface BlockReader extends ByteBufferReadable, Closeable { * @return The DataChecksum used by the read block */ DataChecksum getDataChecksum(); + + /** + * Return the network distance between local machine and the remote machine. + */ + int getNetworkDistance(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java index 5c7bbd7dfa0..8a0050ff540 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java @@ -833,16 +833,19 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { @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); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java index 859380c9d5b..68630c785b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java @@ -640,11 +640,6 @@ class BlockReaderLocal implements BlockReader { return BlockReaderUtil.readAll(this, buf, off, len); } - @Override - public boolean isLocal() { - return true; - } - @Override public boolean isShortCircuit() { return true; @@ -721,4 +716,9 @@ class BlockReaderLocal implements BlockReader { public DataChecksum getDataChecksum() { return checksum; } + + @Override + public int getNetworkDistance() { + return 0; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java index 7206c071283..65a837394de 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java @@ -722,11 +722,6 @@ class BlockReaderLocalLegacy implements BlockReader { return Integer.MAX_VALUE; } - @Override - public boolean isLocal() { - return true; - } - @Override public boolean isShortCircuit() { return true; @@ -741,4 +736,9 @@ class BlockReaderLocalLegacy implements BlockReader { public DataChecksum getDataChecksum() { return checksum; } + + @Override + public int getNetworkDistance() { + return 0; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java index 047645bf584..47d6d492ef1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java @@ -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 nodeToDistance; + + private ClientContext(String name, DfsClientConf conf, + Configuration config) { final ShortCircuitConf scConf = conf.getShortCircuitConf(); this.name = name; @@ -116,14 +133,28 @@ public class ClientContext { 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 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 class ClientContext { 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 class ClientContext { @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 class ClientContext { 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; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 0976920060b..3506d3ac70a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -212,7 +212,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; @@ -357,7 +357,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, 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. @@ -2740,6 +2740,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, } } + 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. diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index d713e8f7836..7661e82b8a9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -775,7 +775,7 @@ public class DFSInputStream extends FSInputStream synchronized(infoLock) { if (blockReader.isShortCircuit()) { readStatistics.addShortCircuitBytes(nRead); - } else if (blockReader.isLocal()) { + } else if (blockReader.getNetworkDistance() == 0) { readStatistics.addLocalBytes(nRead); } else { readStatistics.addRemoteBytes(nRead); @@ -798,6 +798,8 @@ public class DFSInputStream extends FSInputStream throws IOException { int nRead = blockReader.read(buf, off, len); updateReadStatistics(readStatistics, nRead, blockReader); + dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(), + nRead); return nRead; } @@ -828,6 +830,8 @@ public class DFSInputStream extends FSInputStream int ret = blockReader.read(buf); success = true; updateReadStatistics(readStatistics, ret, blockReader); + dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(), + ret); if (ret == 0) { DFSClient.LOG.warn("zero"); } @@ -939,9 +943,6 @@ public class DFSInputStream extends FSInputStream // 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; @@ -1194,6 +1195,8 @@ public class DFSInputStream extends FSInputStream datanode.storageType, datanode.info); int nread = reader.readAll(buf, offset, len); updateReadStatistics(readStatistics, nread, reader); + dfsClient.updateFileSystemReadStats( + reader.getNetworkDistance(), nread); if (nread != len) { throw new IOException("truncated return from reader.read(): " + "excpected " + len + ", got " + nread); @@ -1479,9 +1482,6 @@ public class DFSInputStream extends FSInputStream offset += bytesToRead; } assert remaining == 0 : "Wrong number of bytes read."; - if (dfsClient.stats != null) { - dfsClient.stats.incrementBytesRead(realLen); - } return realLen; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java index f6547f36fcc..38236ad3821 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -447,9 +447,6 @@ public class DFSStripedInputStream extends DFSInputStream { result += ret; pos += ret; } - if (dfsClient.stats != null) { - dfsClient.stats.incrementBytesRead(result); - } return result; } finally { // Check if need to report block replicas corruption either read diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java index 42bec5c9d19..707a56a8a03 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java @@ -109,11 +109,6 @@ public final class ExternalBlockReader implements BlockReader { return BlockReaderUtil.readAll(this, buf, offset, len); } - @Override - public boolean isLocal() { - return accessor.isLocal(); - } - @Override public boolean isShortCircuit() { return accessor.isShortCircuit(); @@ -129,4 +124,9 @@ public final class ExternalBlockReader implements BlockReader { public DataChecksum getDataChecksum() { return null; } + + @Override + public int getNetworkDistance() { + return accessor.getNetworkDistance(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 544e1b3808f..7e094f54d29 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -44,7 +44,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.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 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader { 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 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader { 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 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader { checksumSize = this.checksum.getChecksumSize(); this.peerCache = peerCache; this.tracer = tracer; + this.networkDistance = networkDistance; } /** @@ -400,7 +395,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader { 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 class RemoteBlockReader extends FSInputChecker implements BlockReader { 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 class RemoteBlockReader extends FSInputChecker implements BlockReader { return RemoteBlockReader2.TCP_WINDOW_SIZE; } - @Override - public boolean isLocal() { - return isLocal; - } - @Override public boolean isShortCircuit() { return false; @@ -512,4 +502,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader { public DataChecksum getDataChecksum() { return checksum; } + + @Override + public int getNetworkDistance() { + return networkDistance; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java index 22e4757f1df..94373539213 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; 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 @@ public class RemoteBlockReader2 implements BlockReader { 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 @@ public class RemoteBlockReader2 implements BlockReader { bytesPerChecksum = this.checksum.getBytesPerChecksum(); checksumSize = this.checksum.getChecksumSize(); this.tracer = tracer; + this.networkDistance = networkDistance; } @@ -397,7 +393,8 @@ public class RemoteBlockReader2 implements BlockReader { 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 class RemoteBlockReader2 implements BlockReader { 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 class RemoteBlockReader2 implements BlockReader { return TCP_WINDOW_SIZE; } - @Override - public boolean isLocal() { - return isLocal; - } - @Override public boolean isShortCircuit() { return false; @@ -472,4 +464,9 @@ public class RemoteBlockReader2 implements BlockReader { public DataChecksum getDataChecksum() { return checksum; } + + @Override + public int getNetworkDistance() { + return networkDistance; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ReplicaAccessor.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ReplicaAccessor.java index e0b21e8f586..556c2c2ed40 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ReplicaAccessor.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ReplicaAccessor.java @@ -87,4 +87,11 @@ public abstract class ReplicaAccessor { * 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; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java index bde8d805b39..74fb3e1134d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -849,12 +849,13 @@ public final class ErasureCodingWorker { * read directly from DN and need to check the replica is FINALIZED * state, notice we should not use short-circuit local read which * requires config for domain-socket in UNIX or legacy config in Windows. + * The network distance value isn't used for this scenario. */ return RemoteBlockReader2.newBlockReader( "dummy", block, blockToken, offsetInBlock, block.getNumBytes() - offsetInBlock, true, "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo, - null, cachingStrategy, datanode.getTracer()); + null, cachingStrategy, datanode.getTracer(), -1); } catch (IOException e) { LOG.debug("Exception while creating remote block reader, datanode {}", dnInfo, e); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java index 3455f55a392..a1af1fc5bb0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java @@ -358,7 +358,7 @@ public class TestEnhancedByteBufferAccess { 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 class TestEnhancedByteBufferAccess { 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java index 2d6c63ae546..0048d2a7f68 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm; 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 @@ public class TestBlockReaderLocal { 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, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java index afa5d271fd1..8d2398d5fa5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java @@ -99,8 +99,6 @@ public class TestConnCache { 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]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java index 6217c454638..1db0da86235 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java @@ -70,6 +70,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; import org.apache.hadoop.hdfs.web.WebHdfsConstants; +import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.DataChecksum; @@ -651,6 +652,67 @@ public class TestDistributedFileSystem { 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 { final long seed = RAN.nextLong(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java index 2c36baa224d..5c2b6da02ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; 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 class TestExternalBlockReader { return true; } + @Override + public int getNetworkDistance() { + return 0; + } + synchronized String getError() { return error; } @@ -271,7 +277,7 @@ public class TestExternalBlockReader { 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(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java index 45f6cb4bb24..736230c4bd1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java @@ -129,6 +129,13 @@ public class TestNetworkTopology { 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 From 7fae4c68e6d599d0c01bb2cb2c8d5e52925b3e1e Mon Sep 17 00:00:00 2001 From: Karthik Kambatla Date: Sun, 20 Mar 2016 10:47:38 -0700 Subject: [PATCH 04/43] YARN-4732. *ProcessTree classes have too many whitespace issues --- .../org/apache/hadoop/mapreduce/util/ProcessTree.java | 7 ++----- .../apache/hadoop/yarn/util/ProcfsBasedProcessTree.java | 6 ++---- .../hadoop/yarn/util/ResourceCalculatorProcessTree.java | 8 ++++---- .../apache/hadoop/yarn/util/WindowsBasedProcessTree.java | 5 ++--- .../hadoop/yarn/util/TestProcfsBasedProcessTree.java | 8 -------- .../yarn/util/TestResourceCalculatorProcessTree.java | 3 +-- .../hadoop/yarn/util/TestWindowsBasedProcessTree.java | 3 +-- 7 files changed, 12 insertions(+), 28 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java index 1e2d16ef41d..6cf053edf0a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java @@ -48,8 +48,8 @@ public class ProcessTree { private static final String SIGTERM_STR = "SIGTERM"; private static final String SIGKILL_STR = "SIGKILL"; - public static final boolean isSetsidAvailable = isSetsidSupported(); + private static boolean isSetsidSupported() { ShellCommandExecutor shexec = null; boolean setsidSupported = true; @@ -114,7 +114,6 @@ public class ProcessTree { sigKill(pgrpId, true, sleeptimeBeforeSigkill, inBackground); } - /** * Send a specified signal to the specified pid * @@ -214,7 +213,6 @@ public class ProcessTree { } } } - /** Kills the process(OR process group) by sending the signal SIGKILL * @param pid Process id(OR process group id) of to-be-deleted-process @@ -301,7 +299,7 @@ public class ProcessTree { } return (shexec.getExitCode() == 0 ? true : false); } - + /** * Is the process group with still alive? * @@ -326,7 +324,6 @@ public class ProcessTree { } return (shexec.getExitCode() == 0 ? true : false); } - /** * Helper thread class that kills process-tree with SIGKILL in background diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java index c0c935667fc..7bd5390f3d7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java @@ -331,7 +331,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree { } return total; } - + @Override @SuppressWarnings("deprecation") public long getCumulativeVmem(int olderThanAge) { @@ -358,7 +358,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree { } return isAvailable ? totalPages * PAGE_SIZE : UNAVAILABLE; // convert # pages to byte } - + @Override @SuppressWarnings("deprecation") public long getCumulativeRssmem(int olderThanAge) { @@ -409,13 +409,11 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree { } } } - if (LOG.isDebugEnabled()) { LOG.debug(procMemInfo.toString()); } } } - } if (total > 0) { total *= KB_TO_BYTES; // convert to bytes diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java index 01d79d9f49b..7214c752072 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java @@ -78,7 +78,7 @@ public abstract class ResourceCalculatorProcessTree extends Configured { public long getVirtualMemorySize() { return getVirtualMemorySize(0); } - + /** * Get the virtual memory used by all the processes in the * process-tree. @@ -101,7 +101,7 @@ public abstract class ResourceCalculatorProcessTree extends Configured { public long getRssMemorySize() { return getRssMemorySize(0); } - + /** * Get the resident set size (rss) memory used by all the processes * in the process-tree. @@ -127,7 +127,7 @@ public abstract class ResourceCalculatorProcessTree extends Configured { public long getVirtualMemorySize(int olderThanAge) { return UNAVAILABLE; } - + /** * Get the virtual memory used by all the processes in the * process-tree that are older than the passed in age. @@ -156,7 +156,7 @@ public abstract class ResourceCalculatorProcessTree extends Configured { public long getRssMemorySize(int olderThanAge) { return UNAVAILABLE; } - + /** * Get the resident set size (rss) memory used by all the processes * in the process-tree that are older than the passed in age. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java index 7d5e7f933a1..785829221bf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java @@ -45,7 +45,6 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree { long cpuTimeMsDelta; // delta of cpuTime since last update int age = 1; } - private String taskProcessId = null; private long cpuTimeMs = UNAVAILABLE; private Map processTree = @@ -144,7 +143,7 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree { } return allProcs; } - + @Override public void updateProcessTree() { if(taskProcessId != null) { @@ -214,7 +213,7 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree { } return total; } - + @Override @SuppressWarnings("deprecation") public long getCumulativeVmem(int olderThanAge) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java index 4a7e6fff9c9..da7849d4fa3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java @@ -227,7 +227,6 @@ public class TestProcfsBasedProcessTree { // ProcessTree is gone now. Any further calls should be sane. p.updateProcessTree(); Assert.assertFalse("ProcessTree must have been gone", isAlive(pid)); - Assert.assertTrue( "vmem for the gone-process is " + p.getVirtualMemorySize() + " . It should be zero.", p.getVirtualMemorySize() == 0); @@ -581,7 +580,6 @@ public class TestProcfsBasedProcessTree { // verify virtual memory Assert.assertEquals("Virtual memory does not match", 700000L, processTree.getVirtualMemorySize()); - Assert.assertEquals("Virtual memory (old API) does not match", 700000L, processTree.getCumulativeVmem()); @@ -602,10 +600,8 @@ public class TestProcfsBasedProcessTree { processTree.updateProcessTree(); Assert.assertEquals("vmem does not include new process", 1200000L, processTree.getVirtualMemorySize()); - Assert.assertEquals("vmem (old API) does not include new process", 1200000L, processTree.getCumulativeVmem()); - if (!smapEnabled) { long cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 @@ -632,7 +628,6 @@ public class TestProcfsBasedProcessTree { Assert.assertEquals( "vmem (old API) shouldn't have included new process", 700000L, processTree.getCumulativeVmem(1)); - if (!smapEnabled) { long cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 @@ -645,7 +640,6 @@ public class TestProcfsBasedProcessTree { Assert.assertEquals( "rssmem (old API) shouldn't have included new process", cumuRssMem, processTree.getCumulativeRssmem(1)); - } else { Assert.assertEquals( "rssmem shouldn't have included new process", @@ -676,12 +670,10 @@ public class TestProcfsBasedProcessTree { Assert.assertEquals( "vmem shouldn't have included new processes", 700000L, processTree.getVirtualMemorySize(2)); - // verify old API Assert.assertEquals( "vmem (old API) shouldn't have included new processes", 700000L, processTree.getCumulativeVmem(2)); - if (!smapEnabled) { long cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java index 7a3e0e78666..28cee7f1028 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java @@ -44,7 +44,6 @@ public class TestResourceCalculatorProcessTree { public long getRssMemorySize(int age) { return 0; } - @SuppressWarnings("deprecation") public long getCumulativeRssmem(int age) { return 0; @@ -53,7 +52,7 @@ public class TestResourceCalculatorProcessTree { public long getVirtualMemorySize(int age) { return 0; } - + @SuppressWarnings("deprecation") public long getCumulativeVmem(int age) { return 0; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsBasedProcessTree.java index 290caa6938d..cbab1edf33e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsBasedProcessTree.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsBasedProcessTree.java @@ -29,7 +29,7 @@ import static org.junit.Assert.assertTrue; public class TestWindowsBasedProcessTree { private static final Log LOG = LogFactory .getLog(TestWindowsBasedProcessTree.class); - + class WindowsBasedProcessTreeTester extends WindowsBasedProcessTree { String infoStr = null; @@ -62,7 +62,6 @@ public class TestWindowsBasedProcessTree { assertTrue(pTree.getCumulativeVmem() == 2048); assertTrue(pTree.getVirtualMemorySize(0) == 2048); assertTrue(pTree.getCumulativeVmem(0) == 2048); - assertTrue(pTree.getRssMemorySize() == 2048); assertTrue(pTree.getCumulativeRssmem() == 2048); assertTrue(pTree.getRssMemorySize(0) == 2048); From 8a3f0cb25540c7e70471aebcdd408feb478f878e Mon Sep 17 00:00:00 2001 From: Haohui Mai Date: Sun, 20 Mar 2016 17:21:25 -0700 Subject: [PATCH 05/43] HDFS-7166. SbNN Web UI shows #Under replicated blocks and #pending deletion blocks. Contributed by Wei-Chiu Chuang. --- .../src/main/webapps/hdfs/dfshealth.html | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html index 7ad38d784d4..3d9ca4270e0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html @@ -106,7 +106,8 @@ {/if} -{@if cond="{NumberOfMissingBlocks} > 0"} +{@eq key=nnstat.State value="active"} + {@if cond="{NumberOfMissingBlocks} > 0"}
@@ -119,7 +120,8 @@

Please check the logs or run fsck in order to identify the missing blocks.{@if cond="{NumberOfSnapshottableDirs} > 0"} Please run fsck with -includeSnapshots in order to see detailed reports about snapshots.{/if} See the Hadoop FAQ for common causes and potential solutions.

-{/if} + {/if} +{/eq} {/nn} @@ -173,10 +175,13 @@ Dead Nodes{NumDeadDataNodes} (Decommissioned: {NumDecomDeadDataNodes}) Decommissioning Nodes{NumDecommissioningDataNodes} Total Datanode Volume Failures{VolumeFailuresTotal} ({EstimatedCapacityLostTotal|fmt_bytes}) - Number of Under-Replicated Blocks{UnderReplicatedBlocks} - Number of Blocks Pending Deletion{PendingDeletionBlocks} + {@eq key=nnstat.State value="active"} + Number of Under-Replicated Blocks{UnderReplicatedBlocks} + Number of Blocks Pending Deletion{PendingDeletionBlocks} + {/eq} Block Deletion Start Time{BlockDeletionStartTime|date_tostring} {/fs} + From ed1e23fe8f2d608e68c6d41e81f52d5b7773d96d Mon Sep 17 00:00:00 2001 From: Haohui Mai Date: Sun, 20 Mar 2016 17:40:59 -0700 Subject: [PATCH 06/43] HADOOP-12819. Migrate TestSaslRPC and related codes to rebase on ProtobufRpcEngine. Contributed by Kai Zheng. --- .../org/apache/hadoop/ipc/TestRpcBase.java | 192 ++++++++- .../org/apache/hadoop/ipc/TestSaslRPC.java | 395 +++++------------- .../security/TestDoAsEffectiveUser.java | 6 +- .../hadoop-common/src/test/proto/test.proto | 9 + .../src/test/proto/test_rpc_service.proto | 4 + ...ache.hadoop.security.token.TokenIdentifier | 2 +- 6 files changed, 305 insertions(+), 303 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java index 03fd31ed668..bc604a47ef2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java @@ -29,6 +29,22 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager; import org.junit.Assert; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ipc.protobuf.TestProtos; +import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.KerberosInfo; +import org.apache.hadoop.security.SaslRpcServer.AuthMethod; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.SecretManager; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.security.token.TokenInfo; +import org.apache.hadoop.security.token.TokenSelector; +import org.junit.Assert; + +import java.io.DataInput; +import java.io.DataOutput; import java.io.IOException; import java.lang.management.ManagementFactory; import java.lang.management.ThreadInfo; @@ -37,6 +53,8 @@ import java.net.InetSocketAddress; import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -149,6 +167,89 @@ public class TestRpcBase { return count; } + public static class TestTokenIdentifier extends TokenIdentifier { + private Text tokenid; + private Text realUser; + final static Text KIND_NAME = new Text("test.token"); + + public TestTokenIdentifier() { + this(new Text(), new Text()); + } + public TestTokenIdentifier(Text tokenid) { + this(tokenid, new Text()); + } + public TestTokenIdentifier(Text tokenid, Text realUser) { + this.tokenid = tokenid == null ? new Text() : tokenid; + this.realUser = realUser == null ? new Text() : realUser; + } + @Override + public Text getKind() { + return KIND_NAME; + } + @Override + public UserGroupInformation getUser() { + if (realUser.toString().isEmpty()) { + return UserGroupInformation.createRemoteUser(tokenid.toString()); + } else { + UserGroupInformation realUgi = UserGroupInformation + .createRemoteUser(realUser.toString()); + return UserGroupInformation + .createProxyUser(tokenid.toString(), realUgi); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + tokenid.readFields(in); + realUser.readFields(in); + } + @Override + public void write(DataOutput out) throws IOException { + tokenid.write(out); + realUser.write(out); + } + } + + public static class TestTokenSecretManager extends + SecretManager { + @Override + public byte[] createPassword(TestTokenIdentifier id) { + return id.getBytes(); + } + + @Override + public byte[] retrievePassword(TestTokenIdentifier id) + throws InvalidToken { + return id.getBytes(); + } + + @Override + public TestTokenIdentifier createIdentifier() { + return new TestTokenIdentifier(); + } + } + + public static class TestTokenSelector implements + TokenSelector { + @SuppressWarnings("unchecked") + @Override + public Token selectToken(Text service, + Collection> tokens) { + if (service == null) { + return null; + } + for (Token token : tokens) { + if (TestTokenIdentifier.KIND_NAME.equals(token.getKind()) + && service.equals(token.getService())) { + return (Token) token; + } + } + return null; + } + } + + @KerberosInfo(serverPrincipal = SERVER_PRINCIPAL_KEY) + @TokenInfo(TestTokenSelector.class) @ProtocolInfo(protocolName = "org.apache.hadoop.ipc.TestRpcBase$TestRpcService", protocolVersion = 1) public interface TestRpcService @@ -267,12 +368,80 @@ public class TestRpcBase { } catch (InterruptedException ignore) {} return TestProtos.EmptyResponseProto.newBuilder().build(); } + + @Override + public TestProtos.AuthMethodResponseProto getAuthMethod( + RpcController controller, TestProtos.EmptyRequestProto request) + throws ServiceException { + AuthMethod authMethod = null; + try { + authMethod = UserGroupInformation.getCurrentUser() + .getAuthenticationMethod().getAuthMethod(); + } catch (IOException e) { + throw new ServiceException(e); + } + + return TestProtos.AuthMethodResponseProto.newBuilder() + .setCode(authMethod.code) + .setMechanismName(authMethod.getMechanismName()) + .build(); + } + + @Override + public TestProtos.AuthUserResponseProto getAuthUser( + RpcController controller, TestProtos.EmptyRequestProto request) + throws ServiceException { + UserGroupInformation authUser = null; + try { + authUser = UserGroupInformation.getCurrentUser(); + } catch (IOException e) { + throw new ServiceException(e); + } + + return TestProtos.AuthUserResponseProto.newBuilder() + .setAuthUser(authUser.getUserName()) + .build(); + } + + @Override + public TestProtos.EchoResponseProto echoPostponed( + RpcController controller, TestProtos.EchoRequestProto request) + throws ServiceException { + Server.Call call = Server.getCurCall().get(); + call.postponeResponse(); + postponedCalls.add(call); + + return TestProtos.EchoResponseProto.newBuilder().setMessage( + request.getMessage()) + .build(); + } + + @Override + public TestProtos.EmptyResponseProto sendPostponed( + RpcController controller, TestProtos.EmptyRequestProto request) + throws ServiceException { + Collections.shuffle(postponedCalls); + try { + for (Server.Call call : postponedCalls) { + call.sendResponse(); + } + } catch (IOException e) { + throw new ServiceException(e); + } + postponedCalls.clear(); + + return TestProtos.EmptyResponseProto.newBuilder().build(); + } } protected static TestProtos.EmptyRequestProto newEmptyRequest() { return TestProtos.EmptyRequestProto.newBuilder().build(); } + protected static TestProtos.EmptyResponseProto newEmptyResponse() { + return TestProtos.EmptyResponseProto.newBuilder().build(); + } + protected static TestProtos.EchoRequestProto newEchoRequest(String msg) { return TestProtos.EchoRequestProto.newBuilder().setMessage(msg).build(); } @@ -290,6 +459,27 @@ public class TestRpcBase { protected static TestProtos.SleepRequestProto newSleepRequest( int milliSeconds) { return TestProtos.SleepRequestProto.newBuilder() - .setMilliSeconds(milliSeconds).build(); + .setMilliSeconds(milliSeconds).build(); + } + + protected static TestProtos.EchoResponseProto newEchoResponse(String msg) { + return TestProtos.EchoResponseProto.newBuilder().setMessage(msg).build(); + } + + protected static AuthMethod convert( + TestProtos.AuthMethodResponseProto authMethodResponse) { + String mechanism = authMethodResponse.getMechanismName(); + if (mechanism.equals(AuthMethod.SIMPLE.getMechanismName())) { + return AuthMethod.SIMPLE; + } else if (mechanism.equals(AuthMethod.KERBEROS.getMechanismName())) { + return AuthMethod.KERBEROS; + } else if (mechanism.equals(AuthMethod.TOKEN.getMechanismName())) { + return AuthMethod.TOKEN; + } + return null; + } + + protected static String convert(TestProtos.AuthUserResponseProto response) { + return response.getAuthUser(); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java index 8df3b1ded6e..ec53e8c9762 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java @@ -18,53 +18,7 @@ package org.apache.hadoop.ipc; -import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION; -import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; -import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.KERBEROS; -import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.SIMPLE; -import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.TOKEN; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.lang.annotation.Annotation; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.security.PrivilegedExceptionAction; -import java.security.Security; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Pattern; - -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.auth.callback.UnsupportedCallbackException; -import javax.security.sasl.AuthorizeCallback; -import javax.security.sasl.Sasl; -import javax.security.sasl.SaslClient; -import javax.security.sasl.SaslException; -import javax.security.sasl.SaslServer; - +import com.google.protobuf.ServiceException; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -74,27 +28,13 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.Client.ConnectionId; -import org.apache.hadoop.ipc.Server.Call; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.KerberosInfo; -import org.apache.hadoop.security.SaslInputStream; -import org.apache.hadoop.security.SaslPlainServer; -import org.apache.hadoop.security.SaslPropertiesResolver; -import org.apache.hadoop.security.SaslRpcClient; -import org.apache.hadoop.security.SaslRpcServer; +import org.apache.hadoop.security.*; import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection; -import org.apache.hadoop.security.SecurityInfo; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.TestUserGroupInformation; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; -import org.apache.hadoop.security.token.SecretManager; +import org.apache.hadoop.security.token.*; import org.apache.hadoop.security.token.SecretManager.InvalidToken; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.security.token.TokenInfo; -import org.apache.hadoop.security.token.TokenSelector; import org.apache.log4j.Level; import org.junit.Assert; import org.junit.Before; @@ -104,9 +44,27 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; +import javax.security.auth.callback.*; +import javax.security.sasl.*; +import java.io.IOException; +import java.lang.annotation.Annotation; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.security.PrivilegedExceptionAction; +import java.security.Security; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; + +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; +import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.*; +import static org.junit.Assert.*; + /** Unit tests for using Sasl over RPC. */ @RunWith(Parameterized.class) -public class TestSaslRPC { +public class TestSaslRPC extends TestRpcBase { @Parameters public static Collection data() { Collection params = new ArrayList(); @@ -135,18 +93,14 @@ public class TestSaslRPC { this.expectedQop = expectedQop; this.saslPropertiesResolver = saslPropertiesResolver; } - - private static final String ADDRESS = "0.0.0.0"; public static final Log LOG = LogFactory.getLog(TestSaslRPC.class); static final String ERROR_MESSAGE = "Token is invalid"; - static final String SERVER_PRINCIPAL_KEY = "test.ipc.server.principal"; static final String SERVER_KEYTAB_KEY = "test.ipc.server.keytab"; static final String SERVER_PRINCIPAL_1 = "p1/foo@BAR"; - static final String SERVER_PRINCIPAL_2 = "p2/foo@BAR"; - private static Configuration conf; + // If this is set to true AND the auth-method is not simple, secretManager // will be enabled. static Boolean enableSecretManager = null; @@ -155,7 +109,7 @@ public class TestSaslRPC { static Boolean forceSecretManager = null; static Boolean clientFallBackToSimpleAllowed = true; - static enum UseToken { + enum UseToken { NONE(), VALID(), INVALID(), @@ -174,6 +128,7 @@ public class TestSaslRPC { LOG.info("---------------------------------"); LOG.info("Testing QOP:"+ getQOPNames(qop)); LOG.info("---------------------------------"); + conf = new Configuration(); // the specific tests for kerberos will enable kerberos. forcing it // for all tests will cause tests to fail if the user has a TGT @@ -187,6 +142,9 @@ public class TestSaslRPC { enableSecretManager = null; forceSecretManager = null; clientFallBackToSimpleAllowed = true; + + // Set RPC engine to protobuf RPC engine + RPC.setProtocolEngine(conf, TestRpcService.class, ProtobufRpcEngine.class); } static String getQOPNames (QualityOfProtection[] qops){ @@ -210,68 +168,6 @@ public class TestSaslRPC { ((Log4JLogger) SecurityUtil.LOG).getLogger().setLevel(Level.ALL); } - public static class TestTokenIdentifier extends TokenIdentifier { - private Text tokenid; - private Text realUser; - final static Text KIND_NAME = new Text("test.token"); - - public TestTokenIdentifier() { - this(new Text(), new Text()); - } - public TestTokenIdentifier(Text tokenid) { - this(tokenid, new Text()); - } - public TestTokenIdentifier(Text tokenid, Text realUser) { - this.tokenid = tokenid == null ? new Text() : tokenid; - this.realUser = realUser == null ? new Text() : realUser; - } - @Override - public Text getKind() { - return KIND_NAME; - } - @Override - public UserGroupInformation getUser() { - if (realUser.toString().isEmpty()) { - return UserGroupInformation.createRemoteUser(tokenid.toString()); - } else { - UserGroupInformation realUgi = UserGroupInformation - .createRemoteUser(realUser.toString()); - return UserGroupInformation - .createProxyUser(tokenid.toString(), realUgi); - } - } - - @Override - public void readFields(DataInput in) throws IOException { - tokenid.readFields(in); - realUser.readFields(in); - } - @Override - public void write(DataOutput out) throws IOException { - tokenid.write(out); - realUser.write(out); - } - } - - public static class TestTokenSecretManager extends - SecretManager { - @Override - public byte[] createPassword(TestTokenIdentifier id) { - return id.getBytes(); - } - - @Override - public byte[] retrievePassword(TestTokenIdentifier id) - throws InvalidToken { - return id.getBytes(); - } - - @Override - public TestTokenIdentifier createIdentifier() { - return new TestTokenIdentifier(); - } - } - public static class BadTokenSecretManager extends TestTokenSecretManager { @Override @@ -281,64 +177,6 @@ public class TestSaslRPC { } } - public static class TestTokenSelector implements - TokenSelector { - @SuppressWarnings("unchecked") - @Override - public Token selectToken(Text service, - Collection> tokens) { - if (service == null) { - return null; - } - for (Token token : tokens) { - if (TestTokenIdentifier.KIND_NAME.equals(token.getKind()) - && service.equals(token.getService())) { - return (Token) token; - } - } - return null; - } - } - - @KerberosInfo( - serverPrincipal = SERVER_PRINCIPAL_KEY) - @TokenInfo(TestTokenSelector.class) - public interface TestSaslProtocol extends TestRPC.TestProtocol { - public AuthMethod getAuthMethod() throws IOException; - public String getAuthUser() throws IOException; - public String echoPostponed(String value) throws IOException; - public void sendPostponed() throws IOException; - } - - public static class TestSaslImpl extends TestRPC.TestImpl implements - TestSaslProtocol { - private List postponedCalls = new ArrayList(); - @Override - public AuthMethod getAuthMethod() throws IOException { - return UserGroupInformation.getCurrentUser() - .getAuthenticationMethod().getAuthMethod(); - } - @Override - public String getAuthUser() throws IOException { - return UserGroupInformation.getCurrentUser().getUserName(); - } - @Override - public String echoPostponed(String value) { - Call call = Server.getCurCall().get(); - call.postponeResponse(); - postponedCalls.add(call); - return value; - } - @Override - public void sendPostponed() throws IOException { - Collections.shuffle(postponedCalls); - for (Call call : postponedCalls) { - call.sendResponse(); - } - postponedCalls.clear(); - } - } - public static class CustomSecurityInfo extends SecurityInfo { @Override @@ -363,8 +201,8 @@ public class TestSaslRPC { public TokenInfo getTokenInfo(Class protocol, Configuration conf) { return new TokenInfo() { @Override - public Class> value() { + public Class> value() { return TestTokenSelector.class; } @Override @@ -378,10 +216,7 @@ public class TestSaslRPC { @Test public void testDigestRpc() throws Exception { TestTokenSecretManager sm = new TestTokenSecretManager(); - final Server server = new RPC.Builder(conf) - .setProtocol(TestSaslProtocol.class).setInstance(new TestSaslImpl()) - .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true) - .setSecretManager(sm).build(); + final Server server = setupTestServer(conf, 5, sm); doDigestRpc(server, sm); } @@ -391,10 +226,7 @@ public class TestSaslRPC { TestTokenSecretManager sm = new TestTokenSecretManager(); try { SecurityUtil.setSecurityInfoProviders(new CustomSecurityInfo()); - final Server server = new RPC.Builder(conf) - .setProtocol(TestSaslProtocol.class).setInstance(new TestSaslImpl()) - .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5) - .setVerbose(true).setSecretManager(sm).build(); + final Server server = setupTestServer(conf, 5, sm); doDigestRpc(server, sm); } finally { SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]); @@ -404,59 +236,47 @@ public class TestSaslRPC { @Test public void testErrorMessage() throws Exception { BadTokenSecretManager sm = new BadTokenSecretManager(); - final Server server = new RPC.Builder(conf) - .setProtocol(TestSaslProtocol.class).setInstance(new TestSaslImpl()) - .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true) - .setSecretManager(sm).build(); + final Server server = setupTestServer(conf, 5, sm); boolean succeeded = false; try { doDigestRpc(server, sm); - } catch (RemoteException e) { - LOG.info("LOGGING MESSAGE: " + e.getLocalizedMessage()); - assertEquals(ERROR_MESSAGE, e.getLocalizedMessage()); - assertTrue(e.unwrapRemoteException() instanceof InvalidToken); + } catch (ServiceException e) { + assertTrue(e.getCause() instanceof RemoteException); + RemoteException re = (RemoteException) e.getCause(); + LOG.info("LOGGING MESSAGE: " + re.getLocalizedMessage()); + assertEquals(ERROR_MESSAGE, re.getLocalizedMessage()); + assertTrue(re.unwrapRemoteException() instanceof InvalidToken); succeeded = true; } assertTrue(succeeded); } - private void doDigestRpc(Server server, TestTokenSecretManager sm - ) throws Exception { - server.start(); - + private void doDigestRpc(Server server, TestTokenSecretManager sm) + throws Exception { final UserGroupInformation current = UserGroupInformation.getCurrentUser(); - final InetSocketAddress addr = NetUtils.getConnectAddress(server); + addr = NetUtils.getConnectAddress(server); TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current .getUserName())); - Token token = new Token(tokenId, - sm); + Token token = new Token(tokenId, sm); SecurityUtil.setTokenService(token, addr); current.addToken(token); - TestSaslProtocol proxy = null; + TestRpcService proxy = null; try { - proxy = RPC.getProxy(TestSaslProtocol.class, - TestSaslProtocol.versionID, addr, conf); - AuthMethod authMethod = proxy.getAuthMethod(); + proxy = getClient(addr, conf); + AuthMethod authMethod = convert( + proxy.getAuthMethod(null, newEmptyRequest())); assertEquals(TOKEN, authMethod); //QOP must be auth assertEquals(expectedQop.saslQop, RPC.getConnectionIdForProxy(proxy).getSaslQop()); - proxy.ping(); + proxy.ping(null, newEmptyRequest()); } finally { - server.stop(); - if (proxy != null) { - RPC.stopProxy(proxy); - } + stop(server, proxy); } } - static ConnectionId getConnectionId(Configuration conf) throws IOException { - return ConnectionId.getConnectionId(new InetSocketAddress(0), - TestSaslProtocol.class, null, 0, null, conf); - } - @Test public void testPingInterval() throws Exception { Configuration newConf = new Configuration(conf); @@ -466,29 +286,26 @@ public class TestSaslRPC { // set doPing to true newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true); - ConnectionId remoteId = getConnectionId(newConf); + ConnectionId remoteId = ConnectionId.getConnectionId(new InetSocketAddress(0), + TestRpcService.class, null, 0, null, newConf); assertEquals(CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT, remoteId.getPingInterval()); // set doPing to false newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, false); - remoteId = getConnectionId(newConf); + remoteId = ConnectionId.getConnectionId(new InetSocketAddress(0), + TestRpcService.class, null, 0, null, newConf); assertEquals(0, remoteId.getPingInterval()); } @Test public void testPerConnectionConf() throws Exception { TestTokenSecretManager sm = new TestTokenSecretManager(); - final Server server = new RPC.Builder(conf) - .setProtocol(TestSaslProtocol.class).setInstance(new TestSaslImpl()) - .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true) - .setSecretManager(sm).build(); - server.start(); + final Server server = setupTestServer(conf, 5, sm); final UserGroupInformation current = UserGroupInformation.getCurrentUser(); final InetSocketAddress addr = NetUtils.getConnectAddress(server); TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current .getUserName())); - Token token = new Token(tokenId, - sm); + Token token = new Token<>(tokenId, sm); SecurityUtil.setTokenService(token, addr); current.addToken(token); @@ -497,28 +314,25 @@ public class TestSaslRPC { HADOOP_RPC_SOCKET_FACTORY_CLASS_DEFAULT_KEY, ""); Client client = null; - TestSaslProtocol proxy1 = null; - TestSaslProtocol proxy2 = null; - TestSaslProtocol proxy3 = null; + TestRpcService proxy1 = null; + TestRpcService proxy2 = null; + TestRpcService proxy3 = null; int timeouts[] = {111222, 3333333}; try { newConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, timeouts[0]); - proxy1 = RPC.getProxy(TestSaslProtocol.class, - TestSaslProtocol.versionID, addr, newConf); - proxy1.getAuthMethod(); - client = WritableRpcEngine.getClient(newConf); + proxy1 = getClient(addr, newConf); + proxy1.getAuthMethod(null, newEmptyRequest()); + client = ProtobufRpcEngine.getClient(newConf); Set conns = client.getConnectionIds(); assertEquals("number of connections in cache is wrong", 1, conns.size()); // same conf, connection should be re-used - proxy2 = RPC.getProxy(TestSaslProtocol.class, - TestSaslProtocol.versionID, addr, newConf); - proxy2.getAuthMethod(); + proxy2 = getClient(addr, newConf); + proxy2.getAuthMethod(null, newEmptyRequest()); assertEquals("number of connections in cache is wrong", 1, conns.size()); // different conf, new connection should be set up newConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, timeouts[1]); - proxy3 = RPC.getProxy(TestSaslProtocol.class, - TestSaslProtocol.versionID, addr, newConf); - proxy3.getAuthMethod(); + proxy3 = getClient(addr, newConf); + proxy3.getAuthMethod(null, newEmptyRequest()); assertEquals("number of connections in cache is wrong", 2, conns.size()); // now verify the proxies have the correct connection ids and timeouts ConnectionId[] connsArray = { @@ -551,24 +365,14 @@ public class TestSaslRPC { UserGroupInformation current = UserGroupInformation.getCurrentUser(); System.out.println("UGI: " + current); - Server server = new RPC.Builder(newConf) - .setProtocol(TestSaslProtocol.class).setInstance(new TestSaslImpl()) - .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true) - .build(); - TestSaslProtocol proxy = null; + Server server = setupTestServer(newConf, 5); + TestRpcService proxy = null; - server.start(); - - InetSocketAddress addr = NetUtils.getConnectAddress(server); try { - proxy = RPC.getProxy(TestSaslProtocol.class, - TestSaslProtocol.versionID, addr, newConf); - proxy.ping(); + proxy = getClient(addr, newConf); + proxy.ping(null, newEmptyRequest()); } finally { - server.stop(); - if (proxy != null) { - RPC.stopProxy(proxy); - } + stop(server, proxy); } System.out.println("Test is successful."); } @@ -887,14 +691,7 @@ public class TestSaslRPC { UserGroupInformation.setConfiguration(conf); TestTokenSecretManager sm = new TestTokenSecretManager(); - Server server = new RPC.Builder(conf) - .setProtocol(TestSaslProtocol.class) - .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0) - .setNumHandlers(1) // prevents ordering issues when unblocking calls. - .setVerbose(true) - .setSecretManager(sm) - .build(); - server.start(); + Server server = setupTestServer(conf, 1, sm); try { final InetSocketAddress addr = NetUtils.getConnectAddress(server); final UserGroupInformation clientUgi = @@ -903,14 +700,13 @@ public class TestSaslRPC { TestTokenIdentifier tokenId = new TestTokenIdentifier( new Text(clientUgi.getUserName())); - Token token = new Token(tokenId, sm); + Token token = new Token<>(tokenId, sm); SecurityUtil.setTokenService(token, addr); clientUgi.addToken(token); clientUgi.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - final TestSaslProtocol proxy = RPC.getProxy(TestSaslProtocol.class, - TestSaslProtocol.versionID, addr, conf); + final TestRpcService proxy = getClient(addr, conf); final ExecutorService executor = Executors.newCachedThreadPool(); final AtomicInteger count = new AtomicInteger(); try { @@ -922,7 +718,8 @@ public class TestSaslRPC { @Override public Void call() throws Exception { String expect = "future"+count.getAndIncrement(); - String answer = proxy.echoPostponed(expect); + String answer = convert(proxy.echoPostponed(null, + newEchoRequest(expect))); assertEquals(expect, answer); return null; } @@ -939,7 +736,7 @@ public class TestSaslRPC { // only 1 handler ensures that the prior calls are already // postponed. 1 handler also ensures that this call will // timeout if the postponing doesn't work (ie. free up handler) - proxy.sendPostponed(); + proxy.sendPostponed(null, newEmptyRequest()); for (int i=0; i < futures.length; i++) { LOG.info("waiting for future"+i); futures[i].get(); @@ -1009,14 +806,7 @@ public class TestSaslRPC { Server server = serverUgi.doAs(new PrivilegedExceptionAction() { @Override public Server run() throws IOException { - Server server = new RPC.Builder(serverConf) - .setProtocol(TestSaslProtocol.class) - .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0) - .setNumHandlers(5).setVerbose(true) - .setSecretManager(serverSm) - .build(); - server.start(); - return server; + return setupTestServer(serverConf, 5, serverSm); } }); @@ -1038,17 +828,17 @@ public class TestSaslRPC { Token token = null; switch (tokenType) { case VALID: - token = new Token(tokenId, sm); + token = new Token<>(tokenId, sm); SecurityUtil.setTokenService(token, addr); break; case INVALID: - token = new Token( + token = new Token<>( tokenId.getBytes(), "bad-password!".getBytes(), tokenId.getKind(), null); SecurityUtil.setTokenService(token, addr); break; case OTHER: - token = new Token(); + token = new Token<>(); break; case NONE: // won't get here } @@ -1060,19 +850,28 @@ public class TestSaslRPC { return clientUgi.doAs(new PrivilegedExceptionAction() { @Override public String run() throws IOException { - TestSaslProtocol proxy = null; + TestRpcService proxy = null; try { - proxy = RPC.getProxy(TestSaslProtocol.class, - TestSaslProtocol.versionID, addr, clientConf); - - proxy.ping(); + proxy = getClient(addr, clientConf); + + proxy.ping(null, newEmptyRequest()); // make sure the other side thinks we are who we said we are!!! - assertEquals(clientUgi.getUserName(), proxy.getAuthUser()); - AuthMethod authMethod = proxy.getAuthMethod(); + assertEquals(clientUgi.getUserName(), + convert(proxy.getAuthUser(null, newEmptyRequest()))); + AuthMethod authMethod = + convert(proxy.getAuthMethod(null, newEmptyRequest())); // verify sasl completed with correct QOP assertEquals((authMethod != SIMPLE) ? expectedQop.saslQop : null, - RPC.getConnectionIdForProxy(proxy).getSaslQop()); + RPC.getConnectionIdForProxy(proxy).getSaslQop()); return authMethod.toString(); + } catch (ServiceException se) { + if (se.getCause() instanceof RemoteException) { + throw (RemoteException) se.getCause(); + } else if (se.getCause() instanceof IOException) { + throw (IOException) se.getCause(); + } else { + throw new RuntimeException(se.getCause()); + } } finally { if (proxy != null) { RPC.stopProxy(proxy); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java index b44fa8b85af..50d389c6465 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java @@ -41,9 +41,9 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenInfo; import org.junit.Before; import org.junit.Test; -import org.apache.hadoop.ipc.TestSaslRPC.TestTokenSecretManager; -import org.apache.hadoop.ipc.TestSaslRPC.TestTokenIdentifier; -import org.apache.hadoop.ipc.TestSaslRPC.TestTokenSelector; +import org.apache.hadoop.ipc.TestRpcBase.TestTokenSecretManager; +import org.apache.hadoop.ipc.TestRpcBase.TestTokenIdentifier; +import org.apache.hadoop.ipc.TestRpcBase.TestTokenSelector; import org.apache.commons.logging.*; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test.proto b/hadoop-common-project/hadoop-common/src/test/proto/test.proto index ba0038d0d15..99cd93d711c 100644 --- a/hadoop-common-project/hadoop-common/src/test/proto/test.proto +++ b/hadoop-common-project/hadoop-common/src/test/proto/test.proto @@ -81,4 +81,13 @@ message ExchangeRequestProto { message ExchangeResponseProto { repeated int32 values = 1; +} + +message AuthMethodResponseProto { + required int32 code = 1; + required string mechanismName = 2; +} + +message AuthUserResponseProto { + required string authUser = 1; } \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto index abb38831e5f..32921158857 100644 --- a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto +++ b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto @@ -39,6 +39,10 @@ service TestProtobufRpcProto { rpc testServerGet(EmptyRequestProto) returns (EmptyResponseProto); rpc exchange(ExchangeRequestProto) returns (ExchangeResponseProto); rpc sleep(SleepRequestProto) returns (EmptyResponseProto); + rpc getAuthMethod(EmptyRequestProto) returns (AuthMethodResponseProto); + rpc getAuthUser(EmptyRequestProto) returns (AuthUserResponseProto); + rpc echoPostponed(EchoRequestProto) returns (EchoResponseProto); + rpc sendPostponed(EmptyRequestProto) returns (EmptyResponseProto); } service TestProtobufRpc2Proto { diff --git a/hadoop-common-project/hadoop-common/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier b/hadoop-common-project/hadoop-common/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier index 56eab0553d2..036b9b4d3ed 100644 --- a/hadoop-common-project/hadoop-common/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier +++ b/hadoop-common-project/hadoop-common/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier @@ -11,5 +11,5 @@ # See the License for the specific language governing permissions and # limitations under the License. # -org.apache.hadoop.ipc.TestSaslRPC$TestTokenIdentifier +org.apache.hadoop.ipc.TestRpcBase$TestTokenIdentifier org.apache.hadoop.security.token.delegation.TestDelegationToken$TestDelegationTokenIdentifier From e3bb38d62567eafe57d16b78deeba1b71c58e41c Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Mon, 21 Mar 2016 11:39:05 -0700 Subject: [PATCH 07/43] HDFS-9405. Warmup NameNode EDEK caches in background thread. Contributed by Xiao Chen. --- .../crypto/key/kms/KMSClientProvider.java | 8 +- .../hadoop/crypto/key/kms/ValueQueue.java | 16 +++- .../hadoop/crypto/key/TestValueQueue.java | 6 -- .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 4 + .../namenode/EncryptionZoneManager.java | 15 ++++ .../namenode/FSDirEncryptionZoneOp.java | 83 +++++++++++++++++++ .../hdfs/server/namenode/FSNamesystem.java | 27 ++++++ .../src/main/resources/hdfs-default.xml | 18 ++++ .../hadoop/hdfs/TestEncryptionZones.java | 4 +- .../hdfs/TestEncryptionZonesWithKMS.java | 36 +++++++- 10 files changed, 197 insertions(+), 20 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java index b894c7f14ca..32ef09c9663 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java @@ -849,12 +849,8 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension, } @VisibleForTesting - public int getEncKeyQueueSize(String keyName) throws IOException { - try { - return encKeyVersionQueue.getSize(keyName); - } catch (ExecutionException e) { - throw new IOException(e); - } + public int getEncKeyQueueSize(String keyName) { + return encKeyVersionQueue.getSize(keyName); } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java index 32451d8360d..f38a6b3c984 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java @@ -18,9 +18,11 @@ package org.apache.hadoop.crypto.key.kms; import java.io.IOException; +import java.util.Arrays; import java.util.HashSet; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Queue; import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingQueue; @@ -240,13 +242,19 @@ public class ValueQueue { } /** - * Get size of the Queue for keyName + * Get size of the Queue for keyName. This is only used in unit tests. * @param keyName the key name * @return int queue size - * @throws ExecutionException */ - public int getSize(String keyName) throws ExecutionException { - return keyQueues.get(keyName).size(); + public int getSize(String keyName) { + // We can't do keyQueues.get(keyName).size() here, + // since that will have the side effect of populating the cache. + Map> map = + keyQueues.getAllPresent(Arrays.asList(keyName)); + if (map.get(keyName) == null) { + return 0; + } + return map.get(keyName).size(); } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java index 5eae9a006d2..abc4ebf9b4d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java @@ -19,7 +19,6 @@ package org.apache.hadoop.crypto.key; import java.io.IOException; import java.util.Queue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -158,17 +157,12 @@ public class TestValueQueue { GenericTestUtils.waitFor(new Supplier() { @Override public Boolean get() { - try { int size = vq.getSize("k1"); if (size != 10) { LOG.info("Current ValueQueue size is " + size); return false; } return true; - } catch (ExecutionException e) { - LOG.error("Exception when getSize.", e); - return false; - } } }, 100, 3000); Assert.assertEquals("Failed in async call.", 10, filler.getTop().num); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 5eaada44e59..844fec29843 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -750,6 +750,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES = "dfs.namenode.list.encryption.zones.num.responses"; public static final String DFS_ENCRYPTION_KEY_PROVIDER_URI = HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI; + public static final String DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_KEY = "dfs.namenode.edekcacheloader.interval.ms"; + public static final int DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_DEFAULT = 1000; + public static final String DFS_NAMENODE_EDEKCACHELOADER_INITIAL_DELAY_MS_KEY = "dfs.namenode.edekcacheloader.initial.delay.ms"; + public static final int DFS_NAMENODE_EDEKCACHELOADER_INITIAL_DELAY_MS_DEFAULT = 3000; // Journal-node related configs. These are read on the JN side. public static final String DFS_JOURNALNODE_EDITS_DIR_KEY = "dfs.journalnode.edits.dir"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java index d1621a8bb78..8454c0411c6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java @@ -21,6 +21,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.EnumSet; import java.util.List; +import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; @@ -380,4 +381,18 @@ public class EncryptionZoneManager { public int getNumEncryptionZones() { return encryptionZones.size(); } + + /** + * @return a list of all key names. + */ + String[] getKeyNames() { + assert dir.hasReadLock(); + String[] ret = new String[encryptionZones.size()]; + int index = 0; + for (Map.Entry entry : encryptionZones + .entrySet()) { + ret[index] = entry.getValue().getKeyName(); + } + return ret; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java index b663415a822..bd254190ebf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java @@ -23,6 +23,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.security.GeneralSecurityException; import java.util.AbstractMap; +import java.util.concurrent.ExecutorService; import java.util.EnumSet; import java.util.List; import java.util.Map; @@ -304,4 +305,86 @@ final class FSDirEncryptionZoneOp { fsd.readUnlock(); } } + + /** + * Proactively warm up the edek cache. We'll get all the edek key names, + * then launch up a separate thread to warm them up. + */ + static void warmUpEdekCache(final ExecutorService executor, + final FSDirectory fsd, final int delay, final int interval) { + fsd.readLock(); + try { + String[] edeks = fsd.ezManager.getKeyNames(); + executor.execute( + new EDEKCacheLoader(edeks, fsd.getProvider(), delay, interval)); + } finally { + fsd.readUnlock(); + } + } + + /** + * EDEKCacheLoader is being run in a separate thread to loop through all the + * EDEKs and warm them up in the KMS cache. + */ + static class EDEKCacheLoader implements Runnable { + private final String[] keyNames; + private final KeyProviderCryptoExtension kp; + private int initialDelay; + private int retryInterval; + + EDEKCacheLoader(final String[] names, final KeyProviderCryptoExtension kp, + final int delay, final int interval) { + this.keyNames = names; + this.kp = kp; + this.initialDelay = delay; + this.retryInterval = interval; + } + + @Override + public void run() { + NameNode.LOG.info("Warming up {} EDEKs... (initialDelay={}, " + + "retryInterval={})", keyNames.length, initialDelay, retryInterval); + try { + Thread.sleep(initialDelay); + } catch (InterruptedException ie) { + NameNode.LOG.info("EDEKCacheLoader interrupted before warming up."); + return; + } + + final int logCoolDown = 10000; // periodically print error log (if any) + int sinceLastLog = logCoolDown; // always print the first failure + boolean success = false; + IOException lastSeenIOE = null; + while (true) { + try { + kp.warmUpEncryptedKeys(keyNames); + NameNode.LOG + .info("Successfully warmed up {} EDEKs.", keyNames.length); + success = true; + break; + } catch (IOException ioe) { + lastSeenIOE = ioe; + if (sinceLastLog >= logCoolDown) { + NameNode.LOG.info("Failed to warm up EDEKs.", ioe); + sinceLastLog = 0; + } else { + NameNode.LOG.debug("Failed to warm up EDEKs.", ioe); + } + } + try { + Thread.sleep(retryInterval); + } catch (InterruptedException ie) { + NameNode.LOG.info("EDEKCacheLoader interrupted during retry."); + break; + } + sinceLastLog += retryInterval; + } + if (!success) { + NameNode.LOG.warn("Unable to warm up EDEKs."); + if (lastSeenIOE != null) { + NameNode.LOG.warn("Last seen exception:", lastSeenIOE); + } + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index a5b9dc29fa1..9ff4be637d1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -116,6 +116,8 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -283,6 +285,7 @@ import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ThreadFactoryBuilder; /** * FSNamesystem is a container of both transient @@ -425,6 +428,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, // A daemon to periodically clean up corrupt lazyPersist files // from the name space. Daemon lazyPersistFileScrubber = null; + + // Executor to warm up EDEK cache + private ExecutorService edekCacheLoader = null; + private final int edekCacheLoaderDelay; + private final int edekCacheLoaderInterval; + /** * When an active namenode will roll its own edit log, in # edits */ @@ -787,6 +796,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, + " must be zero (for disable) or greater than zero."); } + this.edekCacheLoaderDelay = conf.getInt( + DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INITIAL_DELAY_MS_KEY, + DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INITIAL_DELAY_MS_DEFAULT); + this.edekCacheLoaderInterval = conf.getInt( + DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_KEY, + DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_DEFAULT); + // For testing purposes, allow the DT secret manager to be started regardless // of whether security is enabled. alwaysUseDelegationTokensForTests = conf.getBoolean( @@ -1128,6 +1144,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, cacheManager.startMonitorThread(); blockManager.getDatanodeManager().setShouldSendCachingCommands(true); + if (provider != null) { + edekCacheLoader = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("Warm Up EDEK Cache Thread #%d") + .build()); + FSDirEncryptionZoneOp.warmUpEdekCache(edekCacheLoader, dir, + edekCacheLoaderDelay, edekCacheLoaderInterval); + } } finally { startingActiveService = false; writeUnlock(); @@ -1162,6 +1186,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, ((NameNodeResourceMonitor) nnrmthread.getRunnable()).stopMonitor(); nnrmthread.interrupt(); } + if (edekCacheLoader != null) { + edekCacheLoader.shutdownNow(); + } if (nnEditLogRoller != null) { ((NameNodeEditLogRoller)nnEditLogRoller.getRunnable()).stop(); nnEditLogRoller.interrupt(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index d1f78cadeb4..1e87626afe7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -2606,6 +2606,24 @@ + + dfs.namenode.edekcacheloader.interval.ms + 1000 + When KeyProvider is configured, the interval time of warming + up edek cache on NN starts up / becomes active. All edeks will be loaded + from KMS into provider cache. The edek cache loader will try to warm up the + cache until succeed or NN leaves active state. + + + + + dfs.namenode.edekcacheloader.initial.delay.ms + 3000 + When KeyProvider is configured, the time delayed until the first + attempt to warm up edek cache on NN start up / become active. + + + dfs.namenode.inotify.max.events.per.rpc 1000 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java index 6c2ce6a2911..c8d98ee130c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java @@ -117,10 +117,10 @@ import javax.xml.parsers.SAXParserFactory; public class TestEncryptionZones { - private Configuration conf; + protected Configuration conf; private FileSystemTestHelper fsHelper; - private MiniDFSCluster cluster; + protected MiniDFSCluster cluster; protected HdfsAdmin dfsAdmin; protected DistributedFileSystem fs; private File testRootDir; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java index 0040d7561be..59c8dd52f2b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs; import static org.junit.Assert.assertTrue; +import com.google.common.base.Supplier; import org.apache.hadoop.crypto.key.kms.KMSClientProvider; import org.apache.hadoop.crypto.key.kms.server.MiniKMS; import org.apache.hadoop.security.Credentials; @@ -26,10 +27,12 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.test.GenericTestUtils; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.internal.util.reflection.Whitebox; import java.io.File; import java.util.Arrays; @@ -71,8 +74,10 @@ public class TestEncryptionZonesWithKMS extends TestEncryptionZones { final Path zonePath = new Path("/TestEncryptionZone"); fsWrapper.mkdir(zonePath, FsPermission.getDirDefault(), false); dfsAdmin.createEncryptionZone(zonePath, TEST_KEY); - assertTrue(((KMSClientProvider)fs.getClient().getKeyProvider()). - getEncKeyQueueSize(TEST_KEY) > 0); + @SuppressWarnings("unchecked") + KMSClientProvider kcp = (KMSClientProvider) Whitebox + .getInternalState(cluster.getNamesystem().getProvider(), "extension"); + assertTrue(kcp.getEncKeyQueueSize(TEST_KEY) > 0); } @Test(timeout = 120000) @@ -92,4 +97,31 @@ public class TestEncryptionZonesWithKMS extends TestEncryptionZones { Assert.assertEquals(0, tokens.length); Assert.assertEquals(2, creds.numberOfTokens()); } + + @Test(timeout = 120000) + public void testWarmupEDEKCacheOnStartup() throws Exception { + final Path zonePath = new Path("/TestEncryptionZone"); + fsWrapper.mkdir(zonePath, FsPermission.getDirDefault(), false); + dfsAdmin.createEncryptionZone(zonePath, TEST_KEY); + + @SuppressWarnings("unchecked") + KMSClientProvider spy = (KMSClientProvider) Whitebox + .getInternalState(cluster.getNamesystem().getProvider(), "extension"); + assertTrue("key queue is empty after creating encryption zone", + spy.getEncKeyQueueSize(TEST_KEY) > 0); + + conf.setInt( + DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INITIAL_DELAY_MS_KEY, 0); + cluster.restartNameNode(true); + + GenericTestUtils.waitFor(new Supplier() { + @Override + public Boolean get() { + final KMSClientProvider kspy = (KMSClientProvider) Whitebox + .getInternalState(cluster.getNamesystem().getProvider(), + "extension"); + return kspy.getEncKeyQueueSize(TEST_KEY) > 0; + } + }, 1000, 60000); + } } From 680716f31e120f4d3ee70b095e4db46c05b891d9 Mon Sep 17 00:00:00 2001 From: Colin Patrick Mccabe Date: Mon, 21 Mar 2016 11:40:37 -0700 Subject: [PATCH 08/43] HDFS-9951. Use string constants for XML tags in OfflineImageReconstructor (Lin Yiqun via cmccabe) --- .../OfflineImageReconstructor.java | 312 +++++++----- .../offlineImageViewer/PBImageXmlWriter.java | 463 ++++++++++++------ 2 files changed, 496 insertions(+), 279 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java index 73f4a22529d..5c71399ac4c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java @@ -22,6 +22,7 @@ import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_ import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAMESPACE_OFFSET; import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAMESPACE_EXT_OFFSET; import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAMESPACE_EXT_MASK; +import static org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter.*; import java.io.BufferedOutputStream; import java.io.File; @@ -434,36 +435,38 @@ class OfflineImageReconstructor { Node node = new Node(); loadNodeChildren(node, "NameSection fields"); NameSystemSection.Builder b = NameSystemSection.newBuilder(); - Integer namespaceId = node.removeChildInt("namespaceId"); + Integer namespaceId = node.removeChildInt(NAME_SECTION_NAMESPACE_ID); if (namespaceId == null) { throw new IOException(" is missing "); } b.setNamespaceId(namespaceId); - Long lval = node.removeChildLong("genstampV1"); + Long lval = node.removeChildLong(NAME_SECTION_GENSTAMPV1); if (lval != null) { b.setGenstampV1(lval); } - lval = node.removeChildLong("genstampV2"); + lval = node.removeChildLong(NAME_SECTION_GENSTAMPV2); if (lval != null) { b.setGenstampV2(lval); } - lval = node.removeChildLong("genstampV1Limit"); + lval = node.removeChildLong(NAME_SECTION_GENSTAMPV1_LIMIT); if (lval != null) { b.setGenstampV1Limit(lval); } - lval = node.removeChildLong("lastAllocatedBlockId"); + lval = node.removeChildLong(NAME_SECTION_LAST_ALLOCATED_BLOCK_ID); if (lval != null) { b.setLastAllocatedBlockId(lval); } - lval = node.removeChildLong("txid"); + lval = node.removeChildLong(NAME_SECTION_TXID); if (lval != null) { b.setTransactionId(lval); } - lval = node.removeChildLong("rollingUpgradeStartTime"); + lval = node.removeChildLong( + NAME_SECTION_ROLLING_UPGRADE_START_TIME); if (lval != null) { b.setRollingUpgradeStartTime(lval); } - lval = node.removeChildLong("lastAllocatedStripedBlockId"); + lval = node.removeChildLong( + NAME_SECTION_LAST_ALLOCATED_STRIPED_BLOCK_ID); if (lval != null) { b.setLastAllocatedStripedBlockId(lval); } @@ -486,11 +489,12 @@ class OfflineImageReconstructor { Node headerNode = new Node(); loadNodeChildren(headerNode, "INodeSection fields", "inode"); INodeSection.Builder b = INodeSection.newBuilder(); - Long lval = headerNode.removeChildLong("lastInodeId"); + Long lval = headerNode.removeChildLong(INODE_SECTION_LAST_INODE_ID); if (lval != null) { b.setLastInodeId(lval); } - Integer expectedNumINodes = headerNode.removeChildInt("numInodes"); + Integer expectedNumINodes = + headerNode.removeChildInt(INODE_SECTION_NUM_INODES); if (expectedNumINodes == null) { throw new IOException("Failed to find in INodeSection."); } @@ -501,7 +505,7 @@ class OfflineImageReconstructor { int actualNumINodes = 0; while (actualNumINodes < expectedNumINodes) { try { - expectTag("inode", false); + expectTag(INODE_SECTION_INODE, false); } catch (IOException e) { throw new IOException("Only found " + actualNumINodes + " entries out of " + expectedNumINodes, e); @@ -512,24 +516,24 @@ class OfflineImageReconstructor { INodeSection.INode.Builder inodeBld = processINodeXml(inode); inodeBld.build().writeDelimitedTo(out); } - expectTagEnd("INodeSection"); + expectTagEnd(INODE_SECTION_NAME); recordSectionLength(SectionName.INODE.name()); } } private INodeSection.INode.Builder processINodeXml(Node node) throws IOException { - String type = node.removeChildStr("type"); + String type = node.removeChildStr(INODE_SECTION_TYPE); if (type == null) { throw new IOException("INode XML found with no tag."); } INodeSection.INode.Builder inodeBld = INodeSection.INode.newBuilder(); - Long id = node.removeChildLong("id"); + Long id = node.removeChildLong(SECTION_ID); if (id == null) { throw new IOException(" found without "); } inodeBld.setId(id); - String name = node.removeChildStr("name"); + String name = node.removeChildStr(SECTION_NAME); if (name != null) { inodeBld.setName(ByteString.copyFrom(name, "UTF8")); } @@ -555,46 +559,46 @@ class OfflineImageReconstructor { throws IOException { inodeBld.setType(INodeSection.INode.Type.FILE); INodeSection.INodeFile.Builder bld = INodeSection.INodeFile.newBuilder(); - Integer ival = node.removeChildInt("replication"); + Integer ival = node.removeChildInt(SECTION_REPLICATION); if (ival != null) { bld.setReplication(ival); } - Long lval = node.removeChildLong("mtime"); + Long lval = node.removeChildLong(INODE_SECTION_MTIME); if (lval != null) { bld.setModificationTime(lval); } - lval = node.removeChildLong("atime"); + lval = node.removeChildLong(INODE_SECTION_ATIME); if (lval != null) { bld.setAccessTime(lval); } - lval = node.removeChildLong("preferredBlockSize"); + lval = node.removeChildLong(INODE_SECTION_PREFERRED_BLOCK_SIZE); if (lval != null) { bld.setPreferredBlockSize(lval); } - String perm = node.removeChildStr("permission"); + String perm = node.removeChildStr(INODE_SECTION_PERMISSION); if (perm != null) { bld.setPermission(permissionXmlToU64(perm)); } - Node blocks = node.removeChild("blocks"); + Node blocks = node.removeChild(INODE_SECTION_BLOCKS); if (blocks != null) { while (true) { - Node block = blocks.removeChild("block"); + Node block = blocks.removeChild(INODE_SECTION_BLOCK); if (block == null) { break; } HdfsProtos.BlockProto.Builder blockBld = HdfsProtos.BlockProto.newBuilder(); - Long id = block.removeChildLong("id"); + Long id = block.removeChildLong(SECTION_ID); if (id == null) { throw new IOException(" found without "); } blockBld.setBlockId(id); - Long genstamp = block.removeChildLong("genstamp"); + Long genstamp = block.removeChildLong(INODE_SECTION_GEMSTAMP); if (genstamp == null) { throw new IOException(" found without "); } blockBld.setGenStamp(genstamp); - Long numBytes = block.removeChildLong("numBytes"); + Long numBytes = block.removeChildLong(INODE_SECTION_NUM_BYTES); if (numBytes == null) { throw new IOException(" found without "); } @@ -602,19 +606,21 @@ class OfflineImageReconstructor { bld.addBlocks(blockBld); } } - Node fileUnderConstruction = node.removeChild("file-under-construction"); + Node fileUnderConstruction = + node.removeChild(INODE_SECTION_FILE_UNDER_CONSTRUCTION); if (fileUnderConstruction != null) { INodeSection.FileUnderConstructionFeature.Builder fb = INodeSection.FileUnderConstructionFeature.newBuilder(); String clientName = - fileUnderConstruction.removeChildStr("clientName"); + fileUnderConstruction.removeChildStr(INODE_SECTION_CLIENT_NAME); if (clientName == null) { throw new IOException(" found without " + ""); } fb.setClientName(clientName); String clientMachine = - fileUnderConstruction.removeChildStr("clientMachine"); + fileUnderConstruction + .removeChildStr(INODE_SECTION_CLIENT_MACHINE); if (clientMachine == null) { throw new IOException(" found without " + ""); @@ -622,19 +628,19 @@ class OfflineImageReconstructor { fb.setClientMachine(clientMachine); bld.setFileUC(fb); } - Node acls = node.removeChild("acls"); + Node acls = node.removeChild(INODE_SECTION_ACLS); if (acls != null) { bld.setAcl(aclXmlToProto(acls)); } - Node xattrs = node.removeChild("xattrs"); + Node xattrs = node.removeChild(INODE_SECTION_XATTRS); if (xattrs != null) { bld.setXAttrs(xattrsXmlToProto(xattrs)); } - ival = node.removeChildInt("storagePolicyId"); + ival = node.removeChildInt(INODE_SECTION_STORAGE_POLICY_ID); if (ival != null) { bld.setStoragePolicyID(ival); } - Boolean bval = node.removeChildBool("isStriped"); + Boolean bval = node.removeChildBool(INODE_SECTION_IS_STRIPED); bld.setIsStriped(bval); inodeBld.setFile(bld); // Will check remaining keys and serialize in processINodeXml @@ -645,40 +651,40 @@ class OfflineImageReconstructor { inodeBld.setType(INodeSection.INode.Type.DIRECTORY); INodeSection.INodeDirectory.Builder bld = INodeSection.INodeDirectory.newBuilder(); - Long lval = node.removeChildLong("mtime"); + Long lval = node.removeChildLong(INODE_SECTION_MTIME); if (lval != null) { bld.setModificationTime(lval); } - lval = node.removeChildLong("nsquota"); + lval = node.removeChildLong(INODE_SECTION_NS_QUOTA); if (lval != null) { bld.setNsQuota(lval); } - lval = node.removeChildLong("dsquota"); + lval = node.removeChildLong(INODE_SECTION_DS_QUOTA); if (lval != null) { bld.setDsQuota(lval); } - String perm = node.removeChildStr("permission"); + String perm = node.removeChildStr(INODE_SECTION_PERMISSION); if (perm != null) { bld.setPermission(permissionXmlToU64(perm)); } - Node acls = node.removeChild("acls"); + Node acls = node.removeChild(INODE_SECTION_ACLS); if (acls != null) { bld.setAcl(aclXmlToProto(acls)); } - Node xattrs = node.removeChild("xattrs"); + Node xattrs = node.removeChild(INODE_SECTION_XATTRS); if (xattrs != null) { bld.setXAttrs(xattrsXmlToProto(xattrs)); } INodeSection.QuotaByStorageTypeFeatureProto.Builder qf = INodeSection.QuotaByStorageTypeFeatureProto.newBuilder(); while (true) { - Node typeQuota = node.removeChild("typeQuota"); + Node typeQuota = node.removeChild(INODE_SECTION_TYPE_QUOTA); if (typeQuota == null) { break; } INodeSection.QuotaByStorageTypeEntryProto.Builder qbld = INodeSection.QuotaByStorageTypeEntryProto.newBuilder(); - String type = typeQuota.removeChildStr("type"); + String type = typeQuota.removeChildStr(INODE_SECTION_TYPE); if (type == null) { throw new IOException(" was missing "); } @@ -688,7 +694,7 @@ class OfflineImageReconstructor { throw new IOException(" had unknown " + type); } qbld.setStorageType(storageType); - Long quota = typeQuota.removeChildLong("quota"); + Long quota = typeQuota.removeChildLong(INODE_SECTION_QUOTA); if (quota == null) { throw new IOException(" was missing "); } @@ -705,19 +711,19 @@ class OfflineImageReconstructor { inodeBld.setType(INodeSection.INode.Type.SYMLINK); INodeSection.INodeSymlink.Builder bld = INodeSection.INodeSymlink.newBuilder(); - String perm = node.removeChildStr("permission"); + String perm = node.removeChildStr(INODE_SECTION_PERMISSION); if (perm != null) { bld.setPermission(permissionXmlToU64(perm)); } - String target = node.removeChildStr("target"); + String target = node.removeChildStr(INODE_SECTION_TARGET); if (target != null) { bld.setTarget(ByteString.copyFrom(target, "UTF8")); } - Long lval = node.removeChildLong("mtime"); + Long lval = node.removeChildLong(INODE_SECTION_MTIME); if (lval != null) { bld.setModificationTime(lval); } - lval = node.removeChildLong("atime"); + lval = node.removeChildLong(INODE_SECTION_ATIME); if (lval != null) { bld.setAccessTime(lval); } @@ -736,23 +742,23 @@ class OfflineImageReconstructor { INodeSection.XAttrFeatureProto.Builder bld = INodeSection.XAttrFeatureProto.newBuilder(); while (true) { - Node xattr = xattrs.removeChild("xattr"); + Node xattr = xattrs.removeChild(INODE_SECTION_XATTR); if (xattr == null) { break; } INodeSection.XAttrCompactProto.Builder b = INodeSection.XAttrCompactProto.newBuilder(); - String ns = xattr.removeChildStr("ns"); + String ns = xattr.removeChildStr(INODE_SECTION_NS); if (ns == null) { throw new IOException(" had no entry."); } int nsIdx = XAttrProtos.XAttrProto. XAttrNamespaceProto.valueOf(ns).ordinal(); - String name = xattr.removeChildStr("name"); - String valStr = xattr.removeChildStr("val"); + String name = xattr.removeChildStr(SECTION_NAME); + String valStr = xattr.removeChildStr(INODE_SECTION_VAL); byte[] val = null; if (valStr == null) { - String valHex = xattr.removeChildStr("valHex"); + String valHex = xattr.removeChildStr(INODE_SECTION_VAL_HEX); if (valHex == null) { throw new IOException(" had no or entry."); } @@ -787,24 +793,28 @@ class OfflineImageReconstructor { loadNodeChildren(secretHeader, "SecretManager fields", "delegationKey", "token"); SecretManagerSection.Builder b = SecretManagerSection.newBuilder(); - Integer currentId = secretHeader.removeChildInt("currentId"); + Integer currentId = + secretHeader.removeChildInt(SECRET_MANAGER_SECTION_CURRENT_ID); if (currentId == null) { throw new IOException("SecretManager section had no "); } b.setCurrentId(currentId); - Integer tokenSequenceNumber = secretHeader.removeChildInt("tokenSequenceNumber"); + Integer tokenSequenceNumber = secretHeader.removeChildInt( + SECRET_MANAGER_SECTION_TOKEN_SEQUENCE_NUMBER); if (tokenSequenceNumber == null) { throw new IOException("SecretManager section had no " + ""); } b.setTokenSequenceNumber(tokenSequenceNumber); - Integer expectedNumKeys = secretHeader.removeChildInt("numDelegationKeys"); + Integer expectedNumKeys = secretHeader.removeChildInt( + SECRET_MANAGER_SECTION_NUM_DELEGATION_KEYS); if (expectedNumKeys == null) { throw new IOException("SecretManager section had no " + ""); } b.setNumKeys(expectedNumKeys); - Integer expectedNumTokens = secretHeader.removeChildInt("numTokens"); + Integer expectedNumTokens = + secretHeader.removeChildInt(SECRET_MANAGER_SECTION_NUM_TOKENS); if (expectedNumTokens == null) { throw new IOException("SecretManager section had no " + ""); @@ -815,7 +825,7 @@ class OfflineImageReconstructor { for (int actualNumKeys = 0; actualNumKeys < expectedNumKeys; actualNumKeys++) { try { - expectTag("delegationKey", false); + expectTag(SECRET_MANAGER_SECTION_DELEGATION_KEY, false); } catch (IOException e) { throw new IOException("Only read " + actualNumKeys + " delegation keys out of " + expectedNumKeys, e); @@ -824,32 +834,32 @@ class OfflineImageReconstructor { SecretManagerSection.DelegationKey.newBuilder(); Node dkey = new Node(); loadNodeChildren(dkey, "Delegation key fields"); - Integer id = dkey.removeChildInt("id"); + Integer id = dkey.removeChildInt(SECTION_ID); if (id == null) { throw new IOException("Delegation key stanza " + "lacked an field."); } dbld.setId(id); - String expiry = dkey.removeChildStr("expiry"); + String expiry = dkey.removeChildStr(SECRET_MANAGER_SECTION_EXPIRY); if (expiry == null) { throw new IOException("Delegation key stanza " + "lacked an field."); } dbld.setExpiryDate(dateStrToLong(expiry)); - String keyHex = dkey.removeChildStr("key"); + String keyHex = dkey.removeChildStr(SECRET_MANAGER_SECTION_KEY); if (keyHex == null) { throw new IOException("Delegation key stanza " + "lacked a field."); } byte[] key = new HexBinaryAdapter().unmarshal(keyHex); - dkey.verifyNoRemainingKeys("delegationKey"); + dkey.verifyNoRemainingKeys(SECRET_MANAGER_SECTION_DELEGATION_KEY); dbld.setKey(ByteString.copyFrom(key)); dbld.build().writeDelimitedTo(out); } for (int actualNumTokens = 0; actualNumTokens < expectedNumTokens; actualNumTokens++) { try { - expectTag("token", false); + expectTag(SECRET_MANAGER_SECTION_TOKEN, false); } catch (IOException e) { throw new IOException("Only read " + actualNumTokens + " tokens out of " + expectedNumTokens, e); @@ -858,46 +868,54 @@ class OfflineImageReconstructor { SecretManagerSection.PersistToken.newBuilder(); Node token = new Node(); loadNodeChildren(token, "PersistToken key fields"); - Integer version = token.removeChildInt("version"); + Integer version = + token.removeChildInt(SECRET_MANAGER_SECTION_VERSION); if (version != null) { tbld.setVersion(version); } - String owner = token.removeChildStr("owner"); + String owner = token.removeChildStr(SECRET_MANAGER_SECTION_OWNER); if (owner != null) { tbld.setOwner(owner); } - String renewer = token.removeChildStr("renewer"); + String renewer = + token.removeChildStr(SECRET_MANAGER_SECTION_RENEWER); if (renewer != null) { tbld.setRenewer(renewer); } - String realUser = token.removeChildStr("realUser"); + String realUser = + token.removeChildStr(SECRET_MANAGER_SECTION_REAL_USER); if (realUser != null) { tbld.setRealUser(realUser); } - String issueDateStr = token.removeChildStr("issueDate"); + String issueDateStr = + token.removeChildStr(SECRET_MANAGER_SECTION_ISSUE_DATE); if (issueDateStr != null) { tbld.setIssueDate(dateStrToLong(issueDateStr)); } - String maxDateStr = token.removeChildStr("maxDate"); + String maxDateStr = + token.removeChildStr(SECRET_MANAGER_SECTION_MAX_DATE); if (maxDateStr != null) { tbld.setMaxDate(dateStrToLong(maxDateStr)); } - Integer seqNo = token.removeChildInt("sequenceNumber"); + Integer seqNo = + token.removeChildInt(SECRET_MANAGER_SECTION_SEQUENCE_NUMBER); if (seqNo != null) { tbld.setSequenceNumber(seqNo); } - Integer masterKeyId = token.removeChildInt("masterKeyId"); + Integer masterKeyId = + token.removeChildInt(SECRET_MANAGER_SECTION_MASTER_KEY_ID); if (masterKeyId != null) { tbld.setMasterKeyId(masterKeyId); } - String expiryDateStr = token.removeChildStr("expiryDate"); + String expiryDateStr = + token.removeChildStr(SECRET_MANAGER_SECTION_EXPIRY_DATE); if (expiryDateStr != null) { tbld.setExpiryDate(dateStrToLong(expiryDateStr)); } token.verifyNoRemainingKeys("token"); tbld.build().writeDelimitedTo(out); } - expectTagEnd("SecretManagerSection"); + expectTagEnd(SECRET_MANAGER_SECTION_NAME); recordSectionLength(SectionName.SECRET_MANAGER.name()); } @@ -919,17 +937,20 @@ class OfflineImageReconstructor { Node node = new Node(); loadNodeChildren(node, "CacheManager fields", "pool", "directive"); CacheManagerSection.Builder b = CacheManagerSection.newBuilder(); - Long nextDirectiveId = node.removeChildLong("nextDirectiveId"); + Long nextDirectiveId = + node.removeChildLong(CACHE_MANAGER_SECTION_NEXT_DIRECTIVE_ID); if (nextDirectiveId == null) { throw new IOException("CacheManager section had no "); } b.setNextDirectiveId(nextDirectiveId); - Integer expectedNumPools = node.removeChildInt("numPools"); + Integer expectedNumPools = + node.removeChildInt(CACHE_MANAGER_SECTION_NUM_POOLS); if (expectedNumPools == null) { throw new IOException("CacheManager section had no "); } b.setNumPools(expectedNumPools); - Integer expectedNumDirectives = node.removeChildInt("numDirectives"); + Integer expectedNumDirectives = + node.removeChildInt(CACHE_MANAGER_SECTION_NUM_DIRECTIVES); if (expectedNumDirectives == null) { throw new IOException("CacheManager section had no "); } @@ -938,7 +959,7 @@ class OfflineImageReconstructor { long actualNumPools = 0; while (actualNumPools < expectedNumPools) { try { - expectTag("pool", false); + expectTag(CACHE_MANAGER_SECTION_POOL, false); } catch (IOException e) { throw new IOException("Only read " + actualNumPools + " cache pools out of " + expectedNumPools, e); @@ -951,7 +972,7 @@ class OfflineImageReconstructor { long actualNumDirectives = 0; while (actualNumDirectives < expectedNumDirectives) { try { - expectTag("directive", false); + expectTag(CACHE_MANAGER_SECTION_DIRECTIVE, false); } catch (IOException e) { throw new IOException("Only read " + actualNumDirectives + " cache pools out of " + expectedNumDirectives, e); @@ -961,38 +982,42 @@ class OfflineImageReconstructor { loadNodeChildren(pool, "directive fields", ""); processDirectiveXml(node); } - expectTagEnd("CacheManagerSection"); + expectTagEnd(CACHE_MANAGER_SECTION_NAME); recordSectionLength(SectionName.CACHE_MANAGER.name()); } private void processPoolXml(Node pool) throws IOException { CachePoolInfoProto.Builder bld = CachePoolInfoProto.newBuilder(); - String poolName = pool.removeChildStr("poolName"); + String poolName = + pool.removeChildStr(CACHE_MANAGER_SECTION_POOL_NAME); if (poolName == null) { throw new IOException(" found without "); } bld.setPoolName(poolName); - String ownerName = pool.removeChildStr("ownerName"); + String ownerName = + pool.removeChildStr(CACHE_MANAGER_SECTION_OWNER_NAME); if (ownerName == null) { throw new IOException(" found without "); } bld.setOwnerName(ownerName); - String groupName = pool.removeChildStr("groupName"); + String groupName = + pool.removeChildStr(CACHE_MANAGER_SECTION_GROUP_NAME); if (groupName == null) { throw new IOException(" found without "); } bld.setGroupName(groupName); - Integer mode = pool.removeChildInt("mode"); + Integer mode = pool.removeChildInt(CACHE_MANAGER_SECTION_MODE); if (mode == null) { throw new IOException(" found without "); } bld.setMode(mode); - Long limit = pool.removeChildLong("limit"); + Long limit = pool.removeChildLong(CACHE_MANAGER_SECTION_LIMIT); if (limit == null) { throw new IOException(" found without "); } bld.setLimit(limit); - Long maxRelativeExpiry = pool.removeChildLong("maxRelativeExpiry"); + Long maxRelativeExpiry = + pool.removeChildLong(CACHE_MANAGER_SECTION_MAX_RELATIVE_EXPIRY); if (maxRelativeExpiry == null) { throw new IOException(" found without "); } @@ -1004,37 +1029,39 @@ class OfflineImageReconstructor { private void processDirectiveXml(Node directive) throws IOException { CacheDirectiveInfoProto.Builder bld = CacheDirectiveInfoProto.newBuilder(); - Long id = directive.removeChildLong("id"); + Long id = directive.removeChildLong(SECTION_ID); if (id == null) { throw new IOException(" found without "); } bld.setId(id); - String path = directive.removeChildStr("path"); + String path = directive.removeChildStr(SECTION_PATH); if (path == null) { throw new IOException(" found without "); } bld.setPath(path); - Integer replication = directive.removeChildInt("replication"); + Integer replication = directive.removeChildInt(SECTION_REPLICATION); if (replication == null) { throw new IOException(" found without "); } bld.setReplication(replication); - String pool = directive.removeChildStr("pool"); + String pool = directive.removeChildStr(CACHE_MANAGER_SECTION_POOL); if (path == null) { throw new IOException(" found without "); } bld.setPool(pool); - Node expiration = directive.removeChild("expiration"); + Node expiration = + directive.removeChild(CACHE_MANAGER_SECTION_EXPIRATION); if (expiration != null) { CacheDirectiveInfoExpirationProto.Builder ebld = CacheDirectiveInfoExpirationProto.newBuilder(); - Long millis = expiration.removeChildLong("millis"); + Long millis = + expiration.removeChildLong(CACHE_MANAGER_SECTION_MILLIS); if (millis == null) { throw new IOException("cache directive found " + "without "); } ebld.setMillis(millis); - if (expiration.removeChildBool("relative")) { + if (expiration.removeChildBool(CACHE_MANAGER_SECTION_RELATIVE)) { ebld.setIsRelative(true); } else { ebld.setIsRelative(false); @@ -1054,7 +1081,7 @@ class OfflineImageReconstructor { // There is no header for this section. // We process the repeated elements. while (true) { - XMLEvent ev = expectTag("ref", true); + XMLEvent ev = expectTag(INODE_REFERENCE_SECTION_REF, true); if (ev.isEndElement()) { break; } @@ -1062,7 +1089,8 @@ class OfflineImageReconstructor { FsImageProto.INodeReferenceSection.INodeReference.Builder bld = FsImageProto.INodeReferenceSection.INodeReference.newBuilder(); loadNodeChildren(inodeRef, "INodeReference"); - Long referredId = inodeRef.removeChildLong("referredId"); + Long referredId = + inodeRef.removeChildLong(INODE_REFERENCE_SECTION_REFERRED_ID); if (referredId != null) { bld.setReferredId(referredId); } @@ -1070,11 +1098,13 @@ class OfflineImageReconstructor { if (name != null) { bld.setName(ByteString.copyFrom(name, "UTF8")); } - Integer dstSnapshotId = inodeRef.removeChildInt("dstSnapshotId"); + Integer dstSnapshotId = inodeRef.removeChildInt( + INODE_REFERENCE_SECTION_DST_SNAPSHOT_ID); if (dstSnapshotId != null) { bld.setDstSnapshotId(dstSnapshotId); } - Integer lastSnapshotId = inodeRef.removeChildInt("lastSnapshotId"); + Integer lastSnapshotId = inodeRef.removeChildInt( + INODE_REFERENCE_SECTION_LAST_SNAPSHOT_ID); if (lastSnapshotId != null) { bld.setLastSnapshotId(lastSnapshotId); } @@ -1093,7 +1123,7 @@ class OfflineImageReconstructor { // No header for this section // Process the repeated elements. while (true) { - XMLEvent ev = expectTag("directory", true); + XMLEvent ev = expectTag(INODE_DIRECTORY_SECTION_DIRECTORY, true); if (ev.isEndElement()) { break; } @@ -1101,19 +1131,22 @@ class OfflineImageReconstructor { FsImageProto.INodeDirectorySection.DirEntry.Builder bld = FsImageProto.INodeDirectorySection.DirEntry.newBuilder(); loadNodeChildren(directory, "directory"); - Long parent = directory.removeChildLong("parent"); + Long parent = directory.removeChildLong( + INODE_DIRECTORY_SECTION_PARENT); if (parent != null) { bld.setParent(parent); } while (true) { - Node child = directory.removeChild("child"); + Node child = directory.removeChild( + INODE_DIRECTORY_SECTION_CHILD); if (child == null) { break; } bld.addChildren(Long.parseLong(child.getVal())); } while (true) { - Node refChild = directory.removeChild("refChild"); + Node refChild = directory.removeChild( + INODE_DIRECTORY_SECTION_REF_CHILD); if (refChild == null) { break; } @@ -1135,7 +1168,7 @@ class OfflineImageReconstructor { // No header for this section type. // Process the repeated files under construction elements. while (true) { - XMLEvent ev = expectTag("inode", true); + XMLEvent ev = expectTag(INODE_SECTION_INODE, true); if (ev.isEndElement()) { break; } @@ -1143,11 +1176,12 @@ class OfflineImageReconstructor { loadNodeChildren(fileUnderConstruction, "file under construction"); FileUnderConstructionEntry.Builder bld = FileUnderConstructionEntry.newBuilder(); - Long id = fileUnderConstruction.removeChildLong("id"); + Long id = fileUnderConstruction.removeChildLong(SECTION_ID); if (id != null) { bld.setInodeId(id); } - String fullpath = fileUnderConstruction.removeChildStr("path"); + String fullpath = + fileUnderConstruction.removeChildStr(SECTION_PATH); if (fullpath != null) { bld.setFullPath(fullpath); } @@ -1167,24 +1201,26 @@ class OfflineImageReconstructor { FsImageProto.SnapshotSection.newBuilder(); Node header = new Node(); loadNodeChildren(header, "SnapshotSection fields", "snapshot"); - Integer snapshotCounter = header.removeChildInt("snapshotCounter"); + Integer snapshotCounter = header.removeChildInt( + SNAPSHOT_SECTION_SNAPSHOT_COUNTER); if (snapshotCounter == null) { throw new IOException("No entry found in " + "SnapshotSection header"); } bld.setSnapshotCounter(snapshotCounter); - Integer expectedNumSnapshots = header.removeChildInt("numSnapshots"); + Integer expectedNumSnapshots = header.removeChildInt( + SNAPSHOT_SECTION_NUM_SNAPSHOTS); if (expectedNumSnapshots == null) { throw new IOException("No entry found in " + "SnapshotSection header"); } bld.setNumSnapshots(expectedNumSnapshots); while (true) { - Node sd = header.removeChild("snapshottableDir"); + Node sd = header.removeChild(SNAPSHOT_SECTION_SNAPSHOT_TABLE_DIR); if (sd == null) { break; } - Long dir = sd.removeChildLong("dir"); + Long dir = sd.removeChildLong(SNAPSHOT_SECTION_DIR); sd.verifyNoRemainingKeys(""); bld.addSnapshottableDir(dir); } @@ -1193,7 +1229,7 @@ class OfflineImageReconstructor { int actualNumSnapshots = 0; while (actualNumSnapshots < expectedNumSnapshots) { try { - expectTag("snapshot", false); + expectTag(SNAPSHOT_SECTION_SNAPSHOT, false); } catch (IOException e) { throw new IOException("Only read " + actualNumSnapshots + " entries out of " + expectedNumSnapshots, e); @@ -1203,17 +1239,17 @@ class OfflineImageReconstructor { loadNodeChildren(snapshot, "snapshot fields"); FsImageProto.SnapshotSection.Snapshot.Builder s = FsImageProto.SnapshotSection.Snapshot.newBuilder(); - Integer snapshotId = snapshot.removeChildInt("id"); + Integer snapshotId = snapshot.removeChildInt(SECTION_ID); if (snapshotId == null) { throw new IOException(" section was missing "); } s.setSnapshotId(snapshotId); - Node snapshotRoot = snapshot.removeChild("root"); + Node snapshotRoot = snapshot.removeChild(SNAPSHOT_SECTION_ROOT); INodeSection.INode.Builder inodeBld = processINodeXml(snapshotRoot); s.setRoot(inodeBld); s.build().writeDelimitedTo(out); } - expectTagEnd("SnapshotSection"); + expectTagEnd(SNAPSHOT_SECTION_NAME); recordSectionLength(SectionName.SNAPSHOT.name()); } } @@ -1229,15 +1265,15 @@ class OfflineImageReconstructor { XMLEvent ev = expectTag("[diff start tag]", true); if (ev.isEndElement()) { String name = ev.asEndElement().getName().getLocalPart(); - if (name.equals("SnapshotDiffSection")) { + if (name.equals(SNAPSHOT_DIFF_SECTION_NAME)) { break; } throw new IOException("Got unexpected end tag for " + name); } String tagName = ev.asStartElement().getName().getLocalPart(); - if (tagName.equals("dirDiffEntry")) { + if (tagName.equals(SNAPSHOT_DIFF_SECTION_DIR_DIFF_ENTRY)) { processDirDiffEntry(); - } else if (tagName.equals("fileDiffEntry")) { + } else if (tagName.equals(SNAPSHOT_DIFF_SECTION_FILE_DIFF_ENTRY)) { processFileDiffEntry(); } else { throw new IOException("SnapshotDiffSection contained unexpected " + @@ -1253,12 +1289,14 @@ class OfflineImageReconstructor { headerBld.setType(DiffEntry.Type.DIRECTORYDIFF); Node dirDiffHeader = new Node(); loadNodeChildren(dirDiffHeader, "dirDiffEntry fields", "dirDiff"); - Long inodeId = dirDiffHeader.removeChildLong("inodeId"); + Long inodeId = dirDiffHeader.removeChildLong( + SNAPSHOT_DIFF_SECTION_INODE_ID); if (inodeId == null) { throw new IOException(" contained no entry."); } headerBld.setInodeId(inodeId); - Integer expectedDiffs = dirDiffHeader.removeChildInt("count"); + Integer expectedDiffs = dirDiffHeader.removeChildInt( + SNAPSHOT_DIFF_SECTION_COUNT); if (expectedDiffs == null) { throw new IOException(" contained no entry."); } @@ -1267,7 +1305,7 @@ class OfflineImageReconstructor { headerBld.build().writeDelimitedTo(out); for (int actualDiffs = 0; actualDiffs < expectedDiffs; actualDiffs++) { try { - expectTag("dirDiff", false); + expectTag(SNAPSHOT_DIFF_SECTION_DIR_DIFF, false); } catch (IOException e) { throw new IOException("Only read " + (actualDiffs + 1) + " diffs out of " + expectedDiffs, e); @@ -1276,38 +1314,43 @@ class OfflineImageReconstructor { loadNodeChildren(dirDiff, "dirDiff fields"); FsImageProto.SnapshotDiffSection.DirectoryDiff.Builder bld = FsImageProto.SnapshotDiffSection.DirectoryDiff.newBuilder(); - Integer snapshotId = dirDiff.removeChildInt("snapshotId"); + Integer snapshotId = dirDiff.removeChildInt( + SNAPSHOT_DIFF_SECTION_SNAPSHOT_ID); if (snapshotId != null) { bld.setSnapshotId(snapshotId); } - Integer childrenSize = dirDiff.removeChildInt("childrenSize"); + Integer childrenSize = dirDiff.removeChildInt( + SNAPSHOT_DIFF_SECTION_CHILDREN_SIZE); if (childrenSize == null) { throw new IOException("Expected to find in " + " section."); } - bld.setIsSnapshotRoot(dirDiff.removeChildBool("isSnapshotRoot")); + bld.setIsSnapshotRoot(dirDiff.removeChildBool( + SNAPSHOT_DIFF_SECTION_IS_SNAPSHOT_ROOT)); bld.setChildrenSize(childrenSize); - String name = dirDiff.removeChildStr("name"); + String name = dirDiff.removeChildStr(SECTION_NAME); if (name != null) { bld.setName(ByteString.copyFrom(name, "UTF8")); } // TODO: add missing snapshotCopy field to XML - Integer expectedCreatedListSize = - dirDiff.removeChildInt("createdListSize"); + Integer expectedCreatedListSize = dirDiff.removeChildInt( + SNAPSHOT_DIFF_SECTION_CREATED_LIST_SIZE); if (expectedCreatedListSize == null) { throw new IOException("Expected to find in " + " section."); } bld.setCreatedListSize(expectedCreatedListSize); while (true) { - Node deleted = dirDiff.removeChild("deletedInode"); + Node deleted = dirDiff.removeChild( + SNAPSHOT_DIFF_SECTION_DELETED_INODE); if (deleted == null){ break; } bld.addDeletedINode(Long.parseLong(deleted.getVal())); } while (true) { - Node deleted = dirDiff.removeChild("deletedInoderef"); + Node deleted = dirDiff.removeChild( + SNAPSHOT_DIFF_SECTION_DELETED_INODE_REF); if (deleted == null){ break; } @@ -1317,11 +1360,12 @@ class OfflineImageReconstructor { // After the DirectoryDiff header comes a list of CreatedListEntry PBs. int actualCreatedListSize = 0; while (true) { - Node created = dirDiff.removeChild("created"); + Node created = dirDiff.removeChild( + SNAPSHOT_DIFF_SECTION_CREATED); if (created == null){ break; } - String cleName = created.removeChildStr("name"); + String cleName = created.removeChildStr(SECTION_NAME); if (cleName == null) { throw new IOException("Expected entry to have " + "a field"); @@ -1339,7 +1383,7 @@ class OfflineImageReconstructor { } dirDiff.verifyNoRemainingKeys("dirDiff"); } - expectTagEnd("dirDiffEntry"); + expectTagEnd(SNAPSHOT_DIFF_SECTION_DIR_DIFF_ENTRY); } private void processFileDiffEntry() throws IOException { @@ -1348,12 +1392,14 @@ class OfflineImageReconstructor { headerBld.setType(DiffEntry.Type.FILEDIFF); Node fileDiffHeader = new Node(); loadNodeChildren(fileDiffHeader, "fileDiffEntry fields", "fileDiff"); - Long inodeId = fileDiffHeader.removeChildLong("inodeid"); + Long inodeId = fileDiffHeader.removeChildLong( + SNAPSHOT_DIFF_SECTION_INODE_ID); if (inodeId == null) { throw new IOException(" contained no entry."); } headerBld.setInodeId(inodeId); - Integer expectedDiffs = fileDiffHeader.removeChildInt("count"); + Integer expectedDiffs = fileDiffHeader.removeChildInt( + SNAPSHOT_DIFF_SECTION_COUNT); if (expectedDiffs == null) { throw new IOException(" contained no entry."); } @@ -1362,7 +1408,7 @@ class OfflineImageReconstructor { headerBld.build().writeDelimitedTo(out); for (int actualDiffs = 0; actualDiffs < expectedDiffs; actualDiffs++) { try { - expectTag("fileDiff", false); + expectTag(SNAPSHOT_DIFF_SECTION_FILE_DIFF, false); } catch (IOException e) { throw new IOException("Only read " + (actualDiffs + 1) + " diffs out of " + expectedDiffs, e); @@ -1371,15 +1417,17 @@ class OfflineImageReconstructor { loadNodeChildren(fileDiff, "fileDiff fields"); FsImageProto.SnapshotDiffSection.FileDiff.Builder bld = FsImageProto.SnapshotDiffSection.FileDiff.newBuilder(); - Integer snapshotId = fileDiff.removeChildInt("snapshotId"); + Integer snapshotId = fileDiff.removeChildInt( + SNAPSHOT_DIFF_SECTION_SNAPSHOT_ID); if (snapshotId != null) { bld.setSnapshotId(snapshotId); } - Long size = fileDiff.removeChildLong("size"); + Long size = fileDiff.removeChildLong( + SNAPSHOT_DIFF_SECTION_SIZE); if (size != null) { bld.setFileSize(size); } - String name = fileDiff.removeChildStr("name"); + String name = fileDiff.removeChildStr(SECTION_NAME); if (name != null) { bld.setName(ByteString.copyFrom(name, "UTF8")); } @@ -1388,7 +1436,7 @@ class OfflineImageReconstructor { fileDiff.verifyNoRemainingKeys("fileDiff"); bld.build().writeDelimitedTo(out); } - expectTagEnd("fileDiffEntry"); + expectTagEnd(SNAPSHOT_DIFF_SECTION_FILE_DIFF_ENTRY); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java index 71dc939098e..ba2dc0605b4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java @@ -77,6 +77,154 @@ import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_ */ @InterfaceAudience.Private public final class PBImageXmlWriter { + public static final String NAME_SECTION_NAME = "NameSection"; + public static final String INODE_SECTION_NAME = "INodeSection"; + public static final String SECRET_MANAGER_SECTION_NAME = + "SecretManagerSection"; + public static final String CACHE_MANAGER_SECTION_NAME = "CacheManagerSection"; + public static final String SNAPSHOT_DIFF_SECTION_NAME = "SnapshotDiffSection"; + public static final String INODE_REFERENCE_SECTION_NAME = + "INodeReferenceSection"; + public static final String INODE_DIRECTORY_SECTION_NAME = + "INodeDirectorySection"; + public static final String FILE_UNDER_CONSTRUCTION_SECTION_NAME = + "FileUnderConstructionSection"; + public static final String SNAPSHOT_SECTION_NAME = "SnapshotSection"; + + public static final String SECTION_ID = "id"; + public static final String SECTION_REPLICATION = "replication"; + public static final String SECTION_PATH = "path"; + public static final String SECTION_NAME = "name"; + + public static final String NAME_SECTION_NAMESPACE_ID = "namespaceId"; + public static final String NAME_SECTION_GENSTAMPV1 = "genstampV1"; + public static final String NAME_SECTION_GENSTAMPV2 = "genstampV2"; + public static final String NAME_SECTION_GENSTAMPV1_LIMIT = "genstampV1Limit"; + public static final String NAME_SECTION_LAST_ALLOCATED_BLOCK_ID = + "lastAllocatedBlockId"; + public static final String NAME_SECTION_TXID = "txid"; + public static final String NAME_SECTION_ROLLING_UPGRADE_START_TIME = + "rollingUpgradeStartTime"; + public static final String NAME_SECTION_LAST_ALLOCATED_STRIPED_BLOCK_ID = + "lastAllocatedStripedBlockId"; + + public static final String INODE_SECTION_LAST_INODE_ID = "lastInodeId"; + public static final String INODE_SECTION_NUM_INODES = "numInodes"; + public static final String INODE_SECTION_TYPE = "type"; + public static final String INODE_SECTION_MTIME = "mtime"; + public static final String INODE_SECTION_ATIME = "atime"; + public static final String INODE_SECTION_PREFERRED_BLOCK_SIZE = + "preferredBlockSize"; + public static final String INODE_SECTION_PERMISSION = "permission"; + public static final String INODE_SECTION_BLOCKS = "blocks"; + public static final String INODE_SECTION_BLOCK = "block"; + public static final String INODE_SECTION_GEMSTAMP = "genstamp"; + public static final String INODE_SECTION_NUM_BYTES = "numBytes"; + public static final String INODE_SECTION_FILE_UNDER_CONSTRUCTION = + "file-under-construction"; + public static final String INODE_SECTION_CLIENT_NAME = "clientName"; + public static final String INODE_SECTION_CLIENT_MACHINE = "clientMachine"; + public static final String INODE_SECTION_ACL = "acl"; + public static final String INODE_SECTION_ACLS = "acls"; + public static final String INODE_SECTION_XATTR = "xattr"; + public static final String INODE_SECTION_XATTRS = "xattrs"; + public static final String INODE_SECTION_STORAGE_POLICY_ID = + "storagePolicyId"; + public static final String INODE_SECTION_IS_STRIPED = "isStriped"; + public static final String INODE_SECTION_NS_QUOTA = "nsquota"; + public static final String INODE_SECTION_DS_QUOTA = "dsquota"; + public static final String INODE_SECTION_TYPE_QUOTA = "typeQuota"; + public static final String INODE_SECTION_QUOTA = "quota"; + public static final String INODE_SECTION_TARGET = "target"; + public static final String INODE_SECTION_NS = "ns"; + public static final String INODE_SECTION_VAL = "val"; + public static final String INODE_SECTION_VAL_HEX = "valHex"; + public static final String INODE_SECTION_INODE = "inode"; + + public static final String SECRET_MANAGER_SECTION_CURRENT_ID = "currentId"; + public static final String SECRET_MANAGER_SECTION_TOKEN_SEQUENCE_NUMBER = + "tokenSequenceNumber"; + public static final String SECRET_MANAGER_SECTION_NUM_DELEGATION_KEYS = + "numDelegationKeys"; + public static final String SECRET_MANAGER_SECTION_NUM_TOKENS = "numTokens"; + public static final String SECRET_MANAGER_SECTION_EXPIRY = "expiry"; + public static final String SECRET_MANAGER_SECTION_KEY = "key"; + public static final String SECRET_MANAGER_SECTION_DELEGATION_KEY = + "delegationKey"; + public static final String SECRET_MANAGER_SECTION_VERSION = "version"; + public static final String SECRET_MANAGER_SECTION_OWNER = "owner"; + public static final String SECRET_MANAGER_SECTION_RENEWER = "renewer"; + public static final String SECRET_MANAGER_SECTION_REAL_USER = "realUser"; + public static final String SECRET_MANAGER_SECTION_ISSUE_DATE = "issueDate"; + public static final String SECRET_MANAGER_SECTION_MAX_DATE = "maxDate"; + public static final String SECRET_MANAGER_SECTION_SEQUENCE_NUMBER = + "sequenceNumber"; + public static final String SECRET_MANAGER_SECTION_MASTER_KEY_ID = + "masterKeyId"; + public static final String SECRET_MANAGER_SECTION_EXPIRY_DATE = "expiryDate"; + public static final String SECRET_MANAGER_SECTION_TOKEN = "token"; + + public static final String CACHE_MANAGER_SECTION_NEXT_DIRECTIVE_ID = + "nextDirectiveId"; + public static final String CACHE_MANAGER_SECTION_NUM_POOLS = "numPools"; + public static final String CACHE_MANAGER_SECTION_NUM_DIRECTIVES = + "numDirectives"; + public static final String CACHE_MANAGER_SECTION_POOL_NAME = "poolName"; + public static final String CACHE_MANAGER_SECTION_OWNER_NAME = "ownerName"; + public static final String CACHE_MANAGER_SECTION_GROUP_NAME = "groupName"; + public static final String CACHE_MANAGER_SECTION_MODE = "mode"; + public static final String CACHE_MANAGER_SECTION_LIMIT = "limit"; + public static final String CACHE_MANAGER_SECTION_MAX_RELATIVE_EXPIRY = + "maxRelativeExpiry"; + public static final String CACHE_MANAGER_SECTION_POOL = "pool"; + public static final String CACHE_MANAGER_SECTION_EXPIRATION = "expiration"; + public static final String CACHE_MANAGER_SECTION_MILLIS = "millis"; + public static final String CACHE_MANAGER_SECTION_RELATIVE = "relative"; + public static final String CACHE_MANAGER_SECTION_DIRECTIVE = "directive"; + + public static final String SNAPSHOT_DIFF_SECTION_INODE_ID = "inodeId"; + public static final String SNAPSHOT_DIFF_SECTION_COUNT = "count"; + public static final String SNAPSHOT_DIFF_SECTION_SNAPSHOT_ID = "snapshotId"; + public static final String SNAPSHOT_DIFF_SECTION_CHILDREN_SIZE = + "childrenSize"; + public static final String SNAPSHOT_DIFF_SECTION_IS_SNAPSHOT_ROOT = + "isSnapshotRoot"; + public static final String SNAPSHOT_DIFF_SECTION_CREATED_LIST_SIZE = + "createdListSize"; + public static final String SNAPSHOT_DIFF_SECTION_DELETED_INODE = + "deletedInode"; + public static final String SNAPSHOT_DIFF_SECTION_DELETED_INODE_REF = + "deletedInoderef"; + public static final String SNAPSHOT_DIFF_SECTION_CREATED = "created"; + public static final String SNAPSHOT_DIFF_SECTION_SIZE = "size"; + public static final String SNAPSHOT_DIFF_SECTION_FILE_DIFF_ENTRY = + "fileDiffEntry"; + public static final String SNAPSHOT_DIFF_SECTION_DIR_DIFF_ENTRY = + "dirDiffEntry"; + public static final String SNAPSHOT_DIFF_SECTION_FILE_DIFF = "fileDiff"; + public static final String SNAPSHOT_DIFF_SECTION_DIR_DIFF = "dirDiff"; + + public static final String INODE_REFERENCE_SECTION_REFERRED_ID = "referredId"; + public static final String INODE_REFERENCE_SECTION_DST_SNAPSHOT_ID = + "dstSnapshotId"; + public static final String INODE_REFERENCE_SECTION_LAST_SNAPSHOT_ID = + "lastSnapshotId"; + public static final String INODE_REFERENCE_SECTION_REF = "ref"; + + public static final String INODE_DIRECTORY_SECTION_PARENT = "parent"; + public static final String INODE_DIRECTORY_SECTION_CHILD = "child"; + public static final String INODE_DIRECTORY_SECTION_REF_CHILD = "refChild"; + public static final String INODE_DIRECTORY_SECTION_DIRECTORY = "directory"; + + public static final String SNAPSHOT_SECTION_SNAPSHOT_COUNTER = + "snapshotCounter"; + public static final String SNAPSHOT_SECTION_NUM_SNAPSHOTS = "numSnapshots"; + public static final String SNAPSHOT_SECTION_SNAPSHOT_TABLE_DIR = + "snapshottableDir"; + public static final String SNAPSHOT_SECTION_DIR = "dir"; + public static final String SNAPSHOT_SECTION_ROOT = "root"; + public static final String SNAPSHOT_SECTION_SNAPSHOT = "snapshot"; + private final Configuration conf; private final PrintStream out; private final SimpleDateFormat isoDateFormat; @@ -177,98 +325,106 @@ public final class PBImageXmlWriter { } private void dumpCacheManagerSection(InputStream is) throws IOException { - out.print(""); + out.print("<" + CACHE_MANAGER_SECTION_NAME + ">"); CacheManagerSection s = CacheManagerSection.parseDelimitedFrom(is); - o("nextDirectiveId", s.getNextDirectiveId()); - o("numDirectives", s.getNumDirectives()); - o("numPools", s.getNumPools()); + o(CACHE_MANAGER_SECTION_NEXT_DIRECTIVE_ID, s.getNextDirectiveId()); + o(CACHE_MANAGER_SECTION_NUM_DIRECTIVES, s.getNumDirectives()); + o(CACHE_MANAGER_SECTION_NUM_POOLS, s.getNumPools()); for (int i = 0; i < s.getNumPools(); ++i) { CachePoolInfoProto p = CachePoolInfoProto.parseDelimitedFrom(is); - out.print(""); - o("poolName", p.getPoolName()).o("ownerName", p.getOwnerName()) - .o("groupName", p.getGroupName()).o("mode", p.getMode()) - .o("limit", p.getLimit()) - .o("maxRelativeExpiry", p.getMaxRelativeExpiry()); - out.print("\n"); + out.print("<" + CACHE_MANAGER_SECTION_POOL +">"); + o(CACHE_MANAGER_SECTION_POOL_NAME, p.getPoolName()). + o(CACHE_MANAGER_SECTION_OWNER_NAME, p.getOwnerName()) + .o(CACHE_MANAGER_SECTION_GROUP_NAME, p.getGroupName()) + .o(CACHE_MANAGER_SECTION_MODE, p.getMode()) + .o(CACHE_MANAGER_SECTION_LIMIT, p.getLimit()) + .o(CACHE_MANAGER_SECTION_MAX_RELATIVE_EXPIRY, + p.getMaxRelativeExpiry()); + out.print("\n"); } for (int i = 0; i < s.getNumDirectives(); ++i) { CacheDirectiveInfoProto p = CacheDirectiveInfoProto .parseDelimitedFrom(is); - out.print(""); - o("id", p.getId()).o("path", p.getPath()) - .o("replication", p.getReplication()).o("pool", p.getPool()); - out.print(""); + out.print("<" + CACHE_MANAGER_SECTION_DIRECTIVE + ">"); + o(SECTION_ID, p.getId()).o(SECTION_PATH, p.getPath()) + .o(SECTION_REPLICATION, p.getReplication()) + .o(CACHE_MANAGER_SECTION_POOL, p.getPool()); + out.print("<" + CACHE_MANAGER_SECTION_EXPIRATION +">"); CacheDirectiveInfoExpirationProto e = p.getExpiration(); - o("millis", e.getMillis()).o("relative", e.getIsRelative()); - out.print("\n"); - out.print("\n"); + o(CACHE_MANAGER_SECTION_MILLIS, e.getMillis()) + .o(CACHE_MANAGER_SECTION_RELATIVE, e.getIsRelative()); + out.print("\n"); + out.print("\n"); } - out.print("\n"); + out.print("\n"); } private void dumpFileUnderConstructionSection(InputStream in) throws IOException { - out.print(""); + out.print("<" + FILE_UNDER_CONSTRUCTION_SECTION_NAME + ">"); while (true) { FileUnderConstructionEntry e = FileUnderConstructionEntry .parseDelimitedFrom(in); if (e == null) { break; } - out.print(""); - o("id", e.getInodeId()).o("path", e.getFullPath()); - out.print("\n"); + out.print("<" + INODE_SECTION_INODE + ">"); + o(SECTION_ID, e.getInodeId()) + .o(SECTION_PATH, e.getFullPath()); + out.print("\n"); } - out.print("\n"); + out.print("\n"); } private void dumpXattrs(INodeSection.XAttrFeatureProto xattrs) { - out.print(""); + out.print("<" + INODE_SECTION_XATTRS + ">"); for (INodeSection.XAttrCompactProto xattr : xattrs.getXAttrsList()) { - out.print(""); + out.print("<" + INODE_SECTION_XATTR + ">"); int encodedName = xattr.getName(); int ns = (XATTR_NAMESPACE_MASK & (encodedName >> XATTR_NAMESPACE_OFFSET)) | ((XATTR_NAMESPACE_EXT_MASK & (encodedName >> XATTR_NAMESPACE_EXT_OFFSET)) << 2); - o("ns", XAttrProtos.XAttrProto. + o(INODE_SECTION_NS, XAttrProtos.XAttrProto. XAttrNamespaceProto.valueOf(ns).toString()); - o("name", stringTable[XATTR_NAME_MASK & (encodedName >> XATTR_NAME_OFFSET)]); + o(SECTION_NAME, + stringTable[XATTR_NAME_MASK & (encodedName >> XATTR_NAME_OFFSET)]); ByteString val = xattr.getValue(); if (val.isValidUtf8()) { - o("val", val.toStringUtf8()); + o(INODE_SECTION_VAL, val.toStringUtf8()); } else { - o("valHex", Hex.encodeHexString(val.toByteArray())); + o(INODE_SECTION_VAL_HEX, Hex.encodeHexString(val.toByteArray())); } - out.print(""); + out.print(""); } - out.print(""); + out.print(""); } private void dumpINodeDirectory(INodeDirectory d) { - o("mtime", d.getModificationTime()).o("permission", - dumpPermission(d.getPermission())); + o(INODE_SECTION_MTIME, d.getModificationTime()) + .o(INODE_SECTION_PERMISSION, dumpPermission(d.getPermission())); if (d.hasXAttrs()) { dumpXattrs(d.getXAttrs()); } dumpAcls(d.getAcl()); if (d.hasDsQuota() && d.hasNsQuota()) { - o("nsquota", d.getNsQuota()).o("dsquota", d.getDsQuota()); + o(INODE_SECTION_NS_QUOTA, d.getNsQuota()) + .o(INODE_SECTION_DS_QUOTA, d.getDsQuota()); } INodeSection.QuotaByStorageTypeFeatureProto typeQuotas = d.getTypeQuotas(); if (typeQuotas != null) { for (INodeSection.QuotaByStorageTypeEntryProto entry: typeQuotas.getQuotasList()) { - out.print(""); - o("type", entry.getStorageType().toString()); - o("quota", entry.getQuota()); - out.print(""); + out.print("<" + INODE_SECTION_TYPE_QUOTA + ">"); + o(INODE_SECTION_TYPE, entry.getStorageType().toString()); + o(INODE_SECTION_QUOTA, entry.getQuota()); + out.print(""); } } } private void dumpINodeDirectorySection(InputStream in) throws IOException { - out.print(""); + out.print("<" + INODE_DIRECTORY_SECTION_NAME + ">"); while (true) { INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry .parseDelimitedFrom(in); @@ -276,21 +432,21 @@ public final class PBImageXmlWriter { if (e == null) { break; } - out.print(""); - o("parent", e.getParent()); + out.print("<" + INODE_DIRECTORY_SECTION_DIRECTORY + ">"); + o(INODE_DIRECTORY_SECTION_PARENT, e.getParent()); for (long id : e.getChildrenList()) { - o("child", id); + o(INODE_DIRECTORY_SECTION_CHILD, id); } for (int refId : e.getRefChildrenList()) { - o("refChild", refId); + o(INODE_DIRECTORY_SECTION_REF_CHILD, refId); } - out.print("\n"); + out.print("\n"); } - out.print("\n"); + out.print("\n"); } private void dumpINodeReferenceSection(InputStream in) throws IOException { - out.print(""); + out.print("<" + INODE_REFERENCE_SECTION_NAME + ">"); while (true) { INodeReferenceSection.INodeReference e = INodeReferenceSection .INodeReference.parseDelimitedFrom(in); @@ -299,49 +455,53 @@ public final class PBImageXmlWriter { } dumpINodeReference(e); } - out.print(""); + out.print(""); } private void dumpINodeReference(INodeReferenceSection.INodeReference r) { - out.print(""); - o("referredId", r.getReferredId()).o("name", r.getName().toStringUtf8()) - .o("dstSnapshotId", r.getDstSnapshotId()) - .o("lastSnapshotId", r.getLastSnapshotId()); - out.print("\n"); + out.print("<" + INODE_REFERENCE_SECTION_REF + ">"); + o(INODE_REFERENCE_SECTION_REFERRED_ID, r.getReferredId()) + .o(SECTION_NAME, r.getName().toStringUtf8()) + .o(INODE_REFERENCE_SECTION_DST_SNAPSHOT_ID, r.getDstSnapshotId()) + .o(INODE_REFERENCE_SECTION_LAST_SNAPSHOT_ID, + r.getLastSnapshotId()); + out.print("\n"); } private void dumpINodeFile(INodeSection.INodeFile f) { - o("replication", f.getReplication()).o("mtime", f.getModificationTime()) - .o("atime", f.getAccessTime()) - .o("preferredBlockSize", f.getPreferredBlockSize()) - .o("permission", dumpPermission(f.getPermission())); + o(SECTION_REPLICATION, f.getReplication()) + .o(INODE_SECTION_MTIME, f.getModificationTime()) + .o(INODE_SECTION_ATIME, f.getAccessTime()) + .o(INODE_SECTION_PREFERRED_BLOCK_SIZE, f.getPreferredBlockSize()) + .o(INODE_SECTION_PERMISSION, dumpPermission(f.getPermission())); if (f.hasXAttrs()) { dumpXattrs(f.getXAttrs()); } dumpAcls(f.getAcl()); if (f.getBlocksCount() > 0) { - out.print(""); + out.print("<" + INODE_SECTION_BLOCKS + ">"); for (BlockProto b : f.getBlocksList()) { - out.print(""); - o("id", b.getBlockId()).o("genstamp", b.getGenStamp()).o("numBytes", - b.getNumBytes()); - out.print("\n"); + out.print("<" + INODE_SECTION_BLOCK + ">"); + o(SECTION_ID, b.getBlockId()) + .o(INODE_SECTION_GEMSTAMP, b.getGenStamp()) + .o(INODE_SECTION_NUM_BYTES, b.getNumBytes()); + out.print("\n"); } - out.print("\n"); + out.print("\n"); } if (f.hasStoragePolicyID()) { - o("storagePolicyId", f.getStoragePolicyID()); + o(INODE_SECTION_STORAGE_POLICY_ID, f.getStoragePolicyID()); } if (f.getIsStriped()) { - out.print(""); + out.print("<" + INODE_SECTION_IS_STRIPED + "/>"); } if (f.hasFileUC()) { INodeSection.FileUnderConstructionFeature u = f.getFileUC(); - out.print(""); - o("clientName", u.getClientName()).o("clientMachine", - u.getClientMachine()); - out.print("\n"); + out.print("<" + INODE_SECTION_FILE_UNDER_CONSTRUCTION + ">"); + o(INODE_SECTION_CLIENT_NAME, u.getClientName()) + .o(INODE_SECTION_CLIENT_MACHINE, u.getClientMachine()); + out.print("\n"); } } @@ -349,31 +509,31 @@ public final class PBImageXmlWriter { ImmutableList aclEntryList = FSImageFormatPBINode.Loader .loadAclEntries(aclFeatureProto, stringTable); if (aclEntryList.size() > 0) { - out.print(""); + out.print("<" + INODE_SECTION_ACLS + ">"); for (AclEntry aclEntry : aclEntryList) { - o("acl", aclEntry.toString()); + o(INODE_SECTION_ACL, aclEntry.toString()); } - out.print(""); + out.print(""); } } private void dumpINodeSection(InputStream in) throws IOException { INodeSection s = INodeSection.parseDelimitedFrom(in); - out.print(""); - o("lastInodeId", s.getLastInodeId()); - o("numInodes", s.getNumInodes()); + out.print("<" + INODE_SECTION_NAME + ">"); + o(INODE_SECTION_LAST_INODE_ID, s.getLastInodeId()); + o(INODE_SECTION_NUM_INODES, s.getNumInodes()); for (int i = 0; i < s.getNumInodes(); ++i) { INodeSection.INode p = INodeSection.INode.parseDelimitedFrom(in); - out.print(""); + out.print("<" + INODE_SECTION_INODE + ">"); dumpINodeFields(p); - out.print("\n"); + out.print("\n"); } - out.print("\n"); + out.print("\n"); } private void dumpINodeFields(INodeSection.INode p) { - o("id", p.getId()).o("type", p.getType()).o("name", - p.getName().toStringUtf8()); + o(SECTION_ID, p.getId()).o(INODE_SECTION_TYPE, p.getType()) + .o(SECTION_NAME, p.getName().toStringUtf8()); if (p.hasFile()) { dumpINodeFile(p.getFile()); } else if (p.hasDirectory()) { @@ -384,20 +544,23 @@ public final class PBImageXmlWriter { } private void dumpINodeSymlink(INodeSymlink s) { - o("permission", dumpPermission(s.getPermission())) - .o("target", s.getTarget().toStringUtf8()) - .o("mtime", s.getModificationTime()).o("atime", s.getAccessTime()); + o(INODE_SECTION_PERMISSION, dumpPermission(s.getPermission())) + .o(INODE_SECTION_TARGET, s.getTarget().toStringUtf8()) + .o(INODE_SECTION_MTIME, s.getModificationTime()) + .o(INODE_SECTION_ATIME, s.getAccessTime()); } private void dumpNameSection(InputStream in) throws IOException { NameSystemSection s = NameSystemSection.parseDelimitedFrom(in); - out.print(""); - o("namespaceId", s.getNamespaceId()); - o("genstampV1", s.getGenstampV1()).o("genstampV2", s.getGenstampV2()) - .o("genstampV1Limit", s.getGenstampV1Limit()) - .o("lastAllocatedBlockId", s.getLastAllocatedBlockId()) - .o("txid", s.getTransactionId()); - out.print("\n"); + out.print("<" + NAME_SECTION_NAME + ">"); + o(NAME_SECTION_NAMESPACE_ID, s.getNamespaceId()); + o(NAME_SECTION_GENSTAMPV1, s.getGenstampV1()) + .o(NAME_SECTION_GENSTAMPV2, s.getGenstampV2()) + .o(NAME_SECTION_GENSTAMPV1_LIMIT, s.getGenstampV1Limit()) + .o(NAME_SECTION_LAST_ALLOCATED_BLOCK_ID, + s.getLastAllocatedBlockId()) + .o(NAME_SECTION_TXID, s.getTransactionId()); + out.print("\n"); } private String dumpPermission(long permission) { @@ -408,59 +571,63 @@ public final class PBImageXmlWriter { } private void dumpSecretManagerSection(InputStream is) throws IOException { - out.print(""); + out.print("<" + SECRET_MANAGER_SECTION_NAME + ">"); SecretManagerSection s = SecretManagerSection.parseDelimitedFrom(is); int expectedNumDelegationKeys = s.getNumKeys(); int expectedNumTokens = s.getNumTokens(); - o("currentId", s.getCurrentId()).o("tokenSequenceNumber", - s.getTokenSequenceNumber()). - o("numDelegationKeys", expectedNumDelegationKeys). - o("numTokens", expectedNumTokens); + o(SECRET_MANAGER_SECTION_CURRENT_ID, s.getCurrentId()) + .o(SECRET_MANAGER_SECTION_TOKEN_SEQUENCE_NUMBER, + s.getTokenSequenceNumber()). + o(SECRET_MANAGER_SECTION_NUM_DELEGATION_KEYS, + expectedNumDelegationKeys). + o(SECRET_MANAGER_SECTION_NUM_TOKENS, expectedNumTokens); for (int i = 0; i < expectedNumDelegationKeys; i++) { SecretManagerSection.DelegationKey dkey = SecretManagerSection.DelegationKey.parseDelimitedFrom(is); - out.print(""); - o("id", dkey.getId()); - o("key", Hex.encodeHexString(dkey.getKey().toByteArray())); + out.print("<" + SECRET_MANAGER_SECTION_DELEGATION_KEY + ">"); + o(SECTION_ID, dkey.getId()); + o(SECRET_MANAGER_SECTION_KEY, + Hex.encodeHexString(dkey.getKey().toByteArray())); if (dkey.hasExpiryDate()) { - dumpDate("expiry", dkey.getExpiryDate()); + dumpDate(SECRET_MANAGER_SECTION_EXPIRY, dkey.getExpiryDate()); } - out.print(""); + out.print(""); } for (int i = 0; i < expectedNumTokens; i++) { SecretManagerSection.PersistToken token = SecretManagerSection.PersistToken.parseDelimitedFrom(is); - out.print(""); + out.print("<" + SECRET_MANAGER_SECTION_TOKEN + ">"); if (token.hasVersion()) { - o("version", token.getVersion()); + o(SECRET_MANAGER_SECTION_VERSION, token.getVersion()); } if (token.hasOwner()) { - o("owner", token.getOwner()); + o(SECRET_MANAGER_SECTION_OWNER, token.getOwner()); } if (token.hasRenewer()) { - o("renewer", token.getRenewer()); + o(SECRET_MANAGER_SECTION_RENEWER, token.getRenewer()); } if (token.hasRealUser()) { - o("realUser", token.getRealUser()); + o(SECRET_MANAGER_SECTION_REAL_USER, token.getRealUser()); } if (token.hasIssueDate()) { - dumpDate("issueDate", token.getIssueDate()); + dumpDate(SECRET_MANAGER_SECTION_ISSUE_DATE, token.getIssueDate()); } if (token.hasMaxDate()) { - dumpDate("maxDate", token.getMaxDate()); + dumpDate(SECRET_MANAGER_SECTION_MAX_DATE, token.getMaxDate()); } if (token.hasSequenceNumber()) { - o("sequenceNumber", token.getSequenceNumber()); + o(SECRET_MANAGER_SECTION_SEQUENCE_NUMBER, + token.getSequenceNumber()); } if (token.hasMasterKeyId()) { - o("masterKeyId", token.getMasterKeyId()); + o(SECRET_MANAGER_SECTION_MASTER_KEY_ID, token.getMasterKeyId()); } if (token.hasExpiryDate()) { - dumpDate("expiryDate", token.getExpiryDate()); + dumpDate(SECRET_MANAGER_SECTION_EXPIRY_DATE, token.getExpiryDate()); } - out.print(""); + out.print(""); } - out.print(""); + out.print(""); } private void dumpDate(String tag, long date) { @@ -469,7 +636,7 @@ public final class PBImageXmlWriter { } private void dumpSnapshotDiffSection(InputStream in) throws IOException { - out.print(""); + out.print("<" + SNAPSHOT_DIFF_SECTION_NAME + ">"); while (true) { SnapshotDiffSection.DiffEntry e = SnapshotDiffSection.DiffEntry .parseDelimitedFrom(in); @@ -478,52 +645,54 @@ public final class PBImageXmlWriter { } switch (e.getType()) { case FILEDIFF: - out.print(""); + out.print("<" + SNAPSHOT_DIFF_SECTION_FILE_DIFF_ENTRY + ">"); break; case DIRECTORYDIFF: - out.print(""); + out.print("<" + SNAPSHOT_DIFF_SECTION_DIR_DIFF_ENTRY + ">"); break; default: throw new IOException("unknown DiffEntry type " + e.getType()); } - o("inodeId", e.getInodeId()); - o("count", e.getNumOfDiff()); + o(SNAPSHOT_DIFF_SECTION_INODE_ID, e.getInodeId()); + o(SNAPSHOT_DIFF_SECTION_COUNT, e.getNumOfDiff()); switch (e.getType()) { case FILEDIFF: { for (int i = 0; i < e.getNumOfDiff(); ++i) { - out.print(""); + out.print("<" + SNAPSHOT_DIFF_SECTION_FILE_DIFF + ">"); SnapshotDiffSection.FileDiff f = SnapshotDiffSection.FileDiff .parseDelimitedFrom(in); - o("snapshotId", f.getSnapshotId()).o("size", f.getFileSize()).o( - "name", f.getName().toStringUtf8()); - out.print("\n"); + o(SNAPSHOT_DIFF_SECTION_SNAPSHOT_ID, f.getSnapshotId()) + .o(SNAPSHOT_DIFF_SECTION_SIZE, f.getFileSize()) + .o(SECTION_NAME, f.getName().toStringUtf8()); + out.print("\n"); } } break; case DIRECTORYDIFF: { for (int i = 0; i < e.getNumOfDiff(); ++i) { - out.print(""); + out.print("<" + SNAPSHOT_DIFF_SECTION_DIR_DIFF + ">"); SnapshotDiffSection.DirectoryDiff d = SnapshotDiffSection.DirectoryDiff .parseDelimitedFrom(in); - o("snapshotId", d.getSnapshotId()) - .o("childrenSize", d.getChildrenSize()) - .o("isSnapshotRoot", d.getIsSnapshotRoot()) - .o("name", d.getName().toStringUtf8()) - .o("createdListSize", d.getCreatedListSize()); + o(SNAPSHOT_DIFF_SECTION_SNAPSHOT_ID, d.getSnapshotId()) + .o(SNAPSHOT_DIFF_SECTION_CHILDREN_SIZE, d.getChildrenSize()) + .o(SNAPSHOT_DIFF_SECTION_IS_SNAPSHOT_ROOT, d.getIsSnapshotRoot()) + .o(SECTION_NAME, d.getName().toStringUtf8()) + .o(SNAPSHOT_DIFF_SECTION_CREATED_LIST_SIZE, + d.getCreatedListSize()); for (long did : d.getDeletedINodeList()) { - o("deletedInode", did); + o(SNAPSHOT_DIFF_SECTION_DELETED_INODE, did); } for (int dRefid : d.getDeletedINodeRefList()) { - o("deletedInoderef", dRefid); + o(SNAPSHOT_DIFF_SECTION_DELETED_INODE_REF, dRefid); } for (int j = 0; j < d.getCreatedListSize(); ++j) { SnapshotDiffSection.CreatedListEntry ce = SnapshotDiffSection.CreatedListEntry .parseDelimitedFrom(in); - out.print(""); - o("name", ce.getName().toStringUtf8()); - out.print("\n"); + out.print("<" + SNAPSHOT_DIFF_SECTION_CREATED + ">"); + o(SECTION_NAME, ce.getName().toStringUtf8()); + out.print("\n"); } - out.print("\n"); + out.print("\n"); } break; } @@ -532,41 +701,41 @@ public final class PBImageXmlWriter { } switch (e.getType()) { case FILEDIFF: - out.print(""); + out.print(""); break; case DIRECTORYDIFF: - out.print(""); + out.print(""); break; default: throw new IOException("unknown DiffEntry type " + e.getType()); } } - out.print("\n"); + out.print("\n"); } private void dumpSnapshotSection(InputStream in) throws IOException { - out.print(""); + out.print("<" + SNAPSHOT_SECTION_NAME + ">"); SnapshotSection s = SnapshotSection.parseDelimitedFrom(in); - o("snapshotCounter", s.getSnapshotCounter()); - o("numSnapshots", s.getNumSnapshots()); + o(SNAPSHOT_SECTION_SNAPSHOT_COUNTER, s.getSnapshotCounter()); + o(SNAPSHOT_SECTION_NUM_SNAPSHOTS, s.getNumSnapshots()); if (s.getSnapshottableDirCount() > 0) { - out.print(""); + out.print("<" + SNAPSHOT_SECTION_SNAPSHOT_TABLE_DIR + ">"); for (long id : s.getSnapshottableDirList()) { - o("dir", id); + o(SNAPSHOT_SECTION_DIR, id); } - out.print("\n"); + out.print("\n"); } for (int i = 0; i < s.getNumSnapshots(); ++i) { SnapshotSection.Snapshot pbs = SnapshotSection.Snapshot .parseDelimitedFrom(in); - out.print(""); - o("id", pbs.getSnapshotId()); - out.print(""); + out.print("<" + SNAPSHOT_SECTION_SNAPSHOT + ">"); + o(SECTION_ID, pbs.getSnapshotId()); + out.print("<" + SNAPSHOT_SECTION_ROOT + ">"); dumpINodeFields(pbs.getRoot()); - out.print(""); - out.print(""); + out.print(""); + out.print(""); } - out.print("\n"); + out.print("\n"); } private void loadStringTable(InputStream in) throws IOException { From e7ed05e4f5b0421e93f2f2cadc5beda3d28b9911 Mon Sep 17 00:00:00 2001 From: Benoy Antony Date: Mon, 21 Mar 2016 13:19:43 -0700 Subject: [PATCH 09/43] HADOOP-12929. JWTRedirectAuthenticationHandler must accommodate null expiration time. Contributed by Larry McCay. --- .../JWTRedirectAuthenticationHandler.java | 16 ++------ .../TestJWTRedirectAuthentictionHandler.java | 37 +++++++++++++++---- 2 files changed, 33 insertions(+), 20 deletions(-) diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/JWTRedirectAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/JWTRedirectAuthenticationHandler.java index cbe923be8eb..61f5b9e8943 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/JWTRedirectAuthenticationHandler.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/JWTRedirectAuthenticationHandler.java @@ -26,20 +26,10 @@ import java.util.List; import java.util.Properties; import java.text.ParseException; -import java.io.ByteArrayInputStream; -import java.io.UnsupportedEncodingException; -import java.security.PublicKey; -import java.security.cert.CertificateFactory; -import java.security.cert.X509Certificate; -import java.security.cert.CertificateException; import java.security.interfaces.RSAPublicKey; -import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.security.authentication.client.AuthenticationException; -import org.apache.hadoop.security.authentication.server.AltKerberosAuthenticationHandler; -import org.apache.hadoop.security.authentication.server.AuthenticationToken; import org.apache.hadoop.security.authentication.util.CertificateUtil; -import org.apache.hadoop.security.authentication.util.KerberosName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,7 +73,8 @@ public class JWTRedirectAuthenticationHandler extends private static Logger LOG = LoggerFactory .getLogger(JWTRedirectAuthenticationHandler.class); - public static final String AUTHENTICATION_PROVIDER_URL = "authentication.provider.url"; + public static final String AUTHENTICATION_PROVIDER_URL = + "authentication.provider.url"; public static final String PUBLIC_KEY_PEM = "public.key.pem"; public static final String EXPECTED_JWT_AUDIENCES = "expected.jwt.audiences"; public static final String JWT_COOKIE_NAME = "jwt.cookie.name"; @@ -205,7 +196,6 @@ public class JWTRedirectAuthenticationHandler extends protected String getJWTFromCookie(HttpServletRequest req) { String serializedJWT = null; Cookie[] cookies = req.getCookies(); - String userName = null; if (cookies != null) { for (Cookie cookie : cookies) { if (cookieName.equals(cookie.getName())) { @@ -350,7 +340,7 @@ public class JWTRedirectAuthenticationHandler extends boolean valid = false; try { Date expires = jwtToken.getJWTClaimsSet().getExpirationTime(); - if (expires != null && new Date().before(expires)) { + if (expires == null || new Date().before(expires)) { LOG.debug("JWT token expiration date has been " + "successfully validated"); valid = true; diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestJWTRedirectAuthentictionHandler.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestJWTRedirectAuthentictionHandler.java index 019ecb416ce..97a8a9d2c5d 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestJWTRedirectAuthentictionHandler.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestJWTRedirectAuthentictionHandler.java @@ -13,19 +13,15 @@ */ package org.apache.hadoop.security.authentication.server; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.File; -import java.net.MalformedURLException; -import java.net.URL; import java.security.KeyPair; import java.security.KeyPairGenerator; import java.security.NoSuchAlgorithmException; import java.security.interfaces.RSAPrivateKey; import java.security.interfaces.RSAPublicKey; -import java.util.Arrays; import java.util.List; import java.util.ArrayList; import java.util.Properties; @@ -50,8 +46,6 @@ import com.nimbusds.jose.*; import com.nimbusds.jwt.JWTClaimsSet; import com.nimbusds.jwt.SignedJWT; import com.nimbusds.jose.crypto.RSASSASigner; -import com.nimbusds.jose.crypto.RSASSAVerifier; -import com.nimbusds.jose.util.Base64URL; public class TestJWTRedirectAuthentictionHandler extends KerberosSecurityTestcase { @@ -261,6 +255,36 @@ public class TestJWTRedirectAuthentictionHandler extends } } + @Test + public void testNoExpirationJWT() throws Exception { + try { + handler.setPublicKey(publicKey); + + Properties props = getProperties(); + handler.init(props); + + SignedJWT jwt = getJWT("bob", null, privateKey); + + Cookie cookie = new Cookie("hadoop-jwt", jwt.serialize()); + HttpServletRequest request = Mockito.mock(HttpServletRequest.class); + Mockito.when(request.getCookies()).thenReturn(new Cookie[] { cookie }); + Mockito.when(request.getRequestURL()).thenReturn( + new StringBuffer(SERVICE_URL)); + HttpServletResponse response = Mockito.mock(HttpServletResponse.class); + Mockito.when(response.encodeRedirectURL(SERVICE_URL)).thenReturn( + SERVICE_URL); + + AuthenticationToken token = handler.alternateAuthenticate(request, + response); + Assert.assertNotNull("Token should not be null.", token); + Assert.assertEquals("bob", token.getUserName()); + } catch (ServletException se) { + fail("alternateAuthentication should NOT have thrown a ServletException"); + } catch (AuthenticationException ae) { + fail("alternateAuthentication should NOT have thrown a AuthenticationException"); + } + } + @Test public void testInvalidAudienceJWT() throws Exception { try { @@ -442,7 +466,6 @@ public class TestJWTRedirectAuthentictionHandler extends JWSHeader header = new JWSHeader.Builder(JWSAlgorithm.RS256).build(); SignedJWT signedJWT = new SignedJWT(header, claimsSet); - Base64URL sigInput = Base64URL.encode(signedJWT.getSigningInput()); JWSSigner signer = new RSASSASigner(privateKey); signedJWT.sign(signer); From 57ead18a85e15aef1993f49157cf05aed38f1c87 Mon Sep 17 00:00:00 2001 From: Li Lu Date: Tue, 22 Mar 2016 14:27:32 -0700 Subject: [PATCH 10/43] MAPREDUCE-6110. JobHistoryServer CLI throws NullPointerException with job ids that do not exist. (Kai Sasaki via gtcarrera9) --- .../apache/hadoop/mapreduce/tools/CLI.java | 48 ++++++++++++------- .../hadoop/mapreduce/tools/TestCLI.java | 38 ++++++++++++++- 2 files changed, 68 insertions(+), 18 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java index bc3c0c80f47..4f557922db9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java @@ -402,8 +402,13 @@ public class CLI extends Configured implements Tool { } } } else if (listEvents) { - listEvents(getJob(JobID.forName(jobid)), fromEvent, nEvents); - exitCode = 0; + Job job = getJob(JobID.forName(jobid)); + if (job == null) { + System.out.println("Could not find job " + jobid); + } else { + listEvents(job, fromEvent, nEvents); + exitCode = 0; + } } else if (listJobs) { listJobs(cluster); exitCode = 0; @@ -417,8 +422,13 @@ public class CLI extends Configured implements Tool { listBlacklistedTrackers(cluster); exitCode = 0; } else if (displayTasks) { - displayTasks(getJob(JobID.forName(jobid)), taskType, taskState); - exitCode = 0; + Job job = getJob(JobID.forName(jobid)); + if (job == null) { + System.out.println("Could not find job " + jobid); + } else { + displayTasks(getJob(JobID.forName(jobid)), taskType, taskState); + exitCode = 0; + } } else if(killTask) { TaskAttemptID taskID = TaskAttemptID.forName(taskid); Job job = getJob(taskID.getJobID()); @@ -444,20 +454,24 @@ public class CLI extends Configured implements Tool { exitCode = -1; } } else if (logs) { - try { JobID jobID = JobID.forName(jobid); - TaskAttemptID taskAttemptID = TaskAttemptID.forName(taskid); - LogParams logParams = cluster.getLogParams(jobID, taskAttemptID); - LogCLIHelpers logDumper = new LogCLIHelpers(); - logDumper.setConf(getConf()); - exitCode = logDumper.dumpAContainersLogs(logParams.getApplicationId(), - logParams.getContainerId(), logParams.getNodeId(), - logParams.getOwner()); - } catch (IOException e) { - if (e instanceof RemoteException) { - throw e; - } - System.out.println(e.getMessage()); + if (getJob(jobID) == null) { + System.out.println("Could not find job " + jobid); + } else { + try { + TaskAttemptID taskAttemptID = TaskAttemptID.forName(taskid); + LogParams logParams = cluster.getLogParams(jobID, taskAttemptID); + LogCLIHelpers logDumper = new LogCLIHelpers(); + logDumper.setConf(getConf()); + exitCode = logDumper.dumpAContainersLogs( + logParams.getApplicationId(), logParams.getContainerId(), + logParams.getNodeId(), logParams.getOwner()); + } catch (IOException e) { + if (e instanceof RemoteException) { + throw e; + } + System.out.println(e.getMessage()); + } } } } catch (RemoteException re) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java index 73f57d5d7dd..112f5855b01 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java @@ -87,7 +87,7 @@ public class TestCLI { JobID jobId = JobID.forName(jobIdStr); Cluster mockCluster = mock(Cluster.class); Job job = mock(Job.class); - CLI cli = spy(new CLI()); + CLI cli = spy(new CLI(new Configuration())); doReturn(mockCluster).when(cli).createCluster(); when(mockCluster.getJob(jobId)).thenReturn(job); @@ -101,12 +101,18 @@ public class TestCLI { int retCode_invalidTaskState = cli.run(new String[] { "-list-attempt-ids", jobIdStr, "REDUCE", "complete" }); + String jobIdStr2 = "job_1015298225799_0016"; + int retCode_invalidJobId = cli.run(new String[] { "-list-attempt-ids", + jobIdStr2, "MAP", "running" }); + assertEquals("JOB_SETUP is an invalid input,exit code should be -1", -1, retCode_JOB_SETUP); assertEquals("JOB_CLEANUP is an invalid input,exit code should be -1", -1, retCode_JOB_CLEANUP); assertEquals("complete is an invalid input,exit code should be -1", -1, retCode_invalidTaskState); + assertEquals("Non existing job id should be skippted with -1", -1, + retCode_invalidJobId); } @@ -176,4 +182,34 @@ public class TestCLI { Assert.assertTrue(end - start < ((i + 1) * sleepTime)); } } + + @Test + public void testListEvents() throws Exception { + Cluster mockCluster = mock(Cluster.class); + CLI cli = spy(new CLI(new Configuration())); + doReturn(mockCluster).when(cli).createCluster(); + String jobId1 = "job_1234654654_001"; + String jobId2 = "job_1234654656_002"; + + Job mockJob1 = mockJob(mockCluster, jobId1, State.RUNNING); + + // Check exiting with non existing job + int exitCode = cli.run(new String[]{"-events", jobId2, "0", "10"}); + assertEquals(-1, exitCode); + } + + @Test + public void testLogs() throws Exception { + Cluster mockCluster = mock(Cluster.class); + CLI cli = spy(new CLI(new Configuration())); + doReturn(mockCluster).when(cli).createCluster(); + String jobId1 = "job_1234654654_001"; + String jobId2 = "job_1234654656_002"; + + Job mockJob1 = mockJob(mockCluster, jobId1, State.SUCCEEDED); + + // Check exiting with non existing job + int exitCode = cli.run(new String[]{"-logs", jobId2}); + assertEquals(-1, exitCode); + } } From a7d8f2b3960d27c74abb17ce2aa4bcd999706ad2 Mon Sep 17 00:00:00 2001 From: Colin Patrick Mccabe Date: Tue, 22 Mar 2016 14:56:02 -0700 Subject: [PATCH 11/43] HDFS-10189. PacketResponder#toString should include the downstreams for PacketResponderType.HAS_DOWNSTREAM_IN_PIPELINE (Joe Pallas via cmccabe) --- .../org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index dfa40319f9b..2e4ee028473 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -1157,7 +1157,7 @@ class BlockReceiver implements Closeable { final StringBuilder b = new StringBuilder(getClass().getSimpleName()) .append(": ").append(block).append(", type=").append(type); - if (type != PacketResponderType.HAS_DOWNSTREAM_IN_PIPELINE) { + if (type == PacketResponderType.HAS_DOWNSTREAM_IN_PIPELINE) { b.append(", downstreams=").append(downstreams.length) .append(":").append(Arrays.asList(downstreams)); } From 0bfe5a0167f0af65f1d7a8bc251a1c404ed41316 Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Wed, 23 Mar 2016 13:18:06 +0900 Subject: [PATCH 12/43] MAPREDUCE-6580. Test failure: TestMRJobsWithProfiler. Contributed by Eric Badger. --- .../org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java index df55f509d9e..cbf1f39b9d4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java @@ -134,7 +134,7 @@ public class TestMRJobsWithProfiler { if (!useDefault) { // use hprof for map to profile.out sleepConf.set(MRJobConfig.TASK_MAP_PROFILE_PARAMS, - "-agentlib:hprof=cpu=times,heap=sites,force=n,thread=y,verbose=n," + "-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n," + "file=%s"); // use Xprof for reduce to stdout From a107cee14bb5446057da81d1c95d7fffd759e497 Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Wed, 23 Mar 2016 14:15:26 +0900 Subject: [PATCH 13/43] HADOOP-12952. BUILDING example of zero-docs dist should skip javadocs. Contributed by Steve Loughran. --- BUILDING.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/BUILDING.txt b/BUILDING.txt index 3eb9e9ca409..408cae13055 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -252,7 +252,7 @@ Building distributions: Create binary distribution without native code and without documentation: - $ mvn package -Pdist -DskipTests -Dtar + $ mvn package -Pdist -DskipTests -Dtar -Dmaven.javadoc.skip=true Create binary distribution with native code and with documentation: From 368c77376a3704e3fa395d72b1816d388b5cd782 Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Wed, 23 Mar 2016 23:22:43 +0900 Subject: [PATCH 14/43] MAPREDUCE-6656. [NNBench] OP_DELETE operation isn't working after MAPREDUCE-6363. Contributed by J.Andreina. --- .../src/test/java/org/apache/hadoop/hdfs/NNBench.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java index 666ef0ec55a..96c4710b48c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java @@ -712,6 +712,7 @@ public class NNBench { doRenameOp(fileName, reporter); } else if (op.equals(OP_DELETE)) { startTimeTPmS = System.currentTimeMillis(); + doDeleteOp(fileName, reporter); } else { throw new IllegalArgumentException( "unsupported operation [" + op + "]"); From af1d125f9ce35ec69a610674a1c5c60cc17141a7 Mon Sep 17 00:00:00 2001 From: Junping Du Date: Wed, 23 Mar 2016 08:57:16 -0700 Subject: [PATCH 15/43] YARN-4814. ATS 1.5 timelineclient impl call flush after every event write. Contributed by Xuan Gong. --- .../hadoop/yarn/conf/YarnConfiguration.java | 6 + .../api/impl/FileSystemTimelineWriter.java | 104 ++++++++++++++---- 2 files changed, 86 insertions(+), 24 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index ff4b4938076..8acee579ff3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -1747,6 +1747,12 @@ public class YarnConfiguration extends Configuration { public static final long TIMELINE_SERVICE_CLIENT_INTERNAL_TIMERS_TTL_SECS_DEFAULT = 7 * 60; + public static final String + TIMELINE_SERVICE_CLIENT_INTERNAL_ATTEMPT_DIR_CACHE_SIZE = + TIMELINE_SERVICE_CLIENT_PREFIX + "internal-attempt-dir-cache-size"; + public static final int + DEFAULT_TIMELINE_SERVICE_CLIENT_INTERNAL_ATTEMPT_DIR_CACHE_SIZE = 1000; + // This is temporary solution. The configuration will be deleted once we have // the FileSystem API to check whether append operation is supported or not. public static final String TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java index b471b3b59b3..35d9970ba40 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java @@ -26,6 +26,7 @@ import java.net.URI; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -97,6 +98,7 @@ public class FileSystemTimelineWriter extends TimelineWriter{ private long ttl; private LogFDsCache logFDsCache = null; private boolean isAppendSupported; + private final AttemptDirCache attemptDirCache; public FileSystemTimelineWriter(Configuration conf, UserGroupInformation authUgi, Client client, URI resURI) @@ -158,6 +160,15 @@ public class FileSystemTimelineWriter extends TimelineWriter{ objMapper = createObjectMapper(); + int attemptDirCacheSize = conf.getInt( + YarnConfiguration + .TIMELINE_SERVICE_CLIENT_INTERNAL_ATTEMPT_DIR_CACHE_SIZE, + YarnConfiguration + .DEFAULT_TIMELINE_SERVICE_CLIENT_INTERNAL_ATTEMPT_DIR_CACHE_SIZE); + + attemptDirCache = + new AttemptDirCache(attemptDirCacheSize, fs, activePath); + if (LOG.isDebugEnabled()) { StringBuilder debugMSG = new StringBuilder(); debugMSG.append( @@ -199,7 +210,7 @@ public class FileSystemTimelineWriter extends TimelineWriter{ = new ArrayList(); List entitiesToEntityCache = new ArrayList(); - Path attemptDir = createAttemptDir(appAttemptId); + Path attemptDir = attemptDirCache.getAppAttemptDir(appAttemptId); for (TimelineEntity entity : entities) { if (summaryEntityTypes.contains(entity.getEntityType())) { @@ -279,32 +290,11 @@ public class FileSystemTimelineWriter extends TimelineWriter{ return mapper; } - private Path createAttemptDir(ApplicationAttemptId appAttemptId) - throws IOException { - Path appDir = createApplicationDir(appAttemptId.getApplicationId()); - - Path attemptDir = new Path(appDir, appAttemptId.toString()); - if (!fs.exists(attemptDir)) { - FileSystem.mkdirs(fs, attemptDir, new FsPermission( - APP_LOG_DIR_PERMISSIONS)); - } - return attemptDir; - } - - private Path createApplicationDir(ApplicationId appId) throws IOException { - Path appDir = - new Path(activePath, appId.toString()); - if (!fs.exists(appDir)) { - FileSystem.mkdirs(fs, appDir, new FsPermission(APP_LOG_DIR_PERMISSIONS)); - } - return appDir; - } - private void writeDomain(ApplicationAttemptId appAttemptId, TimelineDomain domain) throws IOException { Path domainLogPath = - new Path(createAttemptDir(appAttemptId), DOMAIN_LOG_PREFIX - + appAttemptId.toString()); + new Path(attemptDirCache.getAppAttemptDir(appAttemptId), + DOMAIN_LOG_PREFIX + appAttemptId.toString()); if (LOG.isDebugEnabled()) { LOG.debug("Writing domains for " + appAttemptId.toString() + " to " + domainLogPath); @@ -958,4 +948,70 @@ public class FileSystemTimelineWriter extends TimelineWriter{ } } } + + private static class AttemptDirCache { + private final int attemptDirCacheSize; + private final Map attemptDirCache; + private final FileSystem fs; + private final Path activePath; + + public AttemptDirCache(int cacheSize, FileSystem fs, Path activePath) { + this.attemptDirCacheSize = cacheSize; + this.attemptDirCache = + new LinkedHashMap( + attemptDirCacheSize, 0.75f, true) { + private static final long serialVersionUID = 1L; + @Override + protected boolean removeEldestEntry( + Map.Entry eldest) { + return size() > attemptDirCacheSize; + } + }; + this.fs = fs; + this.activePath = activePath; + } + + public Path getAppAttemptDir(ApplicationAttemptId attemptId) + throws IOException { + Path attemptDir = this.attemptDirCache.get(attemptId); + if (attemptDir == null) { + synchronized(this) { + attemptDir = this.attemptDirCache.get(attemptId); + if (attemptDir == null) { + attemptDir = createAttemptDir(attemptId); + attemptDirCache.put(attemptId, attemptDir); + } + } + } + return attemptDir; + } + + private Path createAttemptDir(ApplicationAttemptId appAttemptId) + throws IOException { + Path appDir = createApplicationDir(appAttemptId.getApplicationId()); + + Path attemptDir = new Path(appDir, appAttemptId.toString()); + if (!fs.exists(attemptDir)) { + FileSystem.mkdirs(fs, attemptDir, new FsPermission( + APP_LOG_DIR_PERMISSIONS)); + if (LOG.isDebugEnabled()) { + LOG.debug("New attempt directory created - " + attemptDir); + } + } + return attemptDir; + } + + private Path createApplicationDir(ApplicationId appId) throws IOException { + Path appDir = + new Path(activePath, appId.toString()); + if (!fs.exists(appDir)) { + FileSystem.mkdirs(fs, appDir, + new FsPermission(APP_LOG_DIR_PERMISSIONS)); + if (LOG.isDebugEnabled()) { + LOG.debug("New app directory created - " + appDir); + } + } + return appDir; + } + } } From 0d19a0ce98053572447bdadf88687ec55f2f1f46 Mon Sep 17 00:00:00 2001 From: Colin Patrick Mccabe Date: Wed, 23 Mar 2016 11:19:18 -0700 Subject: [PATCH 16/43] HDFS-10193. fuse_dfs segfaults if uid cannot be resolved to a username (John Thiltges via cmccabe) --- .../src/main/native/fuse-dfs/fuse_connect.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_connect.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_connect.c index e696072d8cf..6ee4ad5130e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_connect.c +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_connect.c @@ -570,6 +570,11 @@ int fuseConnectAsThreadUid(struct hdfsConn **conn) ctx = fuse_get_context(); usrname = getUsername(ctx->uid); + if (!usrname) { + ERROR("fuseConnectAsThreadUid(): failed to get username for uid %"PRId64 + "\n", (uint64_t)ctx->uid); + return EIO; + } ret = fuseConnect(usrname, ctx, conn); free(usrname); return ret; From 8f85e5d2128c54c47d2a0098f6f4d4e04d53d74b Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Wed, 23 Mar 2016 12:55:04 -0700 Subject: [PATCH 17/43] HDFS-10200. Docs for WebHDFS still describe GETDELEGATIONTOKENS operation. Contributed by Wellington Chevreuil. --- .../hadoop-hdfs/src/site/markdown/WebHDFS.md | 61 +------------------ 1 file changed, 3 insertions(+), 58 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md index c7194825d48..7c3099a0e0e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md @@ -65,7 +65,6 @@ WebHDFS REST API * [Rename Snapshot](#Rename_Snapshot) * [Delegation Token Operations](#Delegation_Token_Operations) * [Get Delegation Token](#Get_Delegation_Token) - * [Get Delegation Tokens](#Get_Delegation_Tokens) * [Renew Delegation Token](#Renew_Delegation_Token) * [Cancel Delegation Token](#Cancel_Delegation_Token) * [Error Responses](#Error_Responses) @@ -89,7 +88,6 @@ WebHDFS REST API * [RemoteException JSON Schema](#RemoteException_JSON_Schema) * [Token JSON Schema](#Token_JSON_Schema) * [Token Properties](#Token_Properties) - * [Tokens JSON Schema](#Tokens_JSON_Schema) * [HTTP Query Parameter Dictionary](#HTTP_Query_Parameter_Dictionary) * [ACL Spec](#ACL_Spec) * [XAttr Name](#XAttr_Name) @@ -148,7 +146,6 @@ The HTTP REST API supports the complete [FileSystem](../../api/org/apache/hadoop * [`GETFILECHECKSUM`](#Get_File_Checksum) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileChecksum) * [`GETHOMEDIRECTORY`](#Get_Home_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getHomeDirectory) * [`GETDELEGATIONTOKEN`](#Get_Delegation_Token) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getDelegationToken) - * [`GETDELEGATIONTOKENS`](#Get_Delegation_Tokens) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getDelegationTokens) * [`GETXATTRS`](#Get_an_XAttr) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttr) * [`GETXATTRS`](#Get_multiple_XAttrs) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttrs) * [`GETXATTRS`](#Get_all_XAttrs) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttrs) @@ -1033,32 +1030,6 @@ Delegation Token Operations See also: [`renewer`](#Renewer), [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getDelegationToken, [`kind`](#Token_Kind), [`service`](#Token_Service) -### Get Delegation Tokens - -* Submit a HTTP GET request. - - curl -i "http://:/webhdfs/v1/?op=GETDELEGATIONTOKENS&renewer=" - - The client receives a response with a [`Tokens` JSON object](#Tokens_JSON_Schema): - - HTTP/1.1 200 OK - Content-Type: application/json - Transfer-Encoding: chunked - - { - "Tokens": - { - "Token": - [ - { - "urlString":"KAAKSm9i ..." - } - ] - } - } - -See also: [`renewer`](#Renewer), [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getDelegationTokens - ### Renew Delegation Token * Submit a HTTP PUT request. @@ -1655,7 +1626,7 @@ See also: [`Token` Properties](#Token_Properties), [`GETDELEGATIONTOKEN`](#Get_D #### Token Properties -JavaScript syntax is used to define `tokenProperties` so that it can be referred in both `Token` and `Tokens` JSON schemas. +JavaScript syntax is used to define `tokenProperties` so that it can be referred in `Token` JSON schema. ```json var tokenProperties = @@ -1673,33 +1644,7 @@ var tokenProperties = } ``` -### Tokens JSON Schema - -A `Tokens` JSON object represents an array of `Token` JSON objects. - -```json -{ - "name" : "Tokens", - "properties": - { - "Tokens": - { - "type" : "object", - "properties": - { - "Token": - { - "description": "An array of Token", - "type" : "array", - "items" : "Token": tokenProperties //See Token Properties - } - } - } - } -} -``` - -See also: [`Token` Properties](#Token_Properties), [`GETDELEGATIONTOKENS`](#Get_Delegation_Tokens), the note in [Delegation](#Delegation). +See also: [`Token` Properties](#Token_Properties), the note in [Delegation](#Delegation). HTTP Query Parameter Dictionary ------------------------------- @@ -2013,7 +1958,7 @@ See also: [`RENAME`](#Rename_a_FileDirectory) | Valid Values | Any valid username. | | Syntax | Any string. | -See also: [`GETDELEGATIONTOKEN`](#Get_Delegation_Token), [`GETDELEGATIONTOKENS`](#Get_Delegation_Tokens) +See also: [`GETDELEGATIONTOKEN`](#Get_Delegation_Token) ### Replication From 738155063e6fa3f1811e2e875e2e9611f35ef423 Mon Sep 17 00:00:00 2001 From: Allen Wittenauer Date: Wed, 2 Mar 2016 13:13:36 -0800 Subject: [PATCH 18/43] HADOOP-12857. rework hadoop-tools (aw) --- dev-support/bin/dist-tools-hooks-maker | 182 ++++++++++++++++++ .../hadoop-common/src/main/bin/hadoop | 10 +- .../src/main/bin/hadoop-functions.sh | 92 ++++++--- .../src/main/bin/hadoop-layout.sh.example | 10 +- .../hadoop-common/src/main/conf/hadoop-env.sh | 23 ++- .../hadoop_add_to_classpath_tools.bats | 47 +++++ .../hadoop_add_to_classpath_toolspath.bats | 74 ------- .../src/test/scripts/hadoop_basic_init.bats | 3 +- .../src/test/scripts/hadoop_bootstrap.bats | 15 +- .../src/test/scripts/hadoop_entry_tests.bats | 49 +++++ .../src/test/scripts/hadoop_shellprofile.bats | 7 + hadoop-dist/pom.xml | 18 ++ .../hadoop-hdfs/src/main/bin/hdfs | 5 +- hadoop-mapreduce-project/bin/mapred | 10 +- hadoop-tools/hadoop-archive-logs/pom.xml | 17 ++ hadoop-tools/hadoop-archives/pom.xml | 17 ++ hadoop-tools/hadoop-aws/pom.xml | 17 ++ .../site/markdown/tools/hadoop-aws/index.md | 5 +- hadoop-tools/hadoop-azure/pom.xml | 18 ++ .../hadoop-azure/src/site/markdown/index.md | 3 + hadoop-tools/hadoop-datajoin/pom.xml | 16 ++ hadoop-tools/hadoop-distcp/pom.xml | 11 ++ hadoop-tools/hadoop-extras/pom.xml | 17 ++ hadoop-tools/hadoop-gridmix/pom.xml | 17 ++ hadoop-tools/hadoop-kafka/pom.xml | 17 ++ hadoop-tools/hadoop-openstack/pom.xml | 17 ++ .../src/site/markdown/index.md | 7 +- hadoop-tools/hadoop-rumen/pom.xml | 17 ++ hadoop-tools/hadoop-sls/pom.xml | 17 ++ .../hadoop-sls/src/main/bin/rumen2sls.sh | 2 +- .../hadoop-sls/src/main/bin/slsrun.sh | 2 +- hadoop-tools/hadoop-streaming/pom.xml | 17 ++ hadoop-yarn-project/hadoop-yarn/bin/yarn | 4 +- 33 files changed, 651 insertions(+), 132 deletions(-) create mode 100755 dev-support/bin/dist-tools-hooks-maker mode change 100644 => 100755 hadoop-common-project/hadoop-common/src/main/bin/hadoop-layout.sh.example create mode 100644 hadoop-common-project/hadoop-common/src/test/scripts/hadoop_add_to_classpath_tools.bats delete mode 100644 hadoop-common-project/hadoop-common/src/test/scripts/hadoop_add_to_classpath_toolspath.bats create mode 100644 hadoop-common-project/hadoop-common/src/test/scripts/hadoop_entry_tests.bats diff --git a/dev-support/bin/dist-tools-hooks-maker b/dev-support/bin/dist-tools-hooks-maker new file mode 100755 index 00000000000..a98dffa7ad2 --- /dev/null +++ b/dev-support/bin/dist-tools-hooks-maker @@ -0,0 +1,182 @@ +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +VERSION=${1:-3.0.0-SNAPSHOT} +TARGETDIR=${2:-/tmp/target} +TOOLSDIR=${3:-/tmp/tools} + +function getfilename +{ + declare module=$1 + declare modtype=$2 + + if [[ ${modtype} = builtin ]]; then + echo "${TARGETDIR}/hadoop-${VERSION}/libexec/tools/${module}.sh" + else + echo "${TARGETDIR}/hadoop-${VERSION}/libexec/shellprofile.d/${module}.sh" + fi +} + +function header +{ + declare fn=$1 + + cat >>"${fn}" <<-'TOKEN' +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# +# +# IMPORTANT: This file is automatically generated by hadoop-dist at +# -Pdist time. +# +# +TOKEN + +} + +function optional_prologue +{ + declare fn=$1 + declare module=$2 + + if [[ -z "${OPTMODS}" ]]; then + OPTMODS=${module} + else + OPTMODS="${OPTMODS},${module}" + fi + + { + echo "if hadoop_verify_entry HADOOP_TOOLS_OPTIONS \"${module}\"; then" + echo " hadoop_add_profile \"${module}\"" + echo "fi" + echo "" + echo "function _${module}_hadoop_classpath" + echo "{" + } >> "${fn}" +} + +function builtin_prologue +{ + declare fn=$1 + declare module=$2 + + { + echo "" + echo "function hadoop_classpath_tools_${module}" + echo "{" + } >> "${fn}" +} + +function dependencywork +{ + declare fn=$1 + declare module=$2 + declare depfn=$3 + + declare depline + declare jarname + + while read -r depline; do + jarname=$(echo "${depline}" | awk -F: '{print $2"-"$4".jar"}') + + if [[ -f "${TARGETDIR}/hadoop-${VERSION}/share/hadoop/tools/lib/${jarname}" ]]; then + { + echo " if [[ -f \"\${HADOOP_TOOLS_HOME}/\${HADOOP_TOOLS_LIB_JARS_DIR}/${jarname}\" ]]; then" + echo " hadoop_add_classpath \"\${HADOOP_TOOLS_HOME}/\${HADOOP_TOOLS_LIB_JARS_DIR}/${jarname}\"" + echo " fi" + } >> "${fn}" + + elif [[ -f "${TARGETDIR}/hadoop-${VERSION}/share/hadoop/common/${jarname}" + || -f "${TARGETDIR}/hadoop-${VERSION}/share/hadoop/common/lib/${jarname}" ]]; then + true + else + echo "ERROR: ${module} has missing dependencies: ${jarname}" + fi + done < <(grep compile "${depfn}") + + { + echo " hadoop_add_classpath \"\${HADOOP_TOOLS_HOME}/\${HADOOP_TOOLS_LIB_JARS_DIR}/${module}-${VERSION}.jar\"" + echo "}" + echo "" + } >> "${fn}" +} + +function document_optionals +{ + echo "Rewriting ${TARGETDIR}/hadoop-${VERSION}/etc/hadoop/hadoop-env.sh" + sed -e "s^@@@HADOOP_OPTIONAL_TOOLS@@@^${OPTMODS}^" \ + "${TARGETDIR}/hadoop-${VERSION}/etc/hadoop/hadoop-env.sh" \ + > "${TARGETDIR}/hadoop-${VERSION}/etc/hadoop/hadoop-env.sh.new" + mv "${TARGETDIR}/hadoop-${VERSION}/etc/hadoop/hadoop-env.sh.new" \ + "${TARGETDIR}/hadoop-${VERSION}/etc/hadoop/hadoop-env.sh" +} + +function process +{ + declare fn + declare basefn + declare modtype + declare module + declare newfile + declare newdir + + while read -r fn; do + basefn=${fn##*/} + module=$(echo "${basefn}" | cut -f1 -d.) + modtype=$(echo "${basefn}" | cut -f2 -d.) + modtype=${modtype##tools-} + + newfile=$(getfilename "${module}" "${modtype}") + newdir=$(dirname "${newfile}") + mkdir -p "${newdir}" + + if [[ -f "${newfile}" ]]; then + rm "${newfile}" + fi + + touch "${newfile}" + + header "${newfile}" "${module}" + + "${modtype}_prologue" "${newfile}" "${module}" + + dependencywork "${newfile}" "${module}" "${fn}" + + chmod a+rx "${newfile}" + + done < <(find "${TOOLSDIR}" -name '*.tools-builtin.txt' -o -name '*.tools-optional.txt') + + document_optionals +} + +process diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop b/hadoop-common-project/hadoop-common/src/main/bin/hadoop index 766becd50f6..46eaf273687 100755 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop @@ -114,7 +114,7 @@ case ${COMMAND} in ;; archive) CLASS=org.apache.hadoop.tools.HadoopArchives - hadoop_add_to_classpath_toolspath + hadoop_add_to_classpath_tools hadoop-archives ;; checknative) CLASS=org.apache.hadoop.util.NativeLibraryChecker @@ -133,11 +133,11 @@ case ${COMMAND} in ;; distch) CLASS=org.apache.hadoop.tools.DistCh - hadoop_add_to_classpath_toolspath + hadoop_add_to_classpath_tools hadoop-extras ;; distcp) CLASS=org.apache.hadoop.tools.DistCp - hadoop_add_to_classpath_toolspath + hadoop_add_to_classpath_tools hadoop-distcp ;; envvars) echo "JAVA_HOME='${JAVA_HOME}'" @@ -146,7 +146,9 @@ case ${COMMAND} in echo "HADOOP_COMMON_LIB_JARS_DIR='${HADOOP_COMMON_LIB_JARS_DIR}'" echo "HADOOP_COMMON_LIB_NATIVE_DIR='${HADOOP_COMMON_LIB_NATIVE_DIR}'" echo "HADOOP_CONF_DIR='${HADOOP_CONF_DIR}'" - echo "HADOOP_TOOLS_PATH='${HADOOP_TOOLS_PATH}'" + echo "HADOOP_TOOLS_HOME='${HADOOP_TOOLS_HOME}'" + echo "HADOOP_TOOLS_DIR='${HADOOP_TOOLS_DIR}'" + echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'" exit 0 ;; fs) diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh index fa52e226902..7f293b64754 100755 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh @@ -41,6 +41,44 @@ function hadoop_debug fi } +## @description Given variable $1 delete $2 from it +## @audience public +## @stability stable +## @replaceable no +function hadoop_delete_entry +{ + if [[ ${!1} =~ \ ${2}\ ]] ; then + hadoop_debug "Removing ${2} from ${1}" + eval "${1}"=\""${!1// ${2} }"\" + fi +} + +## @description Given variable $1 add $2 to it +## @audience public +## @stability stable +## @replaceable no +function hadoop_add_entry +{ + if [[ ! ${!1} =~ \ ${2}\ ]] ; then + hadoop_debug "Adding ${2} to ${1}" + #shellcheck disable=SC2140 + eval "${1}"=\""${!1} ${2} "\" + fi +} + +## @description Given variable $1 determine if $2 is in it +## @audience public +## @stability stable +## @replaceable no +## @return 0 = yes, 1 = no +function hadoop_verify_entry +{ + # this unfortunately can't really be tested by bats. :( + # so if this changes, be aware that unit tests effectively + # do this function in them + [[ ${!1} =~ \ ${2}\ ]] +} + ## @description Add a subcommand to the usage output ## @audience private ## @stability evolving @@ -264,10 +302,9 @@ function hadoop_bootstrap YARN_LIB_JARS_DIR=${YARN_LIB_JARS_DIR:-"share/hadoop/yarn/lib"} MAPRED_DIR=${MAPRED_DIR:-"share/hadoop/mapreduce"} MAPRED_LIB_JARS_DIR=${MAPRED_LIB_JARS_DIR:-"share/hadoop/mapreduce/lib"} - - # setup a default HADOOP_TOOLS_PATH - hadoop_deprecate_envvar TOOL_PATH HADOOP_TOOLS_PATH - HADOOP_TOOLS_PATH=${HADOOP_TOOLS_PATH:-${HADOOP_PREFIX}/share/hadoop/tools/lib/*} + HADOOP_TOOLS_HOME=${HADOOP_TOOLS_HOME:-${HADOOP_PREFIX}} + HADOOP_TOOLS_DIR=${HADOOP_TOOLS_DIR:-"share/hadoop/tools"} + HADOOP_TOOLS_LIB_JARS_DIR=${HADOOP_TOOLS_LIB_JARS_DIR:-"${HADOOP_TOOLS_DIR}/lib"} # usage output set to zero hadoop_reset_usage @@ -322,6 +359,7 @@ function hadoop_exec_hadoopenv if [[ -z "${HADOOP_ENV_PROCESSED}" ]]; then if [[ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]]; then export HADOOP_ENV_PROCESSED=true + # shellcheck disable=SC1090 . "${HADOOP_CONF_DIR}/hadoop-env.sh" fi fi @@ -334,6 +372,7 @@ function hadoop_exec_hadoopenv function hadoop_exec_userfuncs { if [[ -e "${HADOOP_CONF_DIR}/hadoop-user-functions.sh" ]]; then + # shellcheck disable=SC1090 . "${HADOOP_CONF_DIR}/hadoop-user-functions.sh" fi } @@ -348,6 +387,7 @@ function hadoop_exec_hadooprc { if [[ -f "${HOME}/.hadooprc" ]]; then hadoop_debug "Applying the user's .hadooprc" + # shellcheck disable=SC1090 . "${HOME}/.hadooprc" fi } @@ -373,11 +413,22 @@ function hadoop_import_shellprofiles files2=(${HADOOP_CONF_DIR}/shellprofile.d/*.sh) fi + # enable bundled shellprofiles that come + # from hadoop-tools. This converts the user-facing HADOOP_OPTIONAL_TOOLS + # to the HADOOP_TOOLS_OPTIONS that the shell profiles expect. + # See dist-tools-hooks-maker for how the example HADOOP_OPTIONAL_TOOLS + # gets populated into hadoop-env.sh + + for i in ${HADOOP_OPTIONAL_TOOLS//,/ }; do + hadoop_add_entry HADOOP_TOOLS_OPTIONS "${i}" + done + for i in "${files1[@]}" "${files2[@]}" do if [[ -n "${i}" && -f "${i}" ]]; then hadoop_debug "Profiles: importing ${i}" + # shellcheck disable=SC1090 . "${i}" fi done @@ -945,34 +996,25 @@ function hadoop_add_common_to_classpath hadoop_add_classpath "${HADOOP_COMMON_HOME}/${HADOOP_COMMON_DIR}"'/*' } -## @description Add the HADOOP_TOOLS_PATH to the classpath +## @description Run libexec/tools/module.sh to add to the classpath ## @description environment ## @audience private ## @stability evolving ## @replaceable yes -function hadoop_add_to_classpath_toolspath +## @param module +function hadoop_add_to_classpath_tools { - declare -a array - declare -i c=0 - declare -i j - declare -i i - declare idx + declare module=$1 - if [[ -n "${HADOOP_TOOLS_PATH}" ]]; then - hadoop_debug "Adding HADOOP_TOOLS_PATH to CLASSPATH" - oldifs=${IFS} - IFS=: - for idx in ${HADOOP_TOOLS_PATH}; do - array[${c}]=${idx} - ((c=c+1)) - done - IFS=${oldifs} - ((j=c-1)) || ${QATESTMODE} - - for ((i=0; i<=j; i++)); do - hadoop_add_classpath "${array[$i]}" after - done + if [[ -f "${HADOOP_LIBEXEC_DIR}/tools/${module}.sh" ]]; then + # shellcheck disable=SC1090 + . "${HADOOP_LIBEXEC_DIR}/tools/${module}.sh" + else + hadoop_error "ERROR: Tools helper ${HADOOP_LIBEXEC_DIR}/tools/${module}.sh was not found." + fi + if declare -f hadoop_classpath_tools_${module} >/dev/null 2>&1; then + "hadoop_classpath_tools_${module}" fi } diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-layout.sh.example b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-layout.sh.example old mode 100644 new mode 100755 index a3f7bde1bbc..faa431740c0 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-layout.sh.example +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-layout.sh.example @@ -1,3 +1,4 @@ +#!/usr/bin/env bash # Copyright 2014 The Apache Software Foundation # # Licensed to the Apache Software Foundation (ASF) under one @@ -87,7 +88,10 @@ # Misc paths #### -# setup a default HADOOP_TOOLS_PATH, where things like distcp lives +# This is where things like distcp, S3, and other things live # note that this path only gets added for certain commands and not -# part of the general classpath -# export HADOOP_TOOLS_PATH="${HADOOP_PREFIX}/share/hadoop/tools/lib/*" +# part of the general classpath unless HADOOP_OPTIONAL_TOOLS is used +# to configure them in +# export HADOOP_TOOLS_HOME=${HADOOP_PREFIX} +# export HADOOP_TOOLS_DIR=${HADOOP_TOOLS_DIR:-"share/hadoop/tools"} +# export HADOOP_TOOLS_LIB_JARS_DIR=${HADOOP_TOOLS_LIB_JARS_DIR:-"${HADOOP_TOOLS_DIR}/lib"} diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh index 04aad8b3562..3c554aa8d02 100644 --- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh +++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh @@ -49,7 +49,7 @@ # preferred. Many sites configure these options outside of Hadoop, # such as in /etc/profile.d -# The java implementation to use. By default, this environment +# The java implementation to use. By default, this environment # variable is REQUIRED on ALL platforms except OS X! # export JAVA_HOME= @@ -64,15 +64,15 @@ # path. # export HADOOP_CONF_DIR=$HADOOP_PREFIX/etc/hadoop -# The maximum amount of heap to use (Java -Xmx). If no unit -# is provided, it will be converted to MB. Daemons will +# The maximum amount of heap to use (Java -Xmx). If no unit +# is provided, it will be converted to MB. Daemons will # prefer any Xmx setting in their respective _OPT variable. # There is no default; the JVM will autoscale based upon machine # memory size. # export HADOOP_HEAPSIZE_MAX= -# The minimum amount of heap to use (Java -Xms). If no unit -# is provided, it will be converted to MB. Daemons will +# The minimum amount of heap to use (Java -Xms). If no unit +# is provided, it will be converted to MB. Daemons will # prefer any Xms setting in their respective _OPT variable. # There is no default; the JVM will autoscale based upon machine # memory size. @@ -107,8 +107,8 @@ case ${HADOOP_OS_TYPE} in esac # Extra Java runtime options for some Hadoop commands -# and clients (i.e., hdfs dfs -blah). These get appended to HADOOP_OPTS for -# such commands. In most cases, # this should be left empty and +# and clients (i.e., hdfs dfs -blah). These get appended to HADOOP_OPTS for +# such commands. In most cases, # this should be left empty and # let users supply it on the command line. # export HADOOP_CLIENT_OPTS="" @@ -146,6 +146,11 @@ esac # names starting with a '-' are treated as negative matches. For example, # export HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES="-org.apache.hadoop.UserClass,java.,javax.,org.apache.hadoop." +# Enable optional, bundled Hadoop features +# This is a comma delimited list. It may NOT be overridden via .hadooprc +# Entries may be added/removed as needed. +# export HADOOP_OPTIONAL_TOOLS="@@@HADOOP_OPTIONAL_TOOLS@@@" + ### # Options for remote shell connectivity ### @@ -181,7 +186,7 @@ esac # non-secure) # -# Where (primarily) daemon log files are stored. # $HADOOP_PREFIX/logs +# Where (primarily) daemon log files are stored. # $HADOOP_PREFIX/logs # by default. # Java property: hadoop.log.dir # export HADOOP_LOG_DIR=${HADOOP_PREFIX}/logs @@ -201,7 +206,7 @@ esac # Java property: hadoop.root.logger # export HADOOP_ROOT_LOGGER=INFO,console -# Default log4j setting for daemons spawned explicitly by +# Default log4j setting for daemons spawned explicitly by # --daemon option of hadoop, hdfs, mapred and yarn command. # Java property: hadoop.root.logger # export HADOOP_DAEMON_ROOT_LOGGER=INFO,RFA diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_add_to_classpath_tools.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_add_to_classpath_tools.bats new file mode 100644 index 00000000000..df0ff01e07a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_add_to_classpath_tools.bats @@ -0,0 +1,47 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +load hadoop-functions_test_helper + +toolsetup () { + HADOOP_LIBEXEC_DIR="${TMP}/libexec" + mkdir -p "${HADOOP_LIBEXEC_DIR}/tools" +} + +@test "hadoop_classpath_tools (load)" { + toolsetup + echo "unittest=libexec" > "${HADOOP_LIBEXEC_DIR}/tools/test.sh" + hadoop_add_to_classpath_tools test + [ -n "${unittest}" ] +} + + +@test "hadoop_classpath_tools (not exist)" { + toolsetup + hadoop_add_to_classpath_tools test + [ -z "${unittest}" ] +} + +@test "hadoop_classpath_tools (function)" { + toolsetup + { + echo "function hadoop_classpath_tools_test {" + echo " unittest=libexec" + echo " }" + } > "${HADOOP_LIBEXEC_DIR}/tools/test.sh" + hadoop_add_to_classpath_tools test + declare -f + [ -n "${unittest}" ] +} diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_add_to_classpath_toolspath.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_add_to_classpath_toolspath.bats deleted file mode 100644 index a8a9bca8f7a..00000000000 --- a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_add_to_classpath_toolspath.bats +++ /dev/null @@ -1,74 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -load hadoop-functions_test_helper - -freetheclasses () { - local j - - for j in HADOOP_TOOLS_PATH \ - CLASSPATH; do - unset ${j} - done -} - -createdirs () { - local j - - for j in new old foo bar baz; do - mkdir -p "${TMP}/${j}" - done -} - -@test "hadoop_add_to_classpath_toolspath (nothing)" { - freetheclasses - hadoop_add_to_classpath_toolspath - [ -z "${CLASSPATH}" ] -} - -@test "hadoop_add_to_classpath_toolspath (none)" { - freetheclasses - CLASSPATH=test - hadoop_add_to_classpath_toolspath - [ "${CLASSPATH}" = "test" ] -} - -@test "hadoop_add_to_classpath_toolspath (only)" { - freetheclasses - createdirs - HADOOP_TOOLS_PATH="${TMP}/new" - hadoop_add_to_classpath_toolspath - [ "${CLASSPATH}" = "${TMP}/new" ] -} - -@test "hadoop_add_to_classpath_toolspath (1+1)" { - freetheclasses - createdirs - CLASSPATH=${TMP}/foo - HADOOP_TOOLS_PATH=${TMP}/foo - hadoop_add_to_classpath_toolspath - echo ">${CLASSPATH}<" - [ ${CLASSPATH} = "${TMP}/foo" ] -} - -@test "hadoop_add_to_classpath_toolspath (3+2)" { - freetheclasses - createdirs - CLASSPATH=${TMP}/foo:${TMP}/bar:${TMP}/baz - HADOOP_TOOLS_PATH=${TMP}/new:${TMP}/old - hadoop_add_to_classpath_toolspath - echo ">${CLASSPATH}<" - [ ${CLASSPATH} = "${TMP}/foo:${TMP}/bar:${TMP}/baz:${TMP}/new:${TMP}/old" ] -} diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_basic_init.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_basic_init.bats index 6a5e3f270c7..ae20248dadf 100644 --- a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_basic_init.bats +++ b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_basic_init.bats @@ -38,7 +38,8 @@ basicinitsetup () { dirvars="HADOOP_COMMON_HOME \ HADOOP_MAPRED_HOME \ HADOOP_HDFS_HOME \ - HADOOP_YARN_HOME" + HADOOP_YARN_HOME \ + HADOOP_TOOLS_HOME" for j in ${testvars}; do unset ${j} diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_bootstrap.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_bootstrap.bats index 0fd5d214121..9114c707336 100644 --- a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_bootstrap.bats +++ b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_bootstrap.bats @@ -15,13 +15,13 @@ load hadoop-functions_test_helper -@test "hadoop_deprecate_envvar (no libexec)" { +@test "hadoop_bootstrap (no libexec)" { unset HADOOP_LIBEXEC_DIR run hadoop_bootstrap [ "${status}" -eq 1 ] } -@test "hadoop_deprecate_envvar (libexec)" { +@test "hadoop_bootstrap (libexec)" { unset HADOOP_PREFIX unset HADOOP_COMMON_DIR unset HADOOP_COMMON_LIB_JARS_DIR @@ -31,7 +31,9 @@ load hadoop-functions_test_helper unset YARN_LIB_JARS_DIR unset MAPRED_DIR unset MAPRED_LIB_JARS_DIR - unset TOOL_PATH + unset HADOOP_TOOLS_HOME + unset HADOOP_TOOLS_DIR + unset HADOOP_TOOLS_LIB_JARS_DIR unset HADOOP_OS_TYPE hadoop_bootstrap @@ -46,6 +48,9 @@ load hadoop-functions_test_helper [ -n ${YARN_LIB_JARS_DIR} ] [ -n ${MAPRED_DIR} ] [ -n ${MAPRED_LIB_JARS_DIR} ] - [ -n ${TOOL_PATH} ] [ -n ${HADOOP_OS_TYPE} ] -} + [ -n ${HADOOP_TOOLS_PATH} ] + [ -n ${HADOOP_TOOLS_HOME} ] + [ -n ${HADOOP_TOOLS_DIR} ] + [ -n ${HADOOP_TOOLS_LIB_JARS_DIR} ] +} diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_entry_tests.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_entry_tests.bats new file mode 100644 index 00000000000..20be3e546c3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_entry_tests.bats @@ -0,0 +1,49 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +load hadoop-functions_test_helper + +@test "hadoop_add_entry (positive 1)" { + hadoop_add_entry testvar foo + echo ">${testvar}<" + [ "${testvar}" = " foo " ] +} + +@test "hadoop_add_entry (negative)" { + hadoop_add_entry testvar foo + hadoop_add_entry testvar foo + echo ">${testvar}<" + [ "${testvar}" = " foo " ] +} + +@test "hadoop_add_entry (positive 2)" { + hadoop_add_entry testvar foo + hadoop_add_entry testvar foo + hadoop_add_entry testvar bar + echo ">${testvar}<" + [ "${testvar}" = " foo bar " ] +} + +@test "hadoop_add_entry (positive 3)" { + hadoop_add_entry testvar foo + hadoop_add_entry testvar foo + hadoop_add_entry testvar bar + hadoop_add_entry testvar bar + hadoop_add_entry testvar baz + hadoop_add_entry testvar baz + + echo ">${testvar}<" + [ "${testvar}" = " foo bar baz " ] +} diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shellprofile.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shellprofile.bats index d6e0a2563e0..305067b9c7c 100644 --- a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shellprofile.bats +++ b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shellprofile.bats @@ -66,6 +66,13 @@ _test_hadoop_finalize () { [ -z "${unittest}" ] } +@test "hadoop_import_shellprofiles (H_O_T)" { + HADOOP_OPTIONAL_TOOLS=1,2 + shellprofilesetup + hadoop_import_shellprofiles + [ "${HADOOP_TOOLS_OPTIONS}" == " 1 2 " ] +} + @test "hadoop_add_profile+hadoop_shellprofiles_init" { hadoop_add_profile test hadoop_shellprofiles_init diff --git a/hadoop-dist/pom.xml b/hadoop-dist/pom.xml index 42e74cbe7c4..8323bc64ace 100644 --- a/hadoop-dist/pom.xml +++ b/hadoop-dist/pom.xml @@ -103,6 +103,24 @@ + + toolshooks + prepare-package + + exec + + + ${shell-executable} + ${basedir} + false + + ${basedir}/../dev-support/bin/dist-tools-hooks-maker + ${project.version} + ${project.build.directory} + ${basedir}/../hadoop-tools + + + tar package diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs index a6644d1af3e..bcd04d16085 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs @@ -141,7 +141,9 @@ case ${COMMAND} in echo "HDFS_DIR='${HDFS_DIR}'" echo "HDFS_LIB_JARS_DIR='${HDFS_LIB_JARS_DIR}'" echo "HADOOP_CONF_DIR='${HADOOP_CONF_DIR}'" - echo "HADOOP_TOOLS_PATH='${HADOOP_TOOLS_PATH}'" + echo "HADOOP_TOOLS_HOME='${HADOOP_TOOLS_HOME}'" + echo "HADOOP_TOOLS_DIR='${HADOOP_TOOLS_DIR}'" + echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'" exit 0 ;; erasurecode) @@ -165,7 +167,6 @@ case ${COMMAND} in ;; haadmin) CLASS=org.apache.hadoop.hdfs.tools.DFSHAAdmin - hadoop_add_to_classpath_toolspath hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS" HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}" ;; diff --git a/hadoop-mapreduce-project/bin/mapred b/hadoop-mapreduce-project/bin/mapred index 66a19aee31c..fab5b87f2e0 100755 --- a/hadoop-mapreduce-project/bin/mapred +++ b/hadoop-mapreduce-project/bin/mapred @@ -69,13 +69,13 @@ case ${COMMAND} in ;; archive) CLASS=org.apache.hadoop.tools.HadoopArchives - hadoop_add_to_classpath_toolspath + hadoop_add_to_classpath_tools hadoop-archives hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS" HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}" ;; archive-logs) CLASS=org.apache.hadoop.tools.HadoopArchiveLogs - hadoop_add_to_classpath_toolspath + hadoop_add_to_classpath_tools hadoop-archive-logs hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS" HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}" ;; @@ -84,7 +84,7 @@ case ${COMMAND} in ;; distcp) CLASS=org.apache.hadoop.tools.DistCp - hadoop_add_to_classpath_toolspath + hadoop_add_to_classpath_tools hadoop-distcp hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS" HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}" ;; @@ -94,7 +94,9 @@ case ${COMMAND} in echo "MAPRED_DIR='${MAPRED_DIR}'" echo "MAPRED_LIB_JARS_DIR='${MAPRED_LIB_JARS_DIR}'" echo "HADOOP_CONF_DIR='${HADOOP_CONF_DIR}'" - echo "HADOOP_TOOLS_PATH='${HADOOP_TOOLS_PATH}'" + echo "HADOOP_TOOLS_HOME='${HADOOP_TOOLS_HOME}'" + echo "HADOOP_TOOLS_DIR='${HADOOP_TOOLS_DIR}'" + echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'" exit 0 ;; historyserver) diff --git a/hadoop-tools/hadoop-archive-logs/pom.xml b/hadoop-tools/hadoop-archive-logs/pom.xml index 7e7da77efa3..f3fc9880b18 100644 --- a/hadoop-tools/hadoop-archive-logs/pom.xml +++ b/hadoop-tools/hadoop-archive-logs/pom.xml @@ -172,6 +172,23 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + org.codehaus.mojo findbugs-maven-plugin diff --git a/hadoop-tools/hadoop-archives/pom.xml b/hadoop-tools/hadoop-archives/pom.xml index f04d7fc252c..cc98e632206 100644 --- a/hadoop-tools/hadoop-archives/pom.xml +++ b/hadoop-tools/hadoop-archives/pom.xml @@ -128,6 +128,23 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index eb87d77b7cc..f4228460eae 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -90,6 +90,23 @@ 3600 + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-optional.txt + + + + diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 5e6eb95a650..af3541fafb1 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -19,6 +19,9 @@ JAR file, `hadoop-aws.jar` also declares a transitive dependency on all external artifacts which are needed for this support —enabling downstream applications to easily use this support. +To make it part of Apache Hadoop's default classpath, simply make sure that +HADOOP_OPTIONAL_TOOLS in hadoop-env.sh has 'hadoop-aws' in the list. + Features 1. The "classic" `s3:` filesystem for storing objects in Amazon S3 Storage @@ -30,7 +33,7 @@ higher performance. The specifics of using these filesystems are documented below. -## Warning: Object Stores are not filesystems. +## Warning #1: Object Stores are not filesystems. Amazon S3 is an example of "an object store". In order to achieve scalability and especially high availability, S3 has —as many other cloud object stores have diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index e9b3af775f2..7d3aa5fff7c 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -82,6 +82,24 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-optional.txt + + + + + diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/index.md b/hadoop-tools/hadoop-azure/src/site/markdown/index.md index a4a761521d5..43c551c4d5b 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/index.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/index.md @@ -34,6 +34,9 @@ The built jar file, named hadoop-azure.jar, also declares transitive dependencie on the additional artifacts it requires, notably the [Azure Storage SDK for Java](https://github.com/Azure/azure-storage-java). +To make it part of Apache Hadoop's default classpath, simply make sure that +HADOOP_OPTIONAL_TOOLS in hadoop-env.sh has 'hadoop-azure' in the list. + ## Features * Read and write data stored in an Azure Blob Storage account. diff --git a/hadoop-tools/hadoop-datajoin/pom.xml b/hadoop-tools/hadoop-datajoin/pom.xml index 946952189b6..40445cb9ab0 100644 --- a/hadoop-tools/hadoop-datajoin/pom.xml +++ b/hadoop-tools/hadoop-datajoin/pom.xml @@ -132,6 +132,22 @@ org.apache.maven.plugins maven-jar-plugin + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + ${project.basedir}/../../hadoop-dist/target/hadoop-tools-deps/${project.artifactId}.txt + + + + diff --git a/hadoop-tools/hadoop-distcp/pom.xml b/hadoop-tools/hadoop-distcp/pom.xml index 7099cec9b2c..b64a85839c3 100644 --- a/hadoop-tools/hadoop-distcp/pom.xml +++ b/hadoop-tools/hadoop-distcp/pom.xml @@ -163,6 +163,17 @@ ${project.build.directory}/lib + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + diff --git a/hadoop-tools/hadoop-extras/pom.xml b/hadoop-tools/hadoop-extras/pom.xml index 5f758758f10..f8851efae09 100644 --- a/hadoop-tools/hadoop-extras/pom.xml +++ b/hadoop-tools/hadoop-extras/pom.xml @@ -137,6 +137,23 @@ org.apache.maven.plugins maven-jar-plugin + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + diff --git a/hadoop-tools/hadoop-gridmix/pom.xml b/hadoop-tools/hadoop-gridmix/pom.xml index db84f8fa371..33b5488f072 100644 --- a/hadoop-tools/hadoop-gridmix/pom.xml +++ b/hadoop-tools/hadoop-gridmix/pom.xml @@ -163,6 +163,23 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + diff --git a/hadoop-tools/hadoop-kafka/pom.xml b/hadoop-tools/hadoop-kafka/pom.xml index c0667ee3391..f000682c5c3 100644 --- a/hadoop-tools/hadoop-kafka/pom.xml +++ b/hadoop-tools/hadoop-kafka/pom.xml @@ -88,6 +88,23 @@ 3600 + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-optional.txt + + + + diff --git a/hadoop-tools/hadoop-openstack/pom.xml b/hadoop-tools/hadoop-openstack/pom.xml index 542a52367f8..20a52fa2704 100644 --- a/hadoop-tools/hadoop-openstack/pom.xml +++ b/hadoop-tools/hadoop-openstack/pom.xml @@ -85,6 +85,23 @@ false + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-optional.txt + + + + diff --git a/hadoop-tools/hadoop-openstack/src/site/markdown/index.md b/hadoop-tools/hadoop-openstack/src/site/markdown/index.md index 0eeb2747278..1815f60c613 100644 --- a/hadoop-tools/hadoop-openstack/src/site/markdown/index.md +++ b/hadoop-tools/hadoop-openstack/src/site/markdown/index.md @@ -54,6 +54,9 @@ Introduction This module enables Apache Hadoop applications -including MapReduce jobs, read and write data to and from instances of the [OpenStack Swift object store](http://www.openstack.org/software/openstack-storage/). +To make it part of Apache Hadoop's default classpath, simply make sure that +HADOOP_OPTIONAL_TOOLS in hadoop-env.sh has 'hadoop-openstack' in the list. + Features -------- @@ -440,7 +443,9 @@ If the host is declared, the proxy port must be set to a valid integer value. The `hadoop-openstack` JAR -or any dependencies- may not be on your classpath. -If it is a remote MapReduce job that is failing, make sure that the JAR is installed on the servers in the cluster -or that the job submission process uploads the JAR file to the distributed cache. +Make sure that the: +* JAR is installed on the servers in the cluster. +* 'hadoop-openstack' is on the HADOOP_OPTIONAL_TOOLS entry in hadoop-env.sh or that the job submission process uploads the JAR file to the distributed cache. #### Failure to Authenticate diff --git a/hadoop-tools/hadoop-rumen/pom.xml b/hadoop-tools/hadoop-rumen/pom.xml index 6828b1c1f1b..d11cd170287 100644 --- a/hadoop-tools/hadoop-rumen/pom.xml +++ b/hadoop-tools/hadoop-rumen/pom.xml @@ -132,6 +132,23 @@ org.apache.maven.plugins maven-jar-plugin + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + diff --git a/hadoop-tools/hadoop-sls/pom.xml b/hadoop-tools/hadoop-sls/pom.xml index 0539a0885e0..6d199bd492e 100644 --- a/hadoop-tools/hadoop-sls/pom.xml +++ b/hadoop-tools/hadoop-sls/pom.xml @@ -177,6 +177,23 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + diff --git a/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh b/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh index c5b6d844bca..f9bfaef2db0 100644 --- a/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh +++ b/hadoop-tools/hadoop-sls/src/main/bin/rumen2sls.sh @@ -55,7 +55,7 @@ function parse_args() function calculate_classpath() { - hadoop_add_to_classpath_toolspath + hadoop_add_to_classpath_tools hadoop-rumen } function run_sls_generator() diff --git a/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh b/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh index 29b0de743a5..30fd60a4440 100644 --- a/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh +++ b/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh @@ -70,7 +70,7 @@ function parse_args() function calculate_classpath { - hadoop_add_to_classpath_toolspath + hadoop_add_to_classpath_tools hadoop-sls hadoop_debug "Injecting ${HADOOP_PREFIX}/share/hadoop/tools/sls/html into CLASSPATH" hadoop_add_classpath "${HADOOP_PREFIX}/share/hadoop/tools/sls/html" } diff --git a/hadoop-tools/hadoop-streaming/pom.xml b/hadoop-tools/hadoop-streaming/pom.xml index 5ea10f63182..b16e1326954 100644 --- a/hadoop-tools/hadoop-streaming/pom.xml +++ b/hadoop-tools/hadoop-streaming/pom.xml @@ -174,6 +174,23 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn index da4ce4042f4..cb2364b3957 100755 --- a/hadoop-yarn-project/hadoop-yarn/bin/yarn +++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn @@ -103,7 +103,9 @@ case "${COMMAND}" in echo "YARN_DIR='${YARN_DIR}'" echo "YARN_LIB_JARS_DIR='${YARN_LIB_JARS_DIR}'" echo "HADOOP_CONF_DIR='${HADOOP_CONF_DIR}'" - echo "HADOOP_TOOLS_PATH='${HADOOP_TOOLS_PATH}'" + echo "HADOOP_TOOLS_HOME='${HADOOP_TOOLS_HOME}'" + echo "HADOOP_TOOLS_DIR='${HADOOP_TOOLS_DIR}'" + echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'" exit 0 ;; jar) From 938222b2e20791d6eb410e36f8fec3e1520e1078 Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Wed, 23 Mar 2016 14:57:14 -0700 Subject: [PATCH 19/43] HADOOP-12947. Update documentation Hadoop Groups Mapping to add static group mapping, negative cache. Contributed by Wei-Chiu Chuang. --- .../src/site/markdown/GroupsMapping.md | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/GroupsMapping.md b/hadoop-common-project/hadoop-common/src/site/markdown/GroupsMapping.md index 5a67bd12e0b..a7420291a17 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/GroupsMapping.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/GroupsMapping.md @@ -15,14 +15,7 @@ Hadoop Groups Mapping =================== -* [Hadoop Groups Mapping](#Hadoop_Groups_Mapping) - * [Overview](#Overview) - * [LDAP Groups Mapping](#LDAP_Groups_Mapping) - * [Active Directory](#Active_Directory) - * [POSIX Groups](#POSIX_Groups) - * [SSL](#SSL) - * [Composite Groups Mapping](#Composite_Groups_Mapping) - * [Multiple group mapping providers configuration sample](#Multiple_group_mapping_providers_configuration_sample) + Overview -------- @@ -61,6 +54,17 @@ For HDFS, the mapping of users to groups is performed on the NameNode. Thus, the Note that HDFS stores the user and group of a file or directory as strings; there is no conversion from user and group identity numbers as is conventional in Unix. +Static Mapping +-------- +It is possible to statically map users to groups by defining the mapping in `hadoop.user.group.static.mapping.overrides` in the format `user1=group1,group2;user2=;user3=group2`. +This property overrides any group mapping service provider. If a user's groups are defined in it, the groups are returned without more lookups; otherwise, the service provider defined in `hadoop.security.group.mapping` is used to look up the groups. By default, `dr.who=;` is defined, so the fake user dr.who will not have any groups. + +Caching/Negative caching +-------- +Since the group mapping resolution relies on external mechanisms, the NameNode performance may be impacted. To reduce the impact due to repeated lookups, Hadoop caches the groups returned by the service provider. The cache invalidate is configurable via `hadoop.security.groups.cache.secs`, and the default is 300 seconds. + +To avoid spamming NameNode with unknown users, Hadoop employs negative caching so that if the result of the lookup is empty, return an empty group directly instead of performing more group mapping queries, +The cache invalidation is configurable via `hadoop.security.groups.negative-cache.secs`. The default is 30 seconds, so if group mapping service providers returns no group for a user, no lookup will be performed for the same user within 30 seconds. LDAP Groups Mapping -------- @@ -85,9 +89,9 @@ in order to be considered a member. The default configuration supports LDAP group name resolution with an Active Directory server. ### POSIX Groups ### -If the LDAP server supports POSIX group semantics, Hadoop can perform LDAP group resolution queries to the server by setting both -`hadoop.security.group.mapping.ldap.search.filter.user` to `posixAccount` and -`hadoop.security.group.mapping.ldap.search.filter.group` to `posixGroup`. +If the LDAP server supports POSIX group semantics (RFC-2307), Hadoop can perform LDAP group resolution queries to the server by setting both +`hadoop.security.group.mapping.ldap.search.filter.user` to `(&(objectClass=posixAccount)(uid={0}))` and +`hadoop.security.group.mapping.ldap.search.filter.group` to `(objectClass=posixGroup)`. ### SSL ### To secure the connection, the implementation supports LDAP over SSL (LDAPS). SSL is enable by setting `hadoop.security.group.mapping.ldap.ssl` to `true`. From 19b645c93801a53d4486f9a7639186525e51f723 Mon Sep 17 00:00:00 2001 From: Junping Du Date: Wed, 23 Mar 2016 19:34:30 -0700 Subject: [PATCH 20/43] YARN-4820. ResourceManager web redirects in HA mode drops query parameters. Contributed by Varun Vasudev. --- .../hadoop/yarn/client/TestRMFailover.java | 7 ++-- .../webapp/RMWebAppFilter.java | 40 ++++++++++++++----- 2 files changed, 35 insertions(+), 12 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java index f32335127d1..b58a7751930 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java @@ -273,8 +273,8 @@ public class TestRMFailover extends ClientBaseWithFixes { redirectURL = getRedirectURL(rm2Url + "/metrics"); assertEquals(redirectURL,rm1Url + "/metrics"); - redirectURL = getRedirectURL(rm2Url + "/jmx"); - assertEquals(redirectURL,rm1Url + "/jmx"); + redirectURL = getRedirectURL(rm2Url + "/jmx?param1=value1+x¶m2=y"); + assertEquals(rm1Url + "/jmx?param1=value1+x¶m2=y", redirectURL); // standby RM links /conf, /stacks, /logLevel, /static, /logs, // /cluster/cluster as well as webService @@ -327,8 +327,9 @@ public class TestRMFailover extends ClientBaseWithFixes { // do not automatically follow the redirection // otherwise we get too many redirections exception conn.setInstanceFollowRedirects(false); - if(conn.getResponseCode() == HttpServletResponse.SC_TEMPORARY_REDIRECT) + if(conn.getResponseCode() == HttpServletResponse.SC_TEMPORARY_REDIRECT) { redirectUrl = conn.getHeaderField("Location"); + } } catch (Exception e) { // throw new RuntimeException(e); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java index 74137480b43..de2a23f786f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java @@ -25,6 +25,8 @@ import java.io.PrintWriter; import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; +import java.nio.charset.Charset; +import java.util.List; import java.util.Random; import java.util.Set; @@ -48,6 +50,8 @@ import org.apache.hadoop.yarn.util.Apps; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.util.WebAppUtils; +import org.apache.http.NameValuePair; +import org.apache.http.client.utils.URLEncodedUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -106,22 +110,40 @@ public class RMWebAppFilter extends GuiceContainer { HttpServletResponse response, FilterChain chain) throws IOException, ServletException { response.setCharacterEncoding("UTF-8"); - String uri = HtmlQuoting.quoteHtmlChars(request.getRequestURI()); + String htmlEscapedUri = HtmlQuoting.quoteHtmlChars(request.getRequestURI()); - if (uri == null) { - uri = "/"; + if (htmlEscapedUri == null) { + htmlEscapedUri = "/"; } + + String uriWithQueryString = htmlEscapedUri; + String htmlEscapedUriWithQueryString = htmlEscapedUri; + + String queryString = request.getQueryString(); + if (queryString != null && !queryString.isEmpty()) { + String reqEncoding = request.getCharacterEncoding(); + if (reqEncoding == null || reqEncoding.isEmpty()) { + reqEncoding = "ISO-8859-1"; + } + Charset encoding = Charset.forName(reqEncoding); + List params = URLEncodedUtils.parse(queryString, encoding); + String urlEncodedQueryString = URLEncodedUtils.format(params, encoding); + uriWithQueryString += "?" + urlEncodedQueryString; + htmlEscapedUriWithQueryString = HtmlQuoting.quoteHtmlChars( + request.getRequestURI() + "?" + urlEncodedQueryString); + } + RMWebApp rmWebApp = injector.getInstance(RMWebApp.class); rmWebApp.checkIfStandbyRM(); if (rmWebApp.isStandby() - && shouldRedirect(rmWebApp, uri)) { + && shouldRedirect(rmWebApp, htmlEscapedUri)) { String redirectPath = rmWebApp.getRedirectPath(); if (redirectPath != null && !redirectPath.isEmpty()) { - redirectPath += uri; - String redirectMsg = - "This is standby RM. The redirect url is: " + redirectPath; + redirectPath += uriWithQueryString; + String redirectMsg = "This is standby RM. The redirect url is: " + + htmlEscapedUriWithQueryString; PrintWriter out = response.getWriter(); out.println(redirectMsg); response.setHeader("Location", redirectPath); @@ -142,7 +164,7 @@ public class RMWebAppFilter extends GuiceContainer { int next = calculateExponentialTime(retryInterval); String redirectUrl = - appendOrReplaceParamter(path + uri, + appendOrReplaceParamter(path + uriWithQueryString, YarnWebParams.NEXT_REFRESH_INTERVAL + "=" + (retryInterval + 1)); if (redirectUrl == null || next > MAX_SLEEP_TIME) { doRetry = false; @@ -161,7 +183,7 @@ public class RMWebAppFilter extends GuiceContainer { } return; } else if (ahsEnabled) { - String ahsRedirectUrl = ahsRedirectPath(uri, rmWebApp); + String ahsRedirectUrl = ahsRedirectPath(uriWithQueryString, rmWebApp); if(ahsRedirectUrl != null) { response.setHeader("Location", ahsRedirectUrl); response.setStatus(HttpServletResponse.SC_TEMPORARY_REDIRECT); From b1394d6307425a41d388c71e39f0880babb2c7a9 Mon Sep 17 00:00:00 2001 From: Allen Wittenauer Date: Thu, 24 Mar 2016 08:15:58 -0700 Subject: [PATCH 21/43] YARN-4850. test-fair-scheduler.xml isn't valid xml (Yufei Gu via aw) --- .../test/resources/test-fair-scheduler.xml | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/test-fair-scheduler.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/test-fair-scheduler.xml index db160c9063c..f7934c4f171 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/test-fair-scheduler.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/test-fair-scheduler.xml @@ -1,21 +1,21 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ + From d82e797b652f248e238bdf1818e6b4a5b91cea7a Mon Sep 17 00:00:00 2001 From: Arun Suresh Date: Thu, 24 Mar 2016 09:59:55 -0700 Subject: [PATCH 22/43] YARN-4825. Remove redundant code in ClientRMService::listReservations. (subru via asuresh) --- .../yarn/server/resourcemanager/ClientRMService.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index ba1edf92ad6..b7eb5f16d8c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -1359,20 +1359,13 @@ public class ClientRMService extends AbstractService implements checkReservationACLs(requestInfo.getQueue(), AuditConstants.LIST_RESERVATION_REQUEST, reservationId); - ReservationId requestedId = null; - if (requestInfo.getReservationId() != null - && !requestInfo.getReservationId().isEmpty()) { - requestedId = ReservationId.parseReservationId(requestInfo - .getReservationId()); - } - long startTime = Math.max(requestInfo.getStartTime(), 0); long endTime = requestInfo.getEndTime() <= -1? Long.MAX_VALUE : requestInfo .getEndTime(); Set reservations; - reservations = plan.getReservations(requestedId, new ReservationInterval( + reservations = plan.getReservations(reservationId, new ReservationInterval( startTime, endTime)); List info = From d820975c1aadd74e79ed6c7da02a4124abcc1172 Mon Sep 17 00:00:00 2001 From: Arun Suresh Date: Thu, 24 Mar 2016 10:30:29 -0700 Subject: [PATCH 23/43] YARN-4683. Document the List Reservations REST API. (Contributed by Sean Po) --- .../src/site/markdown/ResourceManagerRest.md | 230 ++++++++++++++++++ 1 file changed, 230 insertions(+) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md index aed50a0dd10..338778b41a4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md @@ -33,6 +33,7 @@ ResourceManager REST API's. * [Cluster Application Queue API](#Cluster_Application_Queue_API) * [Cluster Application Priority API](#Cluster_Application_Priority_API) * [Cluster Delegation Tokens API](#Cluster_Delegation_Tokens_API) +* [Cluster Reservation API List](#Cluster_Reservation_API_List) Overview -------- @@ -3164,3 +3165,232 @@ Once setup, delegation tokens can be fetched using the web services listed above { "state":"KILLED" } + +Cluster Reservation API List +---------------------------- + +The Cluster Reservation API can be used to list reservations. When listing reservations the user must specify the constraints in terms of a queue, reservation-id, start time or end time. The user must also specify whether or not to include the full resource allocations of the reservations being listed. The resulting page returns a response containing information related to the reservation such as the acceptance time, the user, the resource allocations, the reservation-id, as well as the reservation definition. + +### URI + + * http:///ws/v1/cluster/reservation/list + +### HTTP Operations Supported + + * GET + +### Query Parameters Supported + + * queue - the queue name containing the reservations to be listed. if not set, this value will default to "default". + * reservation-id - the reservation-id of the reservation which will be listed. If this parameter is present, start-time and end-time will be ignored. + * start-time - reservations that end after this start-time will be listed. If unspecified or invalid, this will default to 0. + * end-time - reservations that start after this end-time will be listed. If unspecified or invalid, this will default to Long.MaxValue. + * include-resource-allocations - true or false. If true, the resource allocations of the reservation will be included in the response. If false, no resource allocations will be included in the response. This will default to false. + +### Elements of the *ReservationListInfo* object + +| Item | Data Type | Description | +|:---- |:---- |:---- | +| reservations | array of ReservationInfo(JSON) / zero or more ReservationInfo objects(XML) | The reservations that are listed with the given query | + +### Elements of the *reservations* object + +| Item | Data Type | Description | +|:---- |:---- |:---- | +| acceptance-time | long | Time that the reservation was accepted | +| resource-allocations | array of ResourceAllocationInfo(JSON) / zero or more ResourceAllocationInfo objects(XML) | Resource allocation information for the reservation | +| reservation-id | A single ReservationId object | The unique reservation identifier | +| reservation-definition | A single ReservationDefinition Object | A set of constraints representing the need for resources over time of a user | +| user | string | User who made the reservation | + +### Elements of the *resource-allocations* object + +| Item | Data Type | Description | +|:---- |:---- |:---- | +| resource | A single Resource object | The resources allocated for the reservation allocation | +| startTime | long | Start time that the resource is allocated for | +| endTime | long | End time that the resource is allocated for | + +### elements of the *resource* object + +| Item | Data Type | Description | +|:---- |:---- |:---- | +| memory | int | The memory allocated for the reservation allocation | +| vCores | int | The number of cores allocated for the reservation allocation | + +### Elements of the *reservation-id* object + +| Item | Data Type | Description | +|:---- |:---- |:---- | +| cluster-timestamp | long | Timestamp representing the time the reservation was created | +| reservation-id | long | The id of the reservation that was listed | + +Elements of the *reservation-definition* object + +| Item | Data Type | Description | +|:---- |:---- |:---- | +| arrival | long | The UTC time representation of the earliest time this reservation can be allocated from. | +| deadline | long | The UTC time representation of the latest time within which this reservatino can be allocated. | +| reservation-name | string | A mnemonic name of the reservaiton (not a valid identifier). | +| reservation-requests | object | A list of "stages" or phases of this reservation, each describing resource requirements and duration | + +Elements of the *reservation-requests* object + +| Item | Data Type | Description | +|:---- |:---- |:---- | +| reservation-request-interpreter | int | A numeric choice of how to interpret the set of ReservationRequest: 0 is an ANY, 1 for ALL, 2 for ORDER, 3 for ORDER\_NO\_GAP | +| reservation-request | object | The description of the resource and time capabilities for a phase/stage of this reservation | + +Elements of the *reservation-request* object + +| Item | Data Type | Description | +|:---- |:---- |:---- | +| duration | long | The duration of a ReservationRequest in milliseconds (amount of consecutive milliseconds a satisfiable allocation for this portion of the reservation should exist for). | +| num-containers | int | The number of containers required in this phase of the reservation (capture the maximum parallelism of the job(s) in this phase). | +| min-concurrency | int | The minimum number of containers that must be concurrently allocated to satisfy this allocation (capture min-parallelism, useful to express gang semantics). | +| capability | object | Allows to specify the size of each container (memory, vCores).| + +### GET Response Examples + +Get requests can be used to list reservations to the ResourceManager. As mentioned above, information pertaining to the reservation is returned upon success (in the body of the answer). Successful list requests result in a 200 response. Please note that in order to submit a reservation, you must have an authentication filter setup for the HTTP interface. the functionality requires that the username is set in the HttpServletRequest. If no filter is setup, the response will be an "UNAUTHORIZED" response. Please note that this feature is currently in the alpha stage and may change in the future. + +**JSON response** + +This request return all active reservations within the start time 1455159355000 and 1475160036000. Since include-resource-allocations is set to true, the full set of resource allocations will be included in the response. + +HTTP Request: + + GET http:///ws/v1/cluster/reservation/list?queue=dedicated&start-time=1455159355000&end-time=1475160036000&include-resource-allocations=true + +Response Header: + + HTTP/1.1 200 OK + Content-Type: application/json + Transfer-Encoding: chunked + Cache-Control: no-cache + Content-Encoding: gzip + Pragma: no-cache,no-cache + Server: Jetty(6.1.26) + +Response Body: + +```json +{ + "reservations": { + "acceptance-time": "1455160008442", + "user": "submitter", + "resource-allocations": [ + { + "resource": { + "memory": "0", + "vCores": "0" + }, + "startTime": "1465541532000", + "endTime": "1465542250000" + }, + { + "resource": { + "memory": "1024", + "vCores": "1" + }, + "startTime": "1465542250000", + "endTime": "1465542251000" + }, + { + "resource": { + "memory": "0", + "vCores": "0" + }, + "startTime": "1465542251000", + "endTime": "1465542252000" + } + ], + "reservation-id": { + "cluster-timestamp": "1455133859510", + "reservation-id": "6" + }, + "reservation-definition": { + "arrival": "1465541532000", + "deadline": "1465542252000", + "reservation-requests": { + "reservation-request-interpreter": "0", + "reservation-request": { + "capability": { + "memory": "1024", + "vCores": "1" + }, + "min-concurrency": "1", + "num-containers": "1", + "duration": "60" + } + }, + "reservation-name": "res_1" + } + } +} +``` + +**XML Response** + +HTTP Request: + + GET http:///ws/v1/cluster/reservation/list?queue=dedicated&start-time=1455159355000&end-time=1475160036000&include-resource-allocations=true + +Response Header: + + HTTP/1.1 200 OK + Content-Type: application/xml + Content-length: 395 + Cache-Control: no-cache + Content-Encoding: gzip + Pragma: no-cache,no-cache + Server: Jetty(6.1.26) + +Response Body: + +```xml + + + + 1455233661003 + dr.who + + + 0 + 0 + + 1465541532000 + 1465542251000 + + + + 1024 + 1 + + 1465542251000 + 1465542252000 + + + 1455228059846 + 1 + + + 1465541532000 + 1465542252000 + + 0 + + + 1024 + 1 + + 1 + 1 + 60 + + + res_1 + + + +``` From 2e1d0ff4e901b8313c8d71869735b94ed8bc40a0 Mon Sep 17 00:00:00 2001 From: Arun Suresh Date: Thu, 24 Mar 2016 10:40:51 -0700 Subject: [PATCH 24/43] YARN-4687. Document Reservation ACLs (Contributed by Sean Po) --- .../src/site/markdown/CapacityScheduler.md | 12 ++++++++++++ .../src/site/markdown/FairScheduler.md | 6 +++++- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md index b43a032a69a..e86c4f9065f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md @@ -229,6 +229,18 @@ The following configuration parameters can be configured in yarn-site.xml to con |:---- |:---- | | `yarn.scheduler.capacity..disable_preemption` | This configuration can be set to `true` to selectively disable preemption of application containers submitted to a given queue. This property applies only when system wide preemption is enabled by configuring `yarn.resourcemanager.scheduler.monitor.enable` to *true* and `yarn.resourcemanager.scheduler.monitor.policies` to *ProportionalCapacityPreemptionPolicy*. If this property is not set for a queue, then the property value is inherited from the queue's parent. Default value is false. +###Reservation Properties + + * Reservation Administration & Permissions + + The `CapacityScheduler` supports the following parameters to control the creation, deletion, update, and listing of reservations. Note that any user can update, delete, or list their own reservations. If reservation ACLs are enabled but not defined, everyone will have access. In the examples below, \ is the queue name. For example, to set the reservation ACL to administer reservations on the default queue, use the property `yarn.scheduler.capacity.root.default.acl_administer_reservations` + +| Property | Description | +|:---- |:---- | +| `yarn.scheduler.capacity.root..acl_administer_reservations` | The ACL which controls who can *administer* reservations to the given queue. If the given user/group has necessary ACLs on the given queue or they can submit, delete, update and list all reservations. ACLs for this property *are not* inherited from the parent queue if not specified. | +| `yarn.scheduler.capacity.root..acl_list_reservations` | The ACL which controls who can *list* reservations to the given queue. If the given user/group has necessary ACLs on the given queue they can list all applications. ACLs for this property *are not* inherited from the parent queue if not specified. | +| `yarn.scheduler.capacity.root..acl_submit_reservations` | The ACL which controls who can *submit* reservations to the given queue. If the given user/group has necessary ACLs on the given queue they can submit reservations. ACLs for this property *are not* inherited from the parent queue if not specified. | + ###Other Properties * Resource Calculator diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md index c9c820f75d1..3e111559d79 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md @@ -197,12 +197,16 @@ The allocation file must be in XML format. The format contains five types of ele ###Queue Access Control Lists -Queue Access Control Lists (ACLs) allow administrators to control who may take actions on particular queues. They are configured with the aclSubmitApps and aclAdministerApps properties, which can be set per queue. Currently the only supported administrative action is killing an application. Anybody who may administer a queue may also submit applications to it. These properties take values in a format like "user1,user2 group1,group2" or " group1,group2". An action on a queue will be permitted if its user or group is in the ACL of that queue or in the ACL of any of that queue's ancestors. So if queue2 is inside queue1, and user1 is in queue1's ACL, and user2 is in queue2's ACL, then both users may submit to queue2. +Queue Access Control Lists (ACLs) allow administrators to control who may take actions on particular queues. They are configured with the aclSubmitApps and aclAdministerApps properties, which can be set per queue. Currently the only supported administrative action is killing an application. An administrator may also submit applications to it. These properties take values in a format like "user1,user2 group1,group2" or " group1,group2". Actions on a queue are permitted if the user/group is a member of the queue ACL or a member of the queue ACL of any of that queue's ancestors. So if queue2 is inside queue1, and user1 is in queue1's ACL, and user2 is in queue2's ACL, then both users may submit to queue2. **Note:** The delimiter is a space character. To specify only ACL groups, begin the value with a space character. The root queue's ACLs are "\*" by default which, because ACLs are passed down, means that everybody may submit to and kill applications from every queue. To start restricting access, change the root queue's ACLs to something other than "\*". +###Reservation Access Control Lists + +Reservation Access Control Lists (ACLs) allow administrators to control who may take reservation actions on particular queues. They are configured with the aclAdministerReservations, aclListReservations, and the aclSubmitReservations properties, which can be set per queue. Currently the supported administrative actions are updating and deleting reservations. An administrator may also submit and list *all* reservations on the queue. These properties take values in a format like "user1,user2 group1,group2" or " group1,group2". Actions on a queue are permitted if the user/group is a member of the reservation ACL. Note that any user can update, delete, or list their own reservations. If reservation ACLs are enabled but not defined, everyone will have access. + ##Administration The fair scheduler provides support for administration at runtime through a few mechanisms: From 2c268cc9365851f5b02d967d13c8c0cbca850a86 Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Fri, 25 Mar 2016 15:17:27 +0900 Subject: [PATCH 25/43] MAPREDUCE-6543. Migrate MR client test cases part 2. Contributed by Dustin Cote. --- .../java/org/apache/hadoop/fs/DFSCIOTest.java | 8 +-- .../org/apache/hadoop/fs/TestFileSystem.java | 18 +++++-- .../java/org/apache/hadoop/fs/TestJHLA.java | 6 +-- .../io/TestSequenceFileMergeProgress.java | 12 +++-- .../mapred/ClusterMapReduceTestCase.java | 14 ++--- .../apache/hadoop/mapred/TestAuditLogger.java | 9 ++-- .../apache/hadoop/mapred/TestBadRecords.java | 8 ++- .../mapred/TestClusterMapReduceTestCase.java | 10 ++++ .../org/apache/hadoop/mapred/TestCollect.java | 10 ++-- .../mapred/TestCommandLineJobSubmission.java | 9 ++-- .../hadoop/mapred/TestFieldSelection.java | 6 ++- .../mapred/TestFileInputFormatPathFilter.java | 19 ++++--- .../hadoop/mapred/TestGetSplitHosts.java | 7 +-- .../hadoop/mapred/TestIFileStreams.java | 13 ++--- .../apache/hadoop/mapred/TestInputPath.java | 7 +-- .../hadoop/mapred/TestJavaSerialization.java | 10 ++-- .../org/apache/hadoop/mapred/TestJobName.java | 6 +++ .../hadoop/mapred/TestJobSysDirWithDFS.java | 10 ++-- .../mapred/TestKeyValueTextInputFormat.java | 15 +++--- .../apache/hadoop/mapred/TestLazyOutput.java | 7 +-- .../mapred/TestMRCJCFileInputFormat.java | 32 +++++++----- .../mapred/TestMRCJCFileOutputCommitter.java | 28 ++++++---- .../apache/hadoop/mapred/TestMapProgress.java | 9 ++-- .../org/apache/hadoop/mapred/TestMerge.java | 7 +-- .../hadoop/mapred/TestMiniMRBringup.java | 6 ++- .../hadoop/mapred/TestMiniMRDFSCaching.java | 14 +++-- .../mapred/TestMultiFileInputFormat.java | 19 +++---- .../hadoop/mapred/TestMultiFileSplit.java | 10 ++-- .../mapred/TestMultipleLevelCaching.java | 12 +++-- .../mapred/TestMultipleTextOutputFormat.java | 23 ++++---- .../apache/hadoop/mapred/TestReduceFetch.java | 10 ++-- .../mapred/TestReduceFetchFromPartialMem.java | 46 +++++++--------- .../apache/hadoop/mapred/TestReduceTask.java | 18 ++++--- .../TestSequenceFileAsBinaryInputFormat.java | 19 ++++--- .../TestSequenceFileAsBinaryOutputFormat.java | 31 +++++++---- .../TestSequenceFileAsTextInputFormat.java | 27 +++++----- .../mapred/TestSequenceFileInputFilter.java | 32 ++++++------ .../mapred/TestSequenceFileInputFormat.java | 26 +++++----- .../hadoop/mapred/TestSortedRanges.java | 19 ++++--- .../TestSpecialCharactersInOutputPath.java | 21 ++++---- .../mapred/TestStatisticsCollector.java | 10 ++-- .../mapred/TestUserDefinedCounters.java | 24 +++++---- .../hadoop/mapred/TestWritableJobConf.java | 20 ++++--- .../apache/hadoop/mapred/TestYARNRunner.java | 8 +-- .../hadoop/mapred/join/TestDatamerge.java | 42 ++++++++------- .../hadoop/mapred/join/TestTupleWritable.java | 24 ++++++--- .../TestWrappedRecordReaderClassloader.java | 7 +-- .../mapred/lib/TestDelegatingInputFormat.java | 9 ++-- .../mapred/lib/TestLineInputFormat.java | 7 +-- .../hadoop/mapred/lib/TestMultipleInputs.java | 2 - .../mapred/lib/aggregate/TestAggregates.java | 7 +-- .../mapred/lib/db/TestConstructQuery.java | 16 +++--- .../apache/hadoop/mapred/pipes/TestPipes.java | 9 ++-- .../hadoop/mapreduce/TestLocalRunner.java | 34 +++++++----- .../hadoop/mapreduce/TestMRJobClient.java | 49 +++++++++-------- .../mapreduce/TestMapReduceLazyOutput.java | 9 ++-- .../hadoop/mapreduce/TestValueIterReset.java | 8 +-- .../TestYarnClientProtocolProvider.java | 5 +- .../aggregate/TestMapReduceAggregates.java | 23 ++++---- .../mapreduce/lib/db/TestDBOutputFormat.java | 17 +++--- .../mapreduce/lib/db/TestIntegerSplitter.java | 15 ++++-- .../mapreduce/lib/db/TestTextSplitter.java | 18 +++++-- .../lib/fieldsel/TestMRFieldSelection.java | 20 ++++--- ...TestMRSequenceFileAsBinaryInputFormat.java | 21 +++++--- .../TestMRSequenceFileAsTextInputFormat.java | 27 ++++++---- .../input/TestMRSequenceFileInputFilter.java | 39 +++++++------- .../lib/input/TestNLineInputFormat.java | 34 +++++++----- .../mapreduce/lib/join/TestJoinDatamerge.java | 52 +++++++++++-------- .../lib/join/TestJoinProperties.java | 44 ++++++++-------- .../lib/join/TestJoinTupleWritable.java | 24 ++++++--- .../lib/join/TestWrappedRRClassloader.java | 17 ++++-- ...estMRSequenceFileAsBinaryOutputFormat.java | 35 +++++++++---- .../lib/partition/TestBinaryPartitioner.java | 16 ++++-- .../lib/partition/TestKeyFieldHelper.java | 9 +++- .../TestMRKeyFieldBasedPartitioner.java | 6 ++- .../partition/TestTotalOrderPartitioner.java | 11 ++-- .../util/TestMRAsyncDiskService.java | 15 ++++-- .../mapreduce/v2/TestMiniMRProxyUser.java | 30 ++++++----- .../mapreduce/v2/TestNonExistentJob.java | 18 ++++--- .../streaming/TestStreamingBadRecords.java | 9 +++- 80 files changed, 834 insertions(+), 569 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java index 1caa2cdae6c..12bec0869f6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java @@ -28,8 +28,6 @@ import java.io.PrintStream; import java.util.Date; import java.util.StringTokenizer; -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -39,8 +37,9 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.mapred.*; import org.junit.Ignore; +import org.junit.Test; -/** + /** * Distributed i/o benchmark. *

* This test writes into or reads from a specified number of files. @@ -68,7 +67,7 @@ import org.junit.Ignore; * */ @Ignore -public class DFSCIOTest extends TestCase { +public class DFSCIOTest { // Constants private static final Log LOG = LogFactory.getLog(DFSCIOTest.class); private static final int TEST_TYPE_READ = 0; @@ -98,6 +97,7 @@ public class DFSCIOTest extends TestCase { * * @throws Exception */ + @Test public void testIOs() throws Exception { testIOs(10, 10); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java index 4146b139c50..72840cdfd17 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java @@ -34,8 +34,6 @@ import java.util.HashMap; import java.net.InetSocketAddress; import java.net.URI; -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; @@ -50,8 +48,15 @@ import org.apache.hadoop.mapred.*; import org.apache.hadoop.mapred.lib.LongSumReducer; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.StringUtils; +import org.junit.Test; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.fail; -public class TestFileSystem extends TestCase { + +public class TestFileSystem { private static final Log LOG = FileSystem.LOG; private static Configuration conf = new Configuration(); @@ -66,6 +71,7 @@ public class TestFileSystem extends TestCase { private static Path READ_DIR = new Path(ROOT, "fs_read"); private static Path DATA_DIR = new Path(ROOT, "fs_data"); + @Test public void testFs() throws Exception { testFs(10 * MEGA, 100, 0); } @@ -90,6 +96,7 @@ public class TestFileSystem extends TestCase { fs.delete(READ_DIR, true); } + @Test public static void testCommandFormat() throws Exception { // This should go to TestFsShell.java when it is added. CommandFormat cf; @@ -488,6 +495,7 @@ public class TestFileSystem extends TestCase { } } + @Test public void testFsCache() throws Exception { { long now = System.currentTimeMillis(); @@ -561,6 +569,7 @@ public class TestFileSystem extends TestCase { + StringUtils.toUpperCase(add.getHostName()) + ":" + add.getPort())); } + @Test public void testFsClose() throws Exception { { Configuration conf = new Configuration(); @@ -569,6 +578,7 @@ public class TestFileSystem extends TestCase { } } + @Test public void testFsShutdownHook() throws Exception { final Set closed = Collections.synchronizedSet(new HashSet()); Configuration conf = new Configuration(); @@ -600,7 +610,7 @@ public class TestFileSystem extends TestCase { assertTrue(closed.contains(fsWithoutAuto)); } - + @Test public void testCacheKeysAreCaseInsensitive() throws Exception { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java index f2bc4edc46d..31950fd6104 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java @@ -23,19 +23,18 @@ import java.io.FileOutputStream; import java.io.OutputStreamWriter; import java.io.File; -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.junit.After; import org.junit.Before; +import org.junit.Test; /** * Test Job History Log Analyzer. * * @see JHLogAnalyzer */ -public class TestJHLA extends TestCase { +public class TestJHLA { private static final Log LOG = LogFactory.getLog(JHLogAnalyzer.class); private String historyLog = System.getProperty("test.build.data", "build/test/data") + "/history/test.log"; @@ -133,6 +132,7 @@ public class TestJHLA extends TestCase { /** * Run log analyzer in test mode for file test.log. */ + @Test public void testJHLA() { String[] args = {"-test", historyLog, "-jobDelimiter", ".!!FILE=.*!!"}; JHLogAnalyzer.main(args); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java index 1d7b98a6719..97dfa26acf4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java @@ -32,21 +32,25 @@ import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.mapred.*; -import junit.framework.TestCase; import org.apache.commons.logging.*; +import org.junit.Test; +import static org.junit.Assert.assertEquals; -public class TestSequenceFileMergeProgress extends TestCase { +public class TestSequenceFileMergeProgress { private static final Log LOG = FileInputFormat.LOG; private static final int RECORDS = 10000; - + + @Test public void testMergeProgressWithNoCompression() throws IOException { runTest(SequenceFile.CompressionType.NONE); } + @Test public void testMergeProgressWithRecordCompression() throws IOException { runTest(SequenceFile.CompressionType.RECORD); } + @Test public void testMergeProgressWithBlockCompression() throws IOException { runTest(SequenceFile.CompressionType.BLOCK); } @@ -92,7 +96,7 @@ public class TestSequenceFileMergeProgress extends TestCase { count++; } assertEquals(RECORDS, count); - assertEquals(1.0f, rIter.getProgress().get()); + assertEquals(1.0f, rIter.getProgress().get(), 0.0000); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java index 5bf4ff11b89..8d33b1580a8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java @@ -17,10 +17,11 @@ */ package org.apache.hadoop.mapred; -import junit.framework.TestCase; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Before; import java.io.IOException; import java.util.Map; @@ -41,7 +42,7 @@ import java.util.Properties; *

* The DFS filesystem is formated before the testcase starts and after it ends. */ -public abstract class ClusterMapReduceTestCase extends TestCase { +public abstract class ClusterMapReduceTestCase { private MiniDFSCluster dfsCluster = null; private MiniMRCluster mrCluster = null; @@ -50,9 +51,8 @@ public abstract class ClusterMapReduceTestCase extends TestCase { * * @throws Exception */ - protected void setUp() throws Exception { - super.setUp(); - + @Before + public void setUp() throws Exception { startCluster(true, null); } @@ -139,9 +139,9 @@ public abstract class ClusterMapReduceTestCase extends TestCase { * * @throws Exception */ - protected void tearDown() throws Exception { + @After + public void tearDown() throws Exception { stopCluster(); - super.tearDown(); } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java index 353185b59e3..bc85703bc84 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java @@ -28,13 +28,13 @@ import org.apache.hadoop.ipc.TestRPC.TestImpl; import org.apache.hadoop.ipc.TestRPC.TestProtocol; import org.apache.hadoop.mapred.AuditLogger.Keys; import org.apache.hadoop.net.NetUtils; - -import junit.framework.TestCase; +import org.junit.Test; +import static org.junit.Assert.assertEquals; /** * Tests {@link AuditLogger}. */ -public class TestAuditLogger extends TestCase { +public class TestAuditLogger { private static final String USER = "test"; private static final String OPERATION = "oper"; private static final String TARGET = "tgt"; @@ -44,6 +44,7 @@ public class TestAuditLogger extends TestCase { /** * Test the AuditLog format with key-val pair. */ + @Test public void testKeyValLogFormat() { StringBuilder actLog = new StringBuilder(); StringBuilder expLog = new StringBuilder(); @@ -114,6 +115,7 @@ public class TestAuditLogger extends TestCase { /** * Test {@link AuditLogger} without IP set. */ + @Test public void testAuditLoggerWithoutIP() throws Exception { // test without ip testSuccessLogFormat(false); @@ -137,6 +139,7 @@ public class TestAuditLogger extends TestCase { /** * Test {@link AuditLogger} with IP set. */ + @Test public void testAuditLoggerWithIP() throws Exception { Configuration conf = new Configuration(); // start the IPC server diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java index ea9f3d3f989..c2d6257823e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java @@ -40,6 +40,11 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.TaskCounter; import org.apache.hadoop.util.ReflectionUtils; import org.junit.Ignore; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNotNull; @Ignore public class TestBadRecords extends ClusterMapReduceTestCase { @@ -206,7 +211,8 @@ public class TestBadRecords extends ClusterMapReduceTestCase { } return processed; } - + + @Test public void testBadMapRed() throws Exception { JobConf conf = createJobConf(); conf.setMapperClass(BadMapper.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java index ada2d0c634b..f04fbd7a29a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java @@ -29,6 +29,12 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertFalse; public class TestClusterMapReduceTestCase extends ClusterMapReduceTestCase { public void _testMapReduce(boolean restart) throws Exception { OutputStream os = getFileSystem().create(new Path(getInputDir(), "text.txt")); @@ -85,14 +91,17 @@ public class TestClusterMapReduceTestCase extends ClusterMapReduceTestCase { } + @Test public void testMapReduce() throws Exception { _testMapReduce(false); } + @Test public void testMapReduceRestarting() throws Exception { _testMapReduce(true); } + @Test public void testDFSRestart() throws Exception { Path file = new Path(getInputDir(), "text.txt"); OutputStream os = getFileSystem().create(file); @@ -109,6 +118,7 @@ public class TestClusterMapReduceTestCase extends ClusterMapReduceTestCase { } + @Test public void testMRConfig() throws Exception { JobConf conf = createJobConf(); assertNull(conf.get("xyz")); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCollect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCollect.java index 4bd20d54ad5..595d09cc2a0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCollect.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCollect.java @@ -21,15 +21,15 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.io.*; import org.apache.hadoop.mapred.UtilsForTests.RandomInputFormat; import org.apache.hadoop.mapreduce.MRConfig; +import org.junit.Test; -import junit.framework.TestCase; import java.io.*; import java.util.*; /** * TestCollect checks if the collect can handle simultaneous invocations. */ -public class TestCollect extends TestCase +public class TestCollect { final static Path OUTPUT_DIR = new Path("build/test/test.collect.output"); static final int NUM_FEEDERS = 10; @@ -127,7 +127,7 @@ public class TestCollect extends TestCase conf.setNumMapTasks(1); conf.setNumReduceTasks(1); } - + @Test public void testCollect() throws IOException { JobConf conf = new JobConf(); configure(conf); @@ -144,9 +144,5 @@ public class TestCollect extends TestCase fs.delete(OUTPUT_DIR, true); } } - - public static void main(String[] args) throws IOException { - new TestCollect().testCollect(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java index 69353871cf4..7cf5e71e1a5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java @@ -21,28 +21,29 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.Ignore; +import org.junit.Test; +import static org.junit.Assert.assertTrue; /** * check for the job submission options of * -libjars -files -archives */ @Ignore -public class TestCommandLineJobSubmission extends TestCase { - // Input output paths for this.. +public class TestCommandLineJobSubmission { + // Input output paths for this.. // these are all dummy and does not test // much in map reduce except for the command line // params static final Path input = new Path("/test/input/"); static final Path output = new Path("/test/output"); File buildDir = new File(System.getProperty("test.build.data", "/tmp")); + @Test public void testJobShell() throws Exception { MiniDFSCluster dfs = null; MiniMRCluster mr = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFieldSelection.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFieldSelection.java index 239c239230e..7d7a7b0330f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFieldSelection.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFieldSelection.java @@ -23,11 +23,12 @@ import org.apache.hadoop.mapred.lib.*; import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.lib.fieldsel.FieldSelectionHelper; import org.apache.hadoop.mapreduce.lib.fieldsel.TestMRFieldSelection; +import org.junit.Test; +import static org.junit.Assert.assertEquals; -import junit.framework.TestCase; import java.text.NumberFormat; -public class TestFieldSelection extends TestCase { +public class TestFieldSelection { private static NumberFormat idFormat = NumberFormat.getInstance(); static { @@ -35,6 +36,7 @@ private static NumberFormat idFormat = NumberFormat.getInstance(); idFormat.setGroupingUsed(false); } + @Test public void testFieldSelection() throws Exception { launch(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileInputFormatPathFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileInputFormatPathFilter.java index 1c8be66d084..d87f6fd91a9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileInputFormatPathFilter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileInputFormatPathFilter.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.mapred; -import junit.framework.TestCase; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.assertEquals; import java.io.IOException; import java.io.Writer; @@ -30,7 +32,7 @@ import java.io.OutputStreamWriter; import java.util.Set; import java.util.HashSet; -public class TestFileInputFormatPathFilter extends TestCase { +public class TestFileInputFormatPathFilter { public static class DummyFileInputFormat extends FileInputFormat { @@ -55,12 +57,12 @@ public class TestFileInputFormatPathFilter extends TestCase { new Path(new Path(System.getProperty("test.build.data", "."), "data"), "TestFileInputFormatPathFilter"); - + @Before public void setUp() throws Exception { tearDown(); localFs.mkdirs(workDir); } - + @After public void tearDown() throws Exception { if (localFs.exists(workDir)) { localFs.delete(workDir, true); @@ -129,18 +131,19 @@ public class TestFileInputFormatPathFilter extends TestCase { assertEquals(createdFiles, computedFiles); } + @Test public void testWithoutPathFilterWithoutGlob() throws Exception { _testInputFiles(false, false); } - + @Test public void testWithoutPathFilterWithGlob() throws Exception { _testInputFiles(false, true); } - + @Test public void testWithPathFilterWithoutGlob() throws Exception { _testInputFiles(true, false); } - + @Test public void testWithPathFilterWithGlob() throws Exception { _testInputFiles(true, true); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestGetSplitHosts.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestGetSplitHosts.java index 7891bca7990..3d1c2e71bff 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestGetSplitHosts.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestGetSplitHosts.java @@ -20,10 +20,11 @@ package org.apache.hadoop.mapred; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.net.NetworkTopology; -import junit.framework.TestCase; - -public class TestGetSplitHosts extends TestCase { +import org.junit.Test; +import static org.junit.Assert.assertTrue; +public class TestGetSplitHosts { + @Test public void testGetSplitHosts() throws Exception { int numBlocks = 3; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java index 86431e5c135..2b97d3b95ad 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java @@ -21,11 +21,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; +import org.junit.Test; +import static org.junit.Assert.fail; +import static org.junit.Assert.assertEquals; -import junit.framework.TestCase; - -public class TestIFileStreams extends TestCase { - +public class TestIFileStreams { + @Test public void testIFileStream() throws Exception { final int DLEN = 100; DataOutputBuffer dob = new DataOutputBuffer(DLEN + 4); @@ -42,7 +43,7 @@ public class TestIFileStreams extends TestCase { } ifis.close(); } - + @Test public void testBadIFileStream() throws Exception { final int DLEN = 100; DataOutputBuffer dob = new DataOutputBuffer(DLEN + 4); @@ -73,7 +74,7 @@ public class TestIFileStreams extends TestCase { } fail("Did not detect bad data in checksum"); } - + @Test public void testBadLength() throws Exception { final int DLEN = 100; DataOutputBuffer dob = new DataOutputBuffer(DLEN + 4); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestInputPath.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestInputPath.java index 1398f9e5aaa..0c20c335d89 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestInputPath.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestInputPath.java @@ -17,14 +17,15 @@ */ package org.apache.hadoop.mapred; -import junit.framework.TestCase; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.util.StringUtils; +import org.junit.Test; +import static org.junit.Assert.assertEquals; -public class TestInputPath extends TestCase { +public class TestInputPath { + @Test public void testInputPath() throws Exception { JobConf jobConf = new JobConf(); Path workingDir = jobConf.getWorkingDirectory(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJavaSerialization.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJavaSerialization.java index 265118a70f6..a787e68c124 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJavaSerialization.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJavaSerialization.java @@ -26,8 +26,6 @@ import java.io.Writer; import java.util.Iterator; import java.util.StringTokenizer; -import junit.framework.TestCase; - import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; @@ -36,8 +34,11 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.serializer.JavaSerializationComparator; import org.apache.hadoop.mapreduce.MRConfig; +import org.junit.Test; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; -public class TestJavaSerialization extends TestCase { +public class TestJavaSerialization { private static String TEST_ROOT_DIR = new File(System.getProperty("test.build.data", "/tmp")).toURI() @@ -90,7 +91,7 @@ public class TestJavaSerialization extends TestCase { wr.write("b a\n"); wr.close(); } - + @Test public void testMapReduceJob() throws Exception { JobConf conf = new JobConf(TestJavaSerialization.class); @@ -149,6 +150,7 @@ public class TestJavaSerialization extends TestCase { * coupled to Writable types, if so, the job will fail. * */ + @Test public void testWriteToSequencefile() throws Exception { JobConf conf = new JobConf(TestJavaSerialization.class); conf.setJobName("JavaSerialization"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobName.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobName.java index 4b62b4a1d8e..2659a14a70b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobName.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobName.java @@ -29,8 +29,13 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.lib.IdentityMapper; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + public class TestJobName extends ClusterMapReduceTestCase { + @Test public void testComplexName() throws Exception { OutputStream os = getFileSystem().create(new Path(getInputDir(), "text.txt")); @@ -65,6 +70,7 @@ public class TestJobName extends ClusterMapReduceTestCase { reader.close(); } + @Test public void testComplexNameWithRegex() throws Exception { OutputStream os = getFileSystem().create(new Path(getInputDir(), "text.txt")); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java index 109c781c2b0..3dbc5777bd5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java @@ -21,8 +21,6 @@ package org.apache.hadoop.mapred; import java.io.DataOutputStream; import java.io.IOException; -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -32,11 +30,15 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; /** * A JUnit test to test Job System Directory with Mini-DFS. */ -public class TestJobSysDirWithDFS extends TestCase { +public class TestJobSysDirWithDFS { private static final Log LOG = LogFactory.getLog(TestJobSysDirWithDFS.class.getName()); @@ -115,7 +117,7 @@ public class TestJobSysDirWithDFS extends TestCase { // between Job Client & Job Tracker assertTrue(result.job.isSuccessful()); } - + @Test public void testWithDFS() throws IOException { MiniDFSCluster dfs = null; MiniMRCluster mr = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java index 27070783e14..bacc196008e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java @@ -20,7 +20,6 @@ package org.apache.hadoop.mapred; import java.io.*; import java.util.*; -import junit.framework.TestCase; import org.apache.commons.logging.*; import org.apache.hadoop.fs.*; @@ -28,8 +27,11 @@ import org.apache.hadoop.io.*; import org.apache.hadoop.io.compress.*; import org.apache.hadoop.util.LineReader; import org.apache.hadoop.util.ReflectionUtils; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; -public class TestKeyValueTextInputFormat extends TestCase { +public class TestKeyValueTextInputFormat { private static final Log LOG = LogFactory.getLog(TestKeyValueTextInputFormat.class.getName()); @@ -47,7 +49,7 @@ public class TestKeyValueTextInputFormat extends TestCase { private static Path workDir = new Path(new Path(System.getProperty("test.build.data", "."), "data"), "TestKeyValueTextInputFormat"); - + @Test public void testFormat() throws Exception { JobConf job = new JobConf(); Path file = new Path(workDir, "test.txt"); @@ -134,7 +136,7 @@ public class TestKeyValueTextInputFormat extends TestCase { (str.getBytes("UTF-8")), defaultConf); } - + @Test public void testUTF8() throws Exception { LineReader in = null; @@ -153,7 +155,7 @@ public class TestKeyValueTextInputFormat extends TestCase { } } } - + @Test public void testNewLines() throws Exception { LineReader in = null; try { @@ -219,7 +221,8 @@ public class TestKeyValueTextInputFormat extends TestCase { /** * Test using the gzip codec for reading */ - public static void testGzip() throws IOException { + @Test + public void testGzip() throws IOException { JobConf job = new JobConf(); CompressionCodec gzip = new GzipCodec(); ReflectionUtils.setConf(gzip, job); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java index 7412832d5c2..dde9310607f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java @@ -35,14 +35,15 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.lib.LazyOutputFormat; -import junit.framework.TestCase; +import org.junit.Test; +import static org.junit.Assert.assertTrue; /** * A JUnit test to test the Map-Reduce framework's feature to create part * files only if there is an explicit output.collect. This helps in preventing * 0 byte files */ -public class TestLazyOutput extends TestCase { +public class TestLazyOutput { private static final int NUM_HADOOP_SLAVES = 3; private static final int NUM_MAPS_PER_NODE = 2; private static final Path INPUT = new Path("/testlazy/input"); @@ -132,7 +133,7 @@ public class TestLazyOutput extends TestCase { } } - + @Test public void testLazyOutput() throws Exception { MiniDFSCluster dfs = null; MiniMRCluster mr = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java index fb9e8fcce3a..20d0173cc81 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java @@ -17,16 +17,6 @@ */ package org.apache.hadoop.mapred; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.util.concurrent.TimeoutException; - -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataOutputStream; @@ -36,9 +26,21 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.io.Text; +import org.junit.After; +import org.junit.Test; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.concurrent.TimeoutException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; @SuppressWarnings("deprecation") -public class TestMRCJCFileInputFormat extends TestCase { +public class TestMRCJCFileInputFormat { Configuration conf = new Configuration(); MiniDFSCluster dfs = null; @@ -50,6 +52,7 @@ public class TestMRCJCFileInputFormat extends TestCase { .build(); } + @Test public void testLocality() throws Exception { JobConf job = new JobConf(conf); dfs = newDFSCluster(job); @@ -109,6 +112,7 @@ public class TestMRCJCFileInputFormat extends TestCase { DFSTestUtil.waitReplication(fs, path, replication); } + @Test public void testNumInputs() throws Exception { JobConf job = new JobConf(conf); dfs = newDFSCluster(job); @@ -157,6 +161,7 @@ public class TestMRCJCFileInputFormat extends TestCase { } } + @Test public void testMultiLevelInput() throws Exception { JobConf job = new JobConf(conf); @@ -195,6 +200,7 @@ public class TestMRCJCFileInputFormat extends TestCase { } @SuppressWarnings("rawtypes") + @Test public void testLastInputSplitAtSplitBoundary() throws Exception { FileInputFormat fif = new FileInputFormatForTest(1024l * 1024 * 1024, 128l * 1024 * 1024); @@ -208,6 +214,7 @@ public class TestMRCJCFileInputFormat extends TestCase { } @SuppressWarnings("rawtypes") + @Test public void testLastInputSplitExceedingSplitBoundary() throws Exception { FileInputFormat fif = new FileInputFormatForTest(1027l * 1024 * 1024, 128l * 1024 * 1024); @@ -221,6 +228,7 @@ public class TestMRCJCFileInputFormat extends TestCase { } @SuppressWarnings("rawtypes") + @Test public void testLastInputSplitSingleSplit() throws Exception { FileInputFormat fif = new FileInputFormatForTest(100l * 1024 * 1024, 128l * 1024 * 1024); @@ -305,7 +313,7 @@ public class TestMRCJCFileInputFormat extends TestCase { DFSTestUtil.waitReplication(fileSys, name, replication); } - @Override + @After public void tearDown() throws Exception { if (dfs != null) { dfs.shutdown(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileOutputCommitter.java index 3b86f81cc23..74b6d77f6a0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileOutputCommitter.java @@ -18,18 +18,25 @@ package org.apache.hadoop.mapred; -import java.io.*; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.JobStatus; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; import java.net.URI; -import junit.framework.TestCase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; -import org.apache.hadoop.mapred.JobContextImpl; -import org.apache.hadoop.mapred.TaskAttemptContextImpl; -import org.apache.hadoop.mapreduce.JobStatus; - -public class TestMRCJCFileOutputCommitter extends TestCase { +public class TestMRCJCFileOutputCommitter { private static Path outDir = new Path( System.getProperty("test.build.data", "/tmp"), "output"); @@ -67,6 +74,7 @@ public class TestMRCJCFileOutputCommitter extends TestCase { } @SuppressWarnings("unchecked") + @Test public void testCommitter() throws Exception { JobConf job = new JobConf(); setConfForFileOutputCommitter(job); @@ -108,6 +116,7 @@ public class TestMRCJCFileOutputCommitter extends TestCase { FileUtil.fullyDelete(new File(outDir.toString())); } + @Test public void testAbort() throws IOException { JobConf job = new JobConf(); setConfForFileOutputCommitter(job); @@ -161,6 +170,7 @@ public class TestMRCJCFileOutputCommitter extends TestCase { } } + @Test public void testFailAbort() throws IOException { JobConf job = new JobConf(); job.set(FileSystem.FS_DEFAULT_NAME_KEY, "faildel:///"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java index db6348ba440..b8ff016d6af 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java @@ -22,8 +22,6 @@ import java.io.File; import java.io.IOException; import java.util.List; -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; @@ -40,6 +38,8 @@ import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo; import org.apache.hadoop.mapreduce.split.JobSplitWriter; import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader; import org.apache.hadoop.util.ReflectionUtils; +import org.junit.Test; +import static org.junit.Assert.assertTrue; /** * Validates map phase progress. @@ -55,7 +55,7 @@ import org.apache.hadoop.util.ReflectionUtils; * once mapTask.run() is finished. Sort phase progress in map task is not * validated here. */ -public class TestMapProgress extends TestCase { +public class TestMapProgress { public static final Log LOG = LogFactory.getLog(TestMapProgress.class); private static String TEST_ROOT_DIR; static { @@ -234,7 +234,8 @@ public class TestMapProgress extends TestCase { /** * Validates map phase progress after each record is processed by map task * using custom task reporter. - */ + */ + @Test public void testMapProgress() throws Exception { JobConf job = new JobConf(); fs = FileSystem.getLocal(job); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java index e19ff589fa4..a9e7f64c0b8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java @@ -44,8 +44,8 @@ import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.mapred.Task.TaskReporter; - -import junit.framework.TestCase; +import org.junit.Test; +import static org.junit.Assert.assertEquals; @SuppressWarnings(value={"unchecked", "deprecation"}) /** @@ -56,7 +56,7 @@ import junit.framework.TestCase; * framework's merge on the reduce side will merge the partitions created to * generate the final output which is sorted on the key. */ -public class TestMerge extends TestCase { +public class TestMerge { private static final int NUM_HADOOP_DATA_NODES = 2; // Number of input files is same as the number of mappers. private static final int NUM_MAPPERS = 10; @@ -69,6 +69,7 @@ public class TestMerge extends TestCase { // Where output goes. private static final Path OUTPUT = new Path("/testplugin/output"); + @Test public void testMerge() throws Exception { MiniDFSCluster dfsCluster = null; MiniMRClientCluster mrCluster = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRBringup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRBringup.java index 8b7b8f51b96..b608d756a49 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRBringup.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRBringup.java @@ -18,14 +18,16 @@ package org.apache.hadoop.mapred; +import org.junit.Test; + import java.io.IOException; -import junit.framework.TestCase; /** * A Unit-test to test bringup and shutdown of Mini Map-Reduce Cluster. */ -public class TestMiniMRBringup extends TestCase { +public class TestMiniMRBringup { + @Test public void testBringUp() throws IOException { MiniMRCluster mr = null; try { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java index 45879aff623..3f64f7a35b9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java @@ -18,20 +18,23 @@ package org.apache.hadoop.mapred; -import java.io.*; -import junit.framework.TestCase; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.mapred.MRCaching.TestResult; import org.junit.Ignore; +import org.junit.Test; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; /** * A JUnit test to test caching with DFS * */ @Ignore -public class TestMiniMRDFSCaching extends TestCase { +public class TestMiniMRDFSCaching { + @Test public void testWithDFS() throws IOException { MiniMRCluster mr = null; MiniDFSCluster dfs = null; @@ -70,9 +73,4 @@ public class TestMiniMRDFSCaching extends TestCase { } } } - - public static void main(String[] argv) throws Exception { - TestMiniMRDFSCaching td = new TestMiniMRDFSCaching(); - td.testWithDFS(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java index 49825e99f57..1bd29542fcd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java @@ -21,17 +21,17 @@ import java.io.IOException; import java.util.BitSet; import java.util.HashMap; import java.util.Random; - -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; -public class TestMultiFileInputFormat extends TestCase{ +public class TestMultiFileInputFormat { private static JobConf job = new JobConf(); @@ -79,7 +79,8 @@ public class TestMultiFileInputFormat extends TestCase{ FileInputFormat.setInputPaths(job, multiFileDir); return multiFileDir; } - + + @Test public void testFormat() throws IOException { LOG.info("Test started"); LOG.info("Max split count = " + MAX_SPLIT_COUNT); @@ -122,7 +123,8 @@ public class TestMultiFileInputFormat extends TestCase{ } LOG.info("Test Finished"); } - + + @Test public void testFormatWithLessPathsThanSplits() throws Exception { MultiFileInputFormat format = new DummyMultiFileInputFormat(); FileSystem fs = FileSystem.getLocal(job); @@ -135,9 +137,4 @@ public class TestMultiFileInputFormat extends TestCase{ initFiles(fs, 2, 500); assertEquals(2, format.getSplits(job, 4).length); } - - public static void main(String[] args) throws Exception{ - TestMultiFileInputFormat test = new TestMultiFileInputFormat(); - test.testFormat(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileSplit.java index 16ff6af9271..5bb336e4e81 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileSplit.java @@ -27,16 +27,19 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Arrays; -import junit.framework.TestCase; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + /** * * test MultiFileSplit class */ -public class TestMultiFileSplit extends TestCase{ +public class TestMultiFileSplit { + @Test public void testReadWrite() throws Exception { MultiFileSplit split = new MultiFileSplit(new JobConf(), new Path[] {new Path("/test/path/1"), new Path("/test/path/2")}, new long[] {100,200}); @@ -70,6 +73,7 @@ public class TestMultiFileSplit extends TestCase{ * test method getLocations * @throws IOException */ + @Test public void testgetLocations() throws IOException{ JobConf job= new JobConf(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java index 294723a9c87..7e8dfef03f1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java @@ -17,10 +17,6 @@ */ package org.apache.hadoop.mapred; -import java.io.IOException; - -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -32,12 +28,17 @@ import org.apache.hadoop.mapred.lib.IdentityReducer; import org.apache.hadoop.mapreduce.JobCounter; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; import org.junit.Ignore; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; /** * This test checks whether the task caches are created and used properly. */ @Ignore -public class TestMultipleLevelCaching extends TestCase { +public class TestMultipleLevelCaching { private static final int MAX_LEVEL = 5; final Path inDir = new Path("/cachetesting"); final Path outputPath = new Path("/output"); @@ -71,6 +72,7 @@ public class TestMultipleLevelCaching extends TestCase { return rack.toString(); } + @Test public void testMultiLevelCaching() throws Exception { for (int i = 1 ; i <= MAX_LEVEL; ++i) { testCachingAtLevel(i); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java index 14c097d77e1..b5047fc8331 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java @@ -18,15 +18,19 @@ package org.apache.hadoop.mapred; -import java.io.*; -import junit.framework.TestCase; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.lib.MultipleTextOutputFormat; +import org.junit.Test; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; +import java.io.File; +import java.io.IOException; -import org.apache.hadoop.mapred.lib.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; -public class TestMultipleTextOutputFormat extends TestCase { +public class TestMultipleTextOutputFormat { private static JobConf defaultConf = new JobConf(); private static FileSystem localFs = null; @@ -83,7 +87,8 @@ public class TestMultipleTextOutputFormat extends TestCase { writeData(rw); rw.close(null); } - + + @Test public void testFormat() throws Exception { JobConf job = new JobConf(); job.set(JobContext.TASK_ATTEMPT_ID, attempt); @@ -145,8 +150,4 @@ public class TestMultipleTextOutputFormat extends TestCase { //System.out.printf("File_2 output: %s\n", output); assertEquals(output, expectedOutput.toString()); } - - public static void main(String[] args) throws Exception { - new TestMultipleTextOutputFormat().testFormat(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetch.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetch.java index 586df38dcfc..767459f88b4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetch.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetch.java @@ -19,17 +19,18 @@ package org.apache.hadoop.mapred; import org.apache.hadoop.mapreduce.TaskCounter; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class TestReduceFetch extends TestReduceFetchFromPartialMem { - static { - setSuite(TestReduceFetch.class); - } - /** * Verify that all segments are read from disk * @throws Exception might be thrown */ + @Test public void testReduceFromDisk() throws Exception { final int MAP_TASKS = 8; JobConf job = mrCluster.createJobConf(); @@ -53,6 +54,7 @@ public class TestReduceFetch extends TestReduceFetchFromPartialMem { * Verify that no segment hits disk. * @throws Exception might be thrown */ + @Test public void testReduceFromMem() throws Exception { final int MAP_TASKS = 3; JobConf job = mrCluster.createJobConf(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java index 3a1a275ab91..9b04f64ac60 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java @@ -18,10 +18,6 @@ package org.apache.hadoop.mapred; -import junit.extensions.TestSetup; -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -30,7 +26,9 @@ import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.mapreduce.TaskCounter; -import org.apache.hadoop.mapreduce.MRConfig; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; import java.io.DataInput; import java.io.DataOutput; @@ -39,34 +37,27 @@ import java.util.Arrays; import java.util.Formatter; import java.util.Iterator; -public class TestReduceFetchFromPartialMem extends TestCase { +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class TestReduceFetchFromPartialMem { protected static MiniMRCluster mrCluster = null; protected static MiniDFSCluster dfsCluster = null; - protected static TestSuite mySuite; - protected static void setSuite(Class klass) { - mySuite = new TestSuite(klass); + @Before + public void setUp() throws Exception { + Configuration conf = new Configuration(); + dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + mrCluster = new MiniMRCluster(2, + dfsCluster.getFileSystem().getUri().toString(), 1); } - static { - setSuite(TestReduceFetchFromPartialMem.class); - } - - public static Test suite() { - TestSetup setup = new TestSetup(mySuite) { - protected void setUp() throws Exception { - Configuration conf = new Configuration(); - dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); - mrCluster = new MiniMRCluster(2, - dfsCluster.getFileSystem().getUri().toString(), 1); - } - protected void tearDown() throws Exception { - if (dfsCluster != null) { dfsCluster.shutdown(); } - if (mrCluster != null) { mrCluster.shutdown(); } - } - }; - return setup; + @After + public void tearDown() throws Exception { + if (dfsCluster != null) { dfsCluster.shutdown(); } + if (mrCluster != null) { mrCluster.shutdown(); } } private static final String tagfmt = "%04d"; @@ -78,6 +69,7 @@ public class TestReduceFetchFromPartialMem extends TestCase { } /** Verify that at least one segment does not hit disk */ + @Test public void testReduceFromPartialMem() throws Exception { final int MAP_TASKS = 7; JobConf job = mrCluster.createJobConf(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceTask.java index 43fd94871a2..69546a6cba2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceTask.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceTask.java @@ -17,10 +17,6 @@ */ package org.apache.hadoop.mapred; -import java.io.IOException; - -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; @@ -30,11 +26,17 @@ import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.util.Progressable; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * This test exercises the ValueIterator. */ -public class TestReduceTask extends TestCase { +public class TestReduceTask { static class NullProgress implements Progressable { public void progress() { } @@ -119,9 +121,10 @@ public class TestReduceTask extends TestCase { } assertEquals(vals.length, i); // make sure we have progress equal to 1.0 - assertEquals(1.0f, rawItr.getProgress().get()); + assertEquals(1.0f, rawItr.getProgress().get(),0.0000); } + @Test public void testValueIterator() throws Exception { Path tmpDir = new Path("build/test/test.reduce.task"); Configuration conf = new Configuration(); @@ -129,7 +132,8 @@ public class TestReduceTask extends TestCase { runValueIterator(tmpDir, testCase, conf, null); } } - + + @Test public void testValueIteratorWithCompression() throws Exception { Path tmpDir = new Path("build/test/test.reduce.task.compression"); Configuration conf = new Configuration(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java index b8be7400070..64b0983a5d6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java @@ -18,19 +18,26 @@ package org.apache.hadoop.mapred; +import org.apache.commons.logging.Log; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.junit.Test; + import java.io.IOException; import java.util.Random; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; -import junit.framework.TestCase; -import org.apache.commons.logging.*; - -public class TestSequenceFileAsBinaryInputFormat extends TestCase { +public class TestSequenceFileAsBinaryInputFormat { private static final Log LOG = FileInputFormat.LOG; private static final int RECORDS = 10000; + @Test public void testBinary() throws IOException { JobConf job = new JobConf(); FileSystem fs = FileSystem.getLocal(job); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java index abe21f223ef..03dc6a69003 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java @@ -18,24 +18,35 @@ package org.apache.hadoop.mapred; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.SequenceFile.CompressionType; +import org.junit.Test; + import java.io.IOException; import java.util.Random; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; -import org.apache.hadoop.io.SequenceFile.CompressionType; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; -import junit.framework.TestCase; -import org.apache.commons.logging.*; - -public class TestSequenceFileAsBinaryOutputFormat extends TestCase { +public class TestSequenceFileAsBinaryOutputFormat { private static final Log LOG = LogFactory.getLog(TestSequenceFileAsBinaryOutputFormat.class.getName()); - private static final int RECORDS = 10000; // A random task attempt id for testing. private static final String attempt = "attempt_200707121733_0001_m_000000_0"; + @Test public void testBinary() throws IOException { JobConf job = new JobConf(); FileSystem fs = FileSystem.getLocal(job); @@ -129,7 +140,8 @@ public class TestSequenceFileAsBinaryOutputFormat extends TestCase { assertEquals("Some records not found", RECORDS, count); } - public void testSequenceOutputClassDefaultsToMapRedOutputClass() + @Test + public void testSequenceOutputClassDefaultsToMapRedOutputClass() throws IOException { JobConf job = new JobConf(); FileSystem fs = FileSystem.getLocal(job); @@ -163,6 +175,7 @@ public class TestSequenceFileAsBinaryOutputFormat extends TestCase { job)); } + @Test public void testcheckOutputSpecsForbidRecordCompression() throws IOException { JobConf job = new JobConf(); FileSystem fs = FileSystem.getLocal(job); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java index 4cfd59af745..d4e5e17e11f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java @@ -18,22 +18,29 @@ package org.apache.hadoop.mapred; -import java.io.*; -import java.util.*; -import junit.framework.TestCase; +import org.apache.commons.logging.Log; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.junit.Test; -import org.apache.commons.logging.*; +import java.util.BitSet; +import java.util.Random; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; -import org.apache.hadoop.conf.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; -public class TestSequenceFileAsTextInputFormat extends TestCase { +public class TestSequenceFileAsTextInputFormat { private static final Log LOG = FileInputFormat.LOG; private static int MAX_LENGTH = 10000; private static Configuration conf = new Configuration(); + @Test public void testFormat() throws Exception { JobConf job = new JobConf(conf); FileSystem fs = FileSystem.getLocal(conf); @@ -112,8 +119,4 @@ public class TestSequenceFileAsTextInputFormat extends TestCase { } } - - public static void main(String[] args) throws Exception { - new TestSequenceFileAsTextInputFormat().testFormat(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java index e50c396a434..93f21ce9e49 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java @@ -18,17 +18,21 @@ package org.apache.hadoop.mapred; -import java.io.*; -import java.util.*; -import junit.framework.TestCase; +import org.apache.commons.logging.Log; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.junit.Test; -import org.apache.commons.logging.*; +import java.io.IOException; +import java.util.Random; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; -import org.apache.hadoop.conf.*; +import static org.junit.Assert.assertEquals; -public class TestSequenceFileInputFilter extends TestCase { +public class TestSequenceFileInputFilter { private static final Log LOG = FileInputFormat.LOG; private static final int MAX_LENGTH = 15000; @@ -97,7 +101,8 @@ public class TestSequenceFileInputFilter extends TestCase { } return count; } - + + @Test public void testRegexFilter() throws Exception { // set the filter class LOG.info("Testing Regex Filter with patter: \\A10*"); @@ -121,6 +126,7 @@ public class TestSequenceFileInputFilter extends TestCase { fs.delete(inDir, true); } + @Test public void testPercentFilter() throws Exception { LOG.info("Testing Percent Filter with frequency: 1000"); // set the filter class @@ -147,7 +153,8 @@ public class TestSequenceFileInputFilter extends TestCase { // clean up fs.delete(inDir, true); } - + + @Test public void testMD5Filter() throws Exception { // set the filter class LOG.info("Testing MD5 Filter with frequency: 1000"); @@ -168,9 +175,4 @@ public class TestSequenceFileInputFilter extends TestCase { // clean up fs.delete(inDir, true); } - - public static void main(String[] args) throws Exception { - TestSequenceFileInputFilter filter = new TestSequenceFileInputFilter(); - filter.testRegexFilter(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java index 575ed532545..338e91d4d35 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java @@ -18,22 +18,28 @@ package org.apache.hadoop.mapred; -import java.io.*; -import java.util.*; -import junit.framework.TestCase; +import org.apache.commons.logging.Log; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.SequenceFile; +import org.junit.Test; -import org.apache.commons.logging.*; +import java.util.BitSet; +import java.util.Random; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; -import org.apache.hadoop.conf.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; -public class TestSequenceFileInputFormat extends TestCase { +public class TestSequenceFileInputFormat { private static final Log LOG = FileInputFormat.LOG; private static int MAX_LENGTH = 10000; private static Configuration conf = new Configuration(); + @Test public void testFormat() throws Exception { JobConf job = new JobConf(conf); FileSystem fs = FileSystem.getLocal(conf); @@ -110,8 +116,4 @@ public class TestSequenceFileInputFormat extends TestCase { } } - - public static void main(String[] args) throws Exception { - new TestSequenceFileInputFormat().testFormat(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java index ad4d4ce17a9..82d1d2d09a1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java @@ -17,18 +17,20 @@ */ package org.apache.hadoop.mapred; -import java.util.Iterator; - -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.mapred.SortedRanges.Range; +import org.junit.Test; -public class TestSortedRanges extends TestCase { - private static final Log LOG = +import java.util.Iterator; + +import static org.junit.Assert.assertEquals; + +public class TestSortedRanges { + private static final Log LOG = LogFactory.getLog(TestSortedRanges.class); - + + @Test public void testAdd() { SortedRanges sr = new SortedRanges(); sr.add(new Range(2,9)); @@ -66,7 +68,8 @@ public class TestSortedRanges extends TestCase { assertEquals(77, it.next().longValue()); } - + + @Test public void testRemove() { SortedRanges sr = new SortedRanges(); sr.add(new Range(2,19)); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java index 426686f9bb5..b9e32759fa4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java @@ -18,12 +18,6 @@ package org.apache.hadoop.mapred; -import java.io.DataOutputStream; -import java.io.IOException; -import java.net.URI; - -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -34,14 +28,20 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.IdentityReducer; -import org.apache.hadoop.mapreduce.MRConfig; -import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; import org.apache.hadoop.util.Progressable; +import org.junit.Test; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.net.URI; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * A JUnit test to test that jobs' output filenames are not HTML-encoded (cf HADOOP-1795). */ -public class TestSpecialCharactersInOutputPath extends TestCase { +public class TestSpecialCharactersInOutputPath { private static final Log LOG = LogFactory.getLog(TestSpecialCharactersInOutputPath.class.getName()); @@ -96,7 +96,8 @@ public class TestSpecialCharactersInOutputPath extends TestCase { LOG.info("job is complete: " + runningJob.isSuccessful()); return (runningJob.isSuccessful()); } - + + @Test public void testJobWithDFS() throws IOException { String namenode = null; MiniDFSCluster dfs = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestStatisticsCollector.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestStatisticsCollector.java index 12568d09175..8a83e8153e3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestStatisticsCollector.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestStatisticsCollector.java @@ -19,14 +19,18 @@ package org.apache.hadoop.mapred; import java.util.Map; -import junit.framework.TestCase; - import org.apache.hadoop.mapred.StatisticsCollector.TimeWindow; import org.apache.hadoop.mapred.StatisticsCollector.Stat; +import org.junit.Test; -public class TestStatisticsCollector extends TestCase{ +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +public class TestStatisticsCollector { @SuppressWarnings("rawtypes") + @Test public void testMovingWindow() throws Exception { StatisticsCollector collector = new StatisticsCollector(1); TimeWindow window = new TimeWindow("test", 6, 2); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestUserDefinedCounters.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestUserDefinedCounters.java index 3c2cf215fb3..2d67edc581a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestUserDefinedCounters.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestUserDefinedCounters.java @@ -17,6 +17,15 @@ */ package org.apache.hadoop.mapred; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.lib.IdentityMapper; +import org.apache.hadoop.mapred.lib.IdentityReducer; +import org.junit.Test; + import java.io.BufferedReader; import java.io.File; import java.io.IOException; @@ -26,18 +35,10 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.Writer; -import junit.framework.TestCase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.lib.IdentityMapper; -import org.apache.hadoop.mapred.lib.IdentityReducer; - -public class TestUserDefinedCounters extends TestCase { - +public class TestUserDefinedCounters { private static String TEST_ROOT_DIR = new File(System.getProperty("test.build.data", "/tmp")).toURI() .toString().replace(' ', '+') @@ -75,6 +76,7 @@ public class TestUserDefinedCounters extends TestCase { wr.close(); } + @Test public void testMapReduceJob() throws Exception { JobConf conf = new JobConf(TestUserDefinedCounters.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestWritableJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestWritableJobConf.java index 2c0cedcbb30..82c68db30c5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestWritableJobConf.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestWritableJobConf.java @@ -18,12 +18,6 @@ package org.apache.hadoop.mapred; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; @@ -31,8 +25,15 @@ import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.util.GenericsUtil; +import org.junit.Test; -public class TestWritableJobConf extends TestCase { +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import static org.junit.Assert.assertTrue; + +public class TestWritableJobConf { private static final Configuration CONF = new Configuration(); @@ -78,15 +79,17 @@ public class TestWritableJobConf extends TestCase { } } - assertEquals(map1, map2); + assertTrue(map1.equals(map2)); } + @Test public void testEmptyConfiguration() throws Exception { JobConf conf = new JobConf(); Configuration deser = serDeser(conf); assertEquals(conf, deser); } + @Test public void testNonEmptyConfiguration() throws Exception { JobConf conf = new JobConf(); conf.set("a", "A"); @@ -95,6 +98,7 @@ public class TestWritableJobConf extends TestCase { assertEquals(conf, deser); } + @Test public void testConfigurationWithDefaults() throws Exception { JobConf conf = new JobConf(false); conf.set("a", "A"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java index 0e340428214..abf2e72e0d1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java @@ -18,6 +18,10 @@ package org.apache.hadoop.mapred; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -38,8 +42,6 @@ import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Map; -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -113,7 +115,7 @@ import org.mockito.stubbing.Answer; * Test YarnRunner and make sure the client side plugin works * fine */ -public class TestYARNRunner extends TestCase { +public class TestYARNRunner { private static final Log LOG = LogFactory.getLog(TestYARNRunner.class); private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java index 15cea69dab2..a3066765ec0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java @@ -22,11 +22,6 @@ import java.io.DataOutput; import java.io.IOException; import java.util.Iterator; -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; -import junit.extensions.TestSetup; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -54,23 +49,27 @@ import org.apache.hadoop.mapred.Utils; import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.IdentityReducer; import org.apache.hadoop.util.ReflectionUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; -public class TestDatamerge extends TestCase { +public class TestDatamerge { private static MiniDFSCluster cluster = null; - public static Test suite() { - TestSetup setup = new TestSetup(new TestSuite(TestDatamerge.class)) { - protected void setUp() throws Exception { - Configuration conf = new Configuration(); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); - } - protected void tearDown() throws Exception { - if (cluster != null) { - cluster.shutdown(); - } - } - }; - return setup; + + @Before + public void setUp() throws Exception { + Configuration conf = new Configuration(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + } + @After + public void tearDown() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } } private static SequenceFile.Writer[] createWriters(Path testdir, @@ -246,18 +245,22 @@ public class TestDatamerge extends TestCase { base.getFileSystem(job).delete(base, true); } + @Test public void testSimpleInnerJoin() throws Exception { joinAs("inner", InnerJoinChecker.class); } + @Test public void testSimpleOuterJoin() throws Exception { joinAs("outer", OuterJoinChecker.class); } + @Test public void testSimpleOverride() throws Exception { joinAs("override", OverrideChecker.class); } + @Test public void testNestedJoin() throws Exception { // outer(inner(S1,...,Sn),outer(S1,...Sn)) final int SOURCES = 3; @@ -350,6 +353,7 @@ public class TestDatamerge extends TestCase { } + @Test public void testEmptyJoin() throws Exception { JobConf job = new JobConf(); Path base = cluster.getFileSystem().makeQualified(new Path("/empty")); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestTupleWritable.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestTupleWritable.java index e421ede9827..56871550dc9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestTupleWritable.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestTupleWritable.java @@ -26,8 +26,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.Random; -import junit.framework.TestCase; - import org.apache.hadoop.io.BooleanWritable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.FloatWritable; @@ -36,8 +34,12 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; +import org.junit.Test; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; -public class TestTupleWritable extends TestCase { +public class TestTupleWritable { private TupleWritable makeTuple(Writable[] writs) { Writable[] sub1 = { writs[1], writs[2] }; @@ -100,6 +102,7 @@ public class TestTupleWritable extends TestCase { return i; } + @Test public void testIterable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -121,6 +124,7 @@ public class TestTupleWritable extends TestCase { verifIter(writs, t, 0); } + @Test public void testNestedIterable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -139,6 +143,7 @@ public class TestTupleWritable extends TestCase { assertTrue("Bad count", writs.length == verifIter(writs, sTuple, 0)); } + @Test public void testWritable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -162,6 +167,7 @@ public class TestTupleWritable extends TestCase { assertTrue("Failed to write/read tuple", sTuple.equals(dTuple)); } + @Test public void testWideWritable() throws Exception { Writable[] manyWrits = makeRandomWritables(131); @@ -180,7 +186,8 @@ public class TestTupleWritable extends TestCase { assertTrue("Failed to write/read tuple", sTuple.equals(dTuple)); assertEquals("All tuple data has not been read from the stream",-1,in.read()); } - + + @Test public void testWideWritable2() throws Exception { Writable[] manyWrits = makeRandomWritables(71); @@ -202,6 +209,7 @@ public class TestTupleWritable extends TestCase { * Tests a tuple writable with more than 64 values and the values set written * spread far apart. */ + @Test public void testSparseWideWritable() throws Exception { Writable[] manyWrits = makeRandomWritables(131); @@ -220,7 +228,7 @@ public class TestTupleWritable extends TestCase { assertTrue("Failed to write/read tuple", sTuple.equals(dTuple)); assertEquals("All tuple data has not been read from the stream",-1,in.read()); } - + @Test public void testWideTuple() throws Exception { Text emptyText = new Text("Should be empty"); Writable[] values = new Writable[64]; @@ -240,7 +248,7 @@ public class TestTupleWritable extends TestCase { } } } - + @Test public void testWideTuple2() throws Exception { Text emptyText = new Text("Should be empty"); Writable[] values = new Writable[64]; @@ -264,6 +272,7 @@ public class TestTupleWritable extends TestCase { /** * Tests that we can write more than 64 values. */ + @Test public void testWideTupleBoundary() throws Exception { Text emptyText = new Text("Should not be set written"); Writable[] values = new Writable[65]; @@ -287,6 +296,7 @@ public class TestTupleWritable extends TestCase { /** * Tests compatibility with pre-0.21 versions of TupleWritable */ + @Test public void testPreVersion21Compatibility() throws Exception { Writable[] manyWrits = makeRandomWritables(64); PreVersion21TupleWritable oldTuple = new PreVersion21TupleWritable(manyWrits); @@ -304,7 +314,7 @@ public class TestTupleWritable extends TestCase { assertTrue("Tuple writable is unable to read pre-0.21 versions of TupleWritable", oldTuple.isCompatible(dTuple)); assertEquals("All tuple data has not been read from the stream",-1,in.read()); } - + @Test public void testPreVersion21CompatibilityEmptyTuple() throws Exception { Writable[] manyWrits = new Writable[0]; PreVersion21TupleWritable oldTuple = new PreVersion21TupleWritable(manyWrits); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java index 3ca175a5049..ae5572f5dcd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java @@ -21,8 +21,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import junit.framework.TestCase; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; @@ -35,13 +33,16 @@ import org.apache.hadoop.mapred.JobConfigurable; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.util.ReflectionUtils; +import org.junit.Test; +import static org.junit.Assert.assertTrue; -public class TestWrappedRecordReaderClassloader extends TestCase { +public class TestWrappedRecordReaderClassloader { /** * Tests the class loader set by {@link JobConf#setClassLoader(ClassLoader)} * is inherited by any {@link WrappedRecordReader}s created by * {@link CompositeRecordReader} */ + @Test public void testClassLoader() throws Exception { JobConf job = new JobConf(); Fake_ClassLoader classLoader = new Fake_ClassLoader(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java index 8bd855433ea..b916026272e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java @@ -20,8 +20,6 @@ package org.apache.hadoop.mapred.lib; import java.io.DataOutputStream; import java.io.IOException; -import junit.framework.TestCase; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -32,9 +30,12 @@ import org.apache.hadoop.mapred.Mapper; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.TextInputFormat; +import org.junit.Test; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; -public class TestDelegatingInputFormat extends TestCase { - +public class TestDelegatingInputFormat { + @Test public void testSplitting() throws Exception { JobConf conf = new JobConf(); MiniDFSCluster dfs = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestLineInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestLineInputFormat.java index db9c219e9c1..388de0fb88d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestLineInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestLineInputFormat.java @@ -20,13 +20,14 @@ package org.apache.hadoop.mapred.lib; import java.io.*; import java.util.*; -import junit.framework.TestCase; import org.apache.hadoop.fs.*; import org.apache.hadoop.io.*; import org.apache.hadoop.mapred.*; +import org.junit.Test; +import static org.junit.Assert.assertEquals; -public class TestLineInputFormat extends TestCase { +public class TestLineInputFormat { private static int MAX_LENGTH = 200; private static JobConf defaultConf = new JobConf(); @@ -43,7 +44,7 @@ public class TestLineInputFormat extends TestCase { private static Path workDir = new Path(new Path(System.getProperty("test.build.data", "."), "data"), "TestLineInputFormat"); - + @Test public void testFormat() throws Exception { JobConf job = new JobConf(); Path file = new Path(workDir, "test.txt"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java index 3a9cb9ec337..115a6f70d08 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java @@ -36,7 +36,6 @@ import static org.junit.Assert.assertEquals; * @see TestDelegatingInputFormat */ public class TestMultipleInputs { - @Test public void testAddInputPathWithFormat() { final JobConf conf = new JobConf(); @@ -49,7 +48,6 @@ public class TestMultipleInputs { assertEquals(KeyValueTextInputFormat.class, inputs.get(new Path("/bar")) .getClass()); } - @Test public void testAddInputPathWithMapper() { final JobConf conf = new JobConf(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java index 6da96ce22bd..f33f83cb6c1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java @@ -22,13 +22,14 @@ import org.apache.hadoop.io.*; import org.apache.hadoop.mapred.*; import org.apache.hadoop.mapred.lib.*; import org.apache.hadoop.mapreduce.MapReduceTestUtil; +import org.junit.Test; +import static org.junit.Assert.assertEquals; -import junit.framework.TestCase; import java.io.*; import java.util.*; import java.text.NumberFormat; -public class TestAggregates extends TestCase { +public class TestAggregates { private static NumberFormat idFormat = NumberFormat.getInstance(); static { @@ -36,7 +37,7 @@ public class TestAggregates extends TestCase { idFormat.setGroupingUsed(false); } - + @Test public void testAggregates() throws Exception { launch(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/db/TestConstructQuery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/db/TestConstructQuery.java index 968bb066565..203da4e0b7c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/db/TestConstructQuery.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/db/TestConstructQuery.java @@ -19,13 +19,13 @@ package org.apache.hadoop.mapred.lib.db; import java.io.IOException; -import junit.framework.TestCase; - import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapred.JobConf; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; -public class TestConstructQuery extends TestCase { - +public class TestConstructQuery { private String[] fieldNames = new String[] { "id", "name", "value" }; private String[] nullFieldNames = new String[] { null, null, null }; private String expected = "INSERT INTO hadoop_output (id,name,value) VALUES (?,?,?);"; @@ -33,15 +33,15 @@ public class TestConstructQuery extends TestCase { private DBOutputFormat format = new DBOutputFormat(); - - public void testConstructQuery() { + @Test + public void testConstructQuery() { String actual = format.constructQuery("hadoop_output", fieldNames); assertEquals(expected, actual); - + actual = format.constructQuery("hadoop_output", nullFieldNames); assertEquals(nullExpected, actual); } - + @Test public void testSetOutput() throws IOException { JobConf job = new JobConf(); DBOutputFormat.setOutput(job, "hadoop_output", fieldNames); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java index dd7817d65b5..34b1d75dfed 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java @@ -44,10 +44,13 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.ToolRunner; import org.junit.Ignore; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; -import junit.framework.TestCase; @Ignore -public class TestPipes extends TestCase { +public class TestPipes { private static final Log LOG = LogFactory.getLog(TestPipes.class.getName()); @@ -66,7 +69,7 @@ public class TestPipes extends TestCase { fs.delete(p, true); assertFalse("output not cleaned up", fs.exists(p)); } - + @Test public void testPipes() throws IOException { if (System.getProperty("compile.c++") == null) { LOG.info("compile.c++ is not defined, so skipping TestPipes"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java index 29640c8854b..8177ecd405b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java @@ -17,36 +17,42 @@ */ package org.apache.hadoop.mapreduce; -import java.io.BufferedReader; -import java.io.BufferedWriter; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.fs.*; import org.apache.hadoop.mapred.LocalJobRunner; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.FileSplit; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.ReflectionUtils; - import org.junit.Test; -import junit.framework.TestCase; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; /** * Stress tests for the LocalJobRunner */ -public class TestLocalRunner extends TestCase { +public class TestLocalRunner { private static final Log LOG = LogFactory.getLog(TestLocalRunner.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java index 6f45b5f5dc2..8fe9078e9e0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java @@ -17,6 +17,23 @@ */ package org.apache.hadoop.mapreduce; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.mapred.ClusterMapReduceTestCase; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hadoop.mapreduce.tools.CLI; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.junit.Test; + import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -31,23 +48,11 @@ import java.io.PipedOutputStream; import java.io.PrintStream; import java.util.Arrays; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.RemoteIterator; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; -import org.junit.Assert; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.ClusterMapReduceTestCase; -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import org.apache.hadoop.mapreduce.tools.CLI; -import org.apache.hadoop.util.ExitUtil; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** test CLI class. CLI class implemented the Tool interface. @@ -103,7 +108,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { throw new IOException(); } } - + @Test public void testJobSubmissionSpecsAndFiles() throws Exception { Configuration conf = createJobConf(); Job job = MapReduceTestUtil.createJob(conf, getInputDir(), getOutputDir(), @@ -127,7 +132,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { /** * main test method */ - + @Test public void testJobClient() throws Exception { Configuration conf = createJobConf(); Job job = runJob(conf); @@ -180,8 +185,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { runTool(conf, jc, new String[] { "-fail-task", taid.toString() }, out); String answer = new String(out.toByteArray(), "UTF-8"); - Assert - .assertTrue(answer.contains("Killed task " + taid + " by failing it")); + assertTrue(answer.contains("Killed task " + taid + " by failing it")); } /** @@ -199,7 +203,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { runTool(conf, jc, new String[] { "-kill-task", taid.toString() }, out); String answer = new String(out.toByteArray(), "UTF-8"); - Assert.assertTrue(answer.contains("Killed task " + taid)); + assertTrue(answer.contains("Killed task " + taid)); } /** @@ -686,6 +690,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { * Test -list option displays job name. * The name is capped to 20 characters for display. */ + @Test public void testJobName() throws Exception { Configuration conf = createJobConf(); CLI jc = createJobClient(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java index 1e4f4de9f93..a69e06eacd9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java @@ -25,8 +25,6 @@ import java.io.Writer; import java.util.Arrays; import java.util.List; -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; @@ -42,13 +40,16 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; /** * A JUnit test to test the Map-Reduce framework's feature to create part * files only if there is an explicit output.collect. This helps in preventing * 0 byte files */ -public class TestMapReduceLazyOutput extends TestCase { +public class TestMapReduceLazyOutput { private static final int NUM_HADOOP_SLAVES = 3; private static final int NUM_MAPS_PER_NODE = 2; private static final Path INPUT = new Path("/testlazy/input"); @@ -122,7 +123,7 @@ public class TestMapReduceLazyOutput extends TestCase { } } - + @Test public void testLazyOutput() throws Exception { MiniDFSCluster dfs = null; MiniMRCluster mr = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java index 5cf08991869..b757fb2c34f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java @@ -27,8 +27,6 @@ import java.io.Writer; import java.util.ArrayList; import java.util.StringTokenizer; -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -43,12 +41,15 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; /** * A JUnit test to test the Map-Reduce framework's support for the * "mark-reset" functionality in Reduce Values Iterator */ -public class TestValueIterReset extends TestCase { +public class TestValueIterReset { private static final int NUM_MAPS = 1; private static final int NUM_TESTS = 4; private static final int NUM_VALUES = 40; @@ -518,6 +519,7 @@ public class TestValueIterReset extends TestCase { } } + @Test public void testValueIterReset() { try { Configuration conf = new Configuration(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java index 4d84fa9e108..308b7775a67 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -26,7 +27,6 @@ import static org.mockito.Mockito.doNothing; import java.io.IOException; import java.nio.ByteBuffer; -import junit.framework.TestCase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -44,8 +44,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.junit.Test; -public class TestYarnClientProtocolProvider extends TestCase { - +public class TestYarnClientProtocolProvider { private static final RecordFactory recordFactory = RecordFactoryProvider. getRecordFactory(null); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java index f24dffe2655..789ed98193e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java @@ -18,22 +18,24 @@ package org.apache.hadoop.mapreduce.lib.aggregate; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.*; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.io.*; -import org.apache.hadoop.mapred.Utils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.junit.Test; -import junit.framework.TestCase; -import java.io.*; import java.text.NumberFormat; -public class TestMapReduceAggregates extends TestCase { +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class TestMapReduceAggregates { private static NumberFormat idFormat = NumberFormat.getInstance(); static { @@ -41,7 +43,7 @@ public class TestMapReduceAggregates extends TestCase { idFormat.setGroupingUsed(false); } - + @Test public void testAggregates() throws Exception { launch(); } @@ -122,11 +124,4 @@ public class TestMapReduceAggregates extends TestCase { fs.delete(OUTPUT_DIR, true); fs.delete(INPUT_DIR, true); } - - /** - * Launches all the tasks in order. - */ - public static void main(String[] argv) throws Exception { - launch(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java index bff25d20038..014855f7d6a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java @@ -19,14 +19,15 @@ package org.apache.hadoop.mapreduce.lib.db; import java.io.IOException; -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.Job; +import org.junit.Test; -public class TestDBOutputFormat extends TestCase { - +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class TestDBOutputFormat { private String[] fieldNames = new String[] { "id", "name", "value" }; private String[] nullFieldNames = new String[] { null, null, null }; private String expected = "INSERT INTO hadoop_output " + @@ -35,15 +36,17 @@ public class TestDBOutputFormat extends TestCase { private DBOutputFormat format = new DBOutputFormat(); - - public void testConstructQuery() { + + @Test + public void testConstructQuery() { String actual = format.constructQuery("hadoop_output", fieldNames); assertEquals(expected, actual); actual = format.constructQuery("hadoop_output", nullFieldNames); assertEquals(nullExpected, actual); } - + + @Test public void testSetOutput() throws IOException { Job job = Job.getInstance(new Configuration()); DBOutputFormat.setOutput(job, "hadoop_output", fieldNames); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java index e50aba4f462..8b5d907dcdc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java @@ -17,15 +17,15 @@ */ package org.apache.hadoop.mapreduce.lib.db; -import java.io.IOException; -import java.math.BigDecimal; +import org.junit.Test; + import java.sql.SQLException; -import java.util.ArrayList; import java.util.List; -import junit.framework.TestCase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; -public class TestIntegerSplitter extends TestCase { +public class TestIntegerSplitter { private long [] toLongArray(List in) { long [] out = new long[in.size()]; for (int i = 0; i < in.size(); i++) { @@ -70,12 +70,14 @@ public class TestIntegerSplitter extends TestCase { } } + @Test public void testEvenSplits() throws SQLException { List splits = new IntegerSplitter().split(10, 0, 100); long [] expected = { 0, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100 }; assertLongArrayEquals(expected, toLongArray(splits)); } + @Test public void testOddSplits() throws SQLException { List splits = new IntegerSplitter().split(10, 0, 95); long [] expected = { 0, 9, 18, 27, 36, 45, 54, 63, 72, 81, 90, 95 }; @@ -83,12 +85,14 @@ public class TestIntegerSplitter extends TestCase { } + @Test public void testSingletonSplit() throws SQLException { List splits = new IntegerSplitter().split(1, 5, 5); long [] expected = { 5, 5 }; assertLongArrayEquals(expected, toLongArray(splits)); } + @Test public void testSingletonSplit2() throws SQLException { // Same test, but overly-high numSplits List splits = new IntegerSplitter().split(5, 5, 5); @@ -96,6 +100,7 @@ public class TestIntegerSplitter extends TestCase { assertLongArrayEquals(expected, toLongArray(splits)); } + @Test public void testTooManySplits() throws SQLException { List splits = new IntegerSplitter().split(5, 3, 5); long [] expected = { 3, 4, 5 }; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java index 045e3a1b1f6..e16f4234877 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java @@ -17,15 +17,16 @@ */ package org.apache.hadoop.mapreduce.lib.db; -import java.io.IOException; +import org.junit.Test; + import java.math.BigDecimal; import java.sql.SQLException; -import java.util.ArrayList; import java.util.List; -import junit.framework.TestCase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; -public class TestTextSplitter extends TestCase { +public class TestTextSplitter { public String formatArray(Object [] ar) { StringBuilder sb = new StringBuilder(); @@ -62,48 +63,56 @@ public class TestTextSplitter extends TestCase { } } + @Test public void testStringConvertEmpty() { TextSplitter splitter = new TextSplitter(); BigDecimal emptyBigDec = splitter.stringToBigDecimal(""); assertEquals(BigDecimal.ZERO, emptyBigDec); } + @Test public void testBigDecConvertEmpty() { TextSplitter splitter = new TextSplitter(); String emptyStr = splitter.bigDecimalToString(BigDecimal.ZERO); assertEquals("", emptyStr); } + @Test public void testConvertA() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("A")); assertEquals("A", out); } + @Test public void testConvertZ() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("Z")); assertEquals("Z", out); } + @Test public void testConvertThreeChars() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("abc")); assertEquals("abc", out); } + @Test public void testConvertStr() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("big str")); assertEquals("big str", out); } + @Test public void testConvertChomped() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("AVeryLongStringIndeed")); assertEquals("AVeryLon", out); } + @Test public void testAlphabetSplit() throws SQLException { // This should give us 25 splits, one per letter. TextSplitter splitter = new TextSplitter(); @@ -113,6 +122,7 @@ public class TestTextSplitter extends TestCase { assertArrayEquals(expected, splits.toArray(new String [0])); } + @Test public void testCommonPrefix() throws SQLException { // Splits between 'Hand' and 'Hardy' TextSplitter splitter = new TextSplitter(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java index 91070f89c42..6f9183ab21b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java @@ -18,15 +18,19 @@ package org.apache.hadoop.mapreduce.lib.fieldsel; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.MapReduceTestUtil; +import org.junit.Test; -import junit.framework.TestCase; import java.text.NumberFormat; -public class TestMRFieldSelection extends TestCase { +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class TestMRFieldSelection { private static NumberFormat idFormat = NumberFormat.getInstance(); static { @@ -34,6 +38,7 @@ private static NumberFormat idFormat = NumberFormat.getInstance(); idFormat.setGroupingUsed(false); } + @Test public void testFieldSelection() throws Exception { launch(); } @@ -114,11 +119,4 @@ private static NumberFormat idFormat = NumberFormat.getInstance(); System.out.println("ExpectedData:"); System.out.println(expectedOutput.toString()); } - - /** - * Launches all the tasks in order. - */ - public static void main(String[] argv) throws Exception { - launch(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java index f0b3d57486c..cbf9d183ef2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java @@ -18,11 +18,12 @@ package org.apache.hadoop.mapreduce.lib.input; -import java.io.IOException; -import java.util.Random; - -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; @@ -31,12 +32,18 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.MapContextImpl; +import org.junit.Test; -import junit.framework.TestCase; +import java.io.IOException; +import java.util.Random; -public class TestMRSequenceFileAsBinaryInputFormat extends TestCase { +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class TestMRSequenceFileAsBinaryInputFormat { private static final int RECORDS = 10000; + @Test public void testBinary() throws IOException, InterruptedException { Job job = Job.getInstance(); FileSystem fs = FileSystem.getLocal(job.getConfiguration()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java index 2d03c2dd96a..335ce050d82 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java @@ -18,11 +18,13 @@ package org.apache.hadoop.mapreduce.lib.input; -import java.util.*; -import junit.framework.TestCase; - -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; @@ -31,12 +33,19 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.apache.hadoop.conf.*; +import org.junit.Test; -public class TestMRSequenceFileAsTextInputFormat extends TestCase { +import java.util.BitSet; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class TestMRSequenceFileAsTextInputFormat { private static int MAX_LENGTH = 10000; private static Configuration conf = new Configuration(); + @Test public void testFormat() throws Exception { Job job = Job.getInstance(conf); FileSystem fs = FileSystem.getLocal(conf); @@ -112,8 +121,4 @@ public class TestMRSequenceFileAsTextInputFormat extends TestCase { } } - - public static void main(String[] args) throws Exception { - new TestMRSequenceFileAsTextInputFormat().testFormat(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java index edf7e1ad10d..89aa7b23057 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java @@ -18,14 +18,14 @@ package org.apache.hadoop.mapreduce.lib.input; -import java.io.*; -import java.util.*; -import junit.framework.TestCase; - -import org.apache.commons.logging.*; - -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; @@ -34,10 +34,15 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.apache.hadoop.conf.*; +import org.junit.Test; -public class TestMRSequenceFileInputFilter extends TestCase { - private static final Log LOG = +import java.io.IOException; +import java.util.Random; + +import static org.junit.Assert.assertEquals; + +public class TestMRSequenceFileInputFilter { + private static final Log LOG = LogFactory.getLog(TestMRSequenceFileInputFilter.class.getName()); private static final int MAX_LENGTH = 15000; @@ -113,7 +118,8 @@ public class TestMRSequenceFileInputFilter extends TestCase { } return count; } - + + @Test public void testRegexFilter() throws Exception { // set the filter class LOG.info("Testing Regex Filter with patter: \\A10*"); @@ -138,6 +144,7 @@ public class TestMRSequenceFileInputFilter extends TestCase { fs.delete(inDir, true); } + @Test public void testPercentFilter() throws Exception { LOG.info("Testing Percent Filter with frequency: 1000"); // set the filter class @@ -165,7 +172,8 @@ public class TestMRSequenceFileInputFilter extends TestCase { // clean up fs.delete(inDir, true); } - + + @Test public void testMD5Filter() throws Exception { // set the filter class LOG.info("Testing MD5 Filter with frequency: 1000"); @@ -187,9 +195,4 @@ public class TestMRSequenceFileInputFilter extends TestCase { // clean up fs.delete(inDir, true); } - - public static void main(String[] args) throws Exception { - TestMRSequenceFileInputFilter filter = new TestMRSequenceFileInputFilter(); - filter.testRegexFilter(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java index 7b3878d9475..477866f4e35 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java @@ -18,17 +18,28 @@ package org.apache.hadoop.mapreduce.lib.input; -import java.io.*; -import java.util.*; -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; -import org.apache.hadoop.mapreduce.*; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.MapContext; +import org.apache.hadoop.mapreduce.MapReduceTestUtil; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.MapContextImpl; +import org.junit.Test; -public class TestNLineInputFormat extends TestCase { +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class TestNLineInputFormat { private static int MAX_LENGTH = 200; private static Configuration conf = new Configuration(); @@ -45,7 +56,8 @@ public class TestNLineInputFormat extends TestCase { private static Path workDir = new Path(new Path(System.getProperty("test.build.data", "."), "data"), "TestNLineInputFormat"); - + + @Test public void testFormat() throws Exception { Job job = Job.getInstance(conf); Path file = new Path(workDir, "test.txt"); @@ -116,8 +128,4 @@ public class TestNLineInputFormat extends TestCase { } } } - - public static void main(String[] args) throws Exception { - new TestNLineInputFormat().testFormat(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java index d245bfd6cde..1173ea4fa47 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java @@ -19,11 +19,6 @@ package org.apache.hadoop.mapreduce.lib.join; import java.io.IOException; -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; -import junit.extensions.TestSetup; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -37,23 +32,31 @@ import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; -public class TestJoinDatamerge extends TestCase { +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestJoinDatamerge { private static MiniDFSCluster cluster = null; - public static Test suite() { - TestSetup setup = new TestSetup(new TestSuite(TestJoinDatamerge.class)) { - protected void setUp() throws Exception { - Configuration conf = new Configuration(); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); - } - protected void tearDown() throws Exception { - if (cluster != null) { - cluster.shutdown(); - } - } - }; - return setup; + + @BeforeClass + public static void setUp() throws Exception { + Configuration conf = new Configuration(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + } + + @AfterClass + public static void tearDown() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } } private static SequenceFile.Writer[] createWriters(Path testdir, @@ -111,7 +114,7 @@ public class TestJoinDatamerge extends TestCase { extends Mapper{ protected final static IntWritable one = new IntWritable(1); int srcs; - + public void setup(Context context) { srcs = context.getConfiguration().getInt("testdatamerge.sources", 0); assertTrue("Invalid src count: " + srcs, srcs > 0); @@ -123,7 +126,7 @@ public class TestJoinDatamerge extends TestCase { protected final static IntWritable one = new IntWritable(1); int srcs; - + public void setup(Context context) { srcs = context.getConfiguration().getInt("testdatamerge.sources", 0); assertTrue("Invalid src count: " + srcs, srcs > 0); @@ -272,10 +275,12 @@ public class TestJoinDatamerge extends TestCase { base.getFileSystem(conf).delete(base, true); } + @Test public void testSimpleInnerJoin() throws Exception { joinAs("inner", InnerJoinMapChecker.class, InnerJoinReduceChecker.class); } + @Test public void testSimpleOuterJoin() throws Exception { joinAs("outer", OuterJoinMapChecker.class, OuterJoinReduceChecker.class); } @@ -322,11 +327,13 @@ public class TestJoinDatamerge extends TestCase { } return product; } - + + @Test public void testSimpleOverride() throws Exception { joinAs("override", OverrideMapChecker.class, OverrideReduceChecker.class); } + @Test public void testNestedJoin() throws Exception { // outer(inner(S1,...,Sn),outer(S1,...Sn)) final int SOURCES = 3; @@ -422,6 +429,7 @@ public class TestJoinDatamerge extends TestCase { } + @Test public void testEmptyJoin() throws Exception { Configuration conf = new Configuration(); Path base = cluster.getFileSystem().makeQualified(new Path("/empty")); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java index 151bc875ad3..b6e76069d95 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java @@ -20,11 +20,6 @@ package org.apache.hadoop.mapreduce.lib.join; import java.io.IOException; import java.util.List; -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; -import junit.extensions.TestSetup; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -36,8 +31,14 @@ import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.task.MapContextImpl; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; -public class TestJoinProperties extends TestCase { +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestJoinProperties { private static MiniDFSCluster cluster = null; final static int SOURCES = 3; @@ -46,21 +47,19 @@ public class TestJoinProperties extends TestCase { static Path[] src; static Path base; - public static Test suite() { - TestSetup setup = new TestSetup(new TestSuite(TestJoinProperties.class)) { - protected void setUp() throws Exception { - Configuration conf = new Configuration(); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); - base = cluster.getFileSystem().makeQualified(new Path("/nested")); - src = generateSources(conf); - } - protected void tearDown() throws Exception { - if (cluster != null) { - cluster.shutdown(); - } - } - }; - return setup; + @BeforeClass + public static void setUp() throws Exception { + Configuration conf = new Configuration(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + base = cluster.getFileSystem().makeQualified(new Path("/nested")); + src = generateSources(conf); + } + + @AfterClass + public static void tearDown() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } } // Sources from 0 to srcs-2 have IntWritable key and IntWritable value @@ -233,6 +232,7 @@ public class TestJoinProperties extends TestCase { } // outer(outer(A, B), C) == outer(A,outer(B, C)) == outer(A, B, C) + @Test public void testOuterAssociativity() throws Exception { Configuration conf = new Configuration(); testExpr1(conf, "outer", TestType.OUTER_ASSOCIATIVITY, 33); @@ -241,6 +241,7 @@ public class TestJoinProperties extends TestCase { } // inner(inner(A, B), C) == inner(A,inner(B, C)) == inner(A, B, C) + @Test public void testInnerAssociativity() throws Exception { Configuration conf = new Configuration(); testExpr1(conf, "inner", TestType.INNER_ASSOCIATIVITY, 2); @@ -249,6 +250,7 @@ public class TestJoinProperties extends TestCase { } // override(inner(A, B), A) == A + @Test public void testIdentity() throws Exception { Configuration conf = new Configuration(); testExpr4(conf); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java index d35941fc884..093da266b95 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java @@ -24,8 +24,6 @@ import java.io.DataOutputStream; import java.util.Arrays; import java.util.Random; -import junit.framework.TestCase; - import org.apache.hadoop.io.BooleanWritable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.FloatWritable; @@ -33,8 +31,13 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; +import org.junit.Test; -public class TestJoinTupleWritable extends TestCase { +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestJoinTupleWritable { private TupleWritable makeTuple(Writable[] writs) { Writable[] sub1 = { writs[1], writs[2] }; @@ -97,6 +100,7 @@ public class TestJoinTupleWritable extends TestCase { return i; } + @Test public void testIterable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -118,6 +122,7 @@ public class TestJoinTupleWritable extends TestCase { verifIter(writs, t, 0); } + @Test public void testNestedIterable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -136,6 +141,7 @@ public class TestJoinTupleWritable extends TestCase { assertTrue("Bad count", writs.length == verifIter(writs, sTuple, 0)); } + @Test public void testWritable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -159,6 +165,7 @@ public class TestJoinTupleWritable extends TestCase { assertTrue("Failed to write/read tuple", sTuple.equals(dTuple)); } + @Test public void testWideWritable() throws Exception { Writable[] manyWrits = makeRandomWritables(131); @@ -178,7 +185,8 @@ public class TestJoinTupleWritable extends TestCase { assertEquals("All tuple data has not been read from the stream", -1, in.read()); } - + + @Test public void testWideWritable2() throws Exception { Writable[] manyWrits = makeRandomWritables(71); @@ -201,6 +209,7 @@ public class TestJoinTupleWritable extends TestCase { * Tests a tuple writable with more than 64 values and the values set written * spread far apart. */ + @Test public void testSparseWideWritable() throws Exception { Writable[] manyWrits = makeRandomWritables(131); @@ -220,7 +229,8 @@ public class TestJoinTupleWritable extends TestCase { assertEquals("All tuple data has not been read from the stream", -1, in.read()); } - + + @Test public void testWideTuple() throws Exception { Text emptyText = new Text("Should be empty"); Writable[] values = new Writable[64]; @@ -241,7 +251,8 @@ public class TestJoinTupleWritable extends TestCase { } } } - + + @Test public void testWideTuple2() throws Exception { Text emptyText = new Text("Should be empty"); Writable[] values = new Writable[64]; @@ -266,6 +277,7 @@ public class TestJoinTupleWritable extends TestCase { /** * Tests that we can write more than 64 values. */ + @Test public void testWideTupleBoundary() throws Exception { Text emptyText = new Text("Should not be set written"); Writable[] values = new Writable[65]; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java index 36cf1872ad4..680e246b4e3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java @@ -17,23 +17,32 @@ */ package org.apache.hadoop.mapreduce.lib.join; -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapreduce.*; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.MapReduceTestUtil.Fake_RR; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.junit.Test; -public class TestWrappedRRClassloader extends TestCase { +import static org.junit.Assert.assertTrue; + +public class TestWrappedRRClassloader { /** * Tests the class loader set by * {@link Configuration#setClassLoader(ClassLoader)} * is inherited by any {@link WrappedRecordReader}s created by * {@link CompositeRecordReader} */ + @Test public void testClassLoader() throws Exception { Configuration conf = new Configuration(); Fake_ClassLoader classLoader = new Fake_ClassLoader(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java index 2e40f72fdd2..5a8aeda83be 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java @@ -18,12 +18,17 @@ package org.apache.hadoop.mapreduce.lib.output; -import java.io.IOException; -import java.util.Random; - +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.io.*; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.mapred.InvalidJobConfException; import org.apache.hadoop.mapreduce.InputFormat; @@ -38,16 +43,22 @@ import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.task.MapContextImpl; +import org.junit.Test; -import junit.framework.TestCase; -import org.apache.commons.logging.*; +import java.io.IOException; +import java.util.Random; -public class TestMRSequenceFileAsBinaryOutputFormat extends TestCase { +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class TestMRSequenceFileAsBinaryOutputFormat { private static final Log LOG = LogFactory.getLog(TestMRSequenceFileAsBinaryOutputFormat.class.getName()); private static final int RECORDS = 10000; - + + @Test public void testBinary() throws IOException, InterruptedException { Configuration conf = new Configuration(); Job job = Job.getInstance(conf); @@ -144,7 +155,8 @@ public class TestMRSequenceFileAsBinaryOutputFormat extends TestCase { assertEquals("Some records not found", RECORDS, count); } - public void testSequenceOutputClassDefaultsToMapRedOutputClass() + @Test + public void testSequenceOutputClassDefaultsToMapRedOutputClass() throws IOException { Job job = Job.getInstance(); // Setting Random class to test getSequenceFileOutput{Key,Value}Class @@ -172,7 +184,8 @@ public class TestMRSequenceFileAsBinaryOutputFormat extends TestCase { SequenceFileAsBinaryOutputFormat.getSequenceFileOutputValueClass(job)); } - public void testcheckOutputSpecsForbidRecordCompression() + @Test + public void testcheckOutputSpecsForbidRecordCompression() throws IOException { Job job = Job.getInstance(); FileSystem fs = FileSystem.getLocal(job.getConfiguration()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java index 7be538ecf41..f83bc11a216 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java @@ -22,11 +22,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BinaryComparable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.util.ReflectionUtils; +import org.junit.Test; -import junit.framework.TestCase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; -public class TestBinaryPartitioner extends TestCase { +public class TestBinaryPartitioner { + @Test public void testDefaultOffsets() { Configuration conf = new Configuration(); BinaryPartitioner partitioner = @@ -50,7 +53,8 @@ public class TestBinaryPartitioner extends TestCase { partition2 = partitioner.getPartition(key2, null, 10); assertTrue(partition1 != partition2); } - + + @Test public void testCustomOffsets() { Configuration conf = new Configuration(); BinaryComparable key1 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 }); @@ -75,7 +79,8 @@ public class TestBinaryPartitioner extends TestCase { partition2 = partitioner.getPartition(key2, null, 10); assertEquals(partition1, partition2); } - + + @Test public void testLowerBound() { Configuration conf = new Configuration(); BinaryPartitioner.setLeftOffset(conf, 0); @@ -87,7 +92,8 @@ public class TestBinaryPartitioner extends TestCase { int partition2 = partitioner.getPartition(key2, null, 10); assertTrue(partition1 != partition2); } - + + @Test public void testUpperBound() { Configuration conf = new Configuration(); BinaryPartitioner.setRightOffset(conf, 4); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java index 6bad846f6d3..4d05d13d445 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java @@ -19,14 +19,17 @@ package org.apache.hadoop.mapreduce.lib.partition; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.junit.Test; -import junit.framework.TestCase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; -public class TestKeyFieldHelper extends TestCase { +public class TestKeyFieldHelper { private static final Log LOG = LogFactory.getLog(TestKeyFieldHelper.class); /** * Test is key-field-helper's parse option. */ + @Test public void testparseOption() throws Exception { KeyFieldHelper helper = new KeyFieldHelper(); helper.setKeyFieldSeparator("\t"); @@ -212,6 +215,7 @@ public class TestKeyFieldHelper extends TestCase { /** * Test is key-field-helper's getWordLengths. */ + @Test public void testGetWordLengths() throws Exception { KeyFieldHelper helper = new KeyFieldHelper(); helper.setKeyFieldSeparator("\t"); @@ -270,6 +274,7 @@ public class TestKeyFieldHelper extends TestCase { /** * Test is key-field-helper's getStartOffset/getEndOffset. */ + @Test public void testgetStartEndOffset() throws Exception { KeyFieldHelper helper = new KeyFieldHelper(); helper.setKeyFieldSeparator("\t"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java index 9c2fb48d9bf..00b415f32cb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java @@ -19,14 +19,16 @@ package org.apache.hadoop.mapreduce.lib.partition; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; +import org.junit.Test; -import junit.framework.TestCase; +import static org.junit.Assert.assertEquals; -public class TestMRKeyFieldBasedPartitioner extends TestCase { +public class TestMRKeyFieldBasedPartitioner { /** * Test is key-field-based partitioned works with empty key. */ + @Test public void testEmptyKey() throws Exception { int numReducers = 10; KeyFieldBasedPartitioner kfbp = diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java index a844737e09d..bdb4ff4794e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java @@ -23,8 +23,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; @@ -41,8 +39,11 @@ import org.apache.hadoop.io.serializer.JavaSerializationComparator; import org.apache.hadoop.io.serializer.Serialization; import org.apache.hadoop.io.serializer.WritableSerialization; import org.apache.hadoop.mapreduce.MRJobConfig; +import org.junit.Test; -public class TestTotalOrderPartitioner extends TestCase { +import static org.junit.Assert.assertEquals; + +public class TestTotalOrderPartitioner { private static final Text[] splitStrings = new Text[] { // -inf // 0 @@ -140,6 +141,7 @@ public class TestTotalOrderPartitioner extends TestCase { return p; } + @Test public void testTotalOrderWithCustomSerialization() throws Exception { TotalOrderPartitioner partitioner = new TotalOrderPartitioner(); @@ -165,6 +167,7 @@ public class TestTotalOrderPartitioner extends TestCase { } } + @Test public void testTotalOrderMemCmp() throws Exception { TotalOrderPartitioner partitioner = new TotalOrderPartitioner(); @@ -184,6 +187,7 @@ public class TestTotalOrderPartitioner extends TestCase { } } + @Test public void testTotalOrderBinarySearch() throws Exception { TotalOrderPartitioner partitioner = new TotalOrderPartitioner(); @@ -216,6 +220,7 @@ public class TestTotalOrderPartitioner extends TestCase { } } + @Test public void testTotalOrderCustomComparator() throws Exception { TotalOrderPartitioner partitioner = new TotalOrderPartitioner(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java index e1849a3ce9c..07b5d8b9f50 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java @@ -20,8 +20,6 @@ package org.apache.hadoop.mapreduce.util; import java.io.File; import java.io.IOException; -import junit.framework.TestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,20 +28,27 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.mapreduce.util.MRAsyncDiskService; +import org.junit.Before; import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * A test for MRAsyncDiskService. */ -public class TestMRAsyncDiskService extends TestCase { +public class TestMRAsyncDiskService { public static final Log LOG = LogFactory.getLog(TestMRAsyncDiskService.class); private static String TEST_ROOT_DIR = new Path(System.getProperty( "test.build.data", "/tmp")).toString(); - @Override - protected void setUp() { + @Before + public void setUp() { FileUtil.fullyDelete(new File(TEST_ROOT_DIR)); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java index aa769f85974..f68cc8310a6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.mapreduce.v2; -import junit.framework.TestCase; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; @@ -29,22 +28,25 @@ import org.apache.hadoop.mapred.MiniMRCluster; import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.ProxyUsers; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; -import java.net.InetAddress; -import java.io.File; -import java.io.FileOutputStream; -import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.Writer; +import java.net.InetAddress; import java.security.PrivilegedExceptionAction; -public class TestMiniMRProxyUser extends TestCase { +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class TestMiniMRProxyUser { private MiniDFSCluster dfsCluster = null; private MiniMRCluster mrCluster = null; - - protected void setUp() throws Exception { - super.setUp(); + + @Before + public void setUp() throws Exception { if (System.getProperty("hadoop.log.dir") == null) { System.setProperty("hadoop.log.dir", "/tmp"); } @@ -91,15 +93,14 @@ public class TestMiniMRProxyUser extends TestCase { return mrCluster.createJobConf(); } - @Override - protected void tearDown() throws Exception { + @After + public void tearDown() throws Exception { if (mrCluster != null) { mrCluster.shutdown(); } if (dfsCluster != null) { dfsCluster.shutdown(); } - super.tearDown(); } private void mrRun() throws Exception { @@ -125,11 +126,13 @@ public class TestMiniMRProxyUser extends TestCase { assertTrue(runJob.isComplete()); assertTrue(runJob.isSuccessful()); } - + + @Test public void __testCurrentUser() throws Exception { mrRun(); } + @Test public void testValidProxyUser() throws Exception { UserGroupInformation ugi = UserGroupInformation.createProxyUser("u1", UserGroupInformation.getLoginUser()); ugi.doAs(new PrivilegedExceptionAction() { @@ -142,6 +145,7 @@ public class TestMiniMRProxyUser extends TestCase { }); } + @Test public void ___testInvalidProxyUser() throws Exception { UserGroupInformation ugi = UserGroupInformation.createProxyUser("u2", UserGroupInformation.getLoginUser()); ugi.doAs(new PrivilegedExceptionAction() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java index b6947f3fc48..e90c509d7a8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.mapreduce.v2; -import junit.framework.TestCase; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; @@ -28,17 +27,22 @@ import org.apache.hadoop.mapred.JobID; import org.apache.hadoop.mapred.MiniMRCluster; import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.security.authorize.ProxyUsers; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; import java.io.IOException; import java.net.InetAddress; -public class TestNonExistentJob extends TestCase { +import static org.junit.Assert.assertNull; + +public class TestNonExistentJob { private MiniDFSCluster dfsCluster = null; private MiniMRCluster mrCluster = null; - protected void setUp() throws Exception { - super.setUp(); + @Before + public void setUp() throws Exception { if (System.getProperty("hadoop.log.dir") == null) { System.setProperty("hadoop.log.dir", "/tmp"); } @@ -78,17 +82,17 @@ public class TestNonExistentJob extends TestCase { return mrCluster.createJobConf(); } - @Override - protected void tearDown() throws Exception { + @After + public void tearDown() throws Exception { if (mrCluster != null) { mrCluster.shutdown(); } if (dfsCluster != null) { dfsCluster.shutdown(); } - super.tearDown(); } + @Test public void testGetInvalidJob() throws Exception { RunningJob runJob = new JobClient(getJobConf()).getJob(JobID.forName("job_0_0")); assertNull(runJob); diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingBadRecords.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingBadRecords.java index 7b7901faad1..860fb89cfcf 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingBadRecords.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingBadRecords.java @@ -42,6 +42,11 @@ import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.SkipBadRecords; import org.apache.hadoop.mapred.Utils; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class TestStreamingBadRecords extends ClusterMapReduceTestCase { @@ -68,7 +73,8 @@ public class TestStreamingBadRecords extends ClusterMapReduceTestCase utilTest.redirectIfAntJunit(); } - protected void setUp() throws Exception { + @Before + public void setUp() throws Exception { Properties props = new Properties(); props.setProperty(JTConfig.JT_RETIREJOBS, "false"); props.setProperty(JTConfig.JT_PERSIST_JOBSTATUS, "false"); @@ -242,6 +248,7 @@ public class TestStreamingBadRecords extends ClusterMapReduceTestCase } */ + @Test public void testNoOp() { // Added to avoid warnings when running this disabled test } From 3f622a143c5fb15fee7e5dded99e4a4136f19810 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 25 Mar 2016 20:04:32 +0000 Subject: [PATCH 26/43] MAPREDUCE-6535. TaskID default constructor results in NPE on toString(). Contributed by Daniel Templeton --- .../org/apache/hadoop/mapreduce/TaskID.java | 57 ++- .../apache/hadoop/mapreduce/TestTaskID.java | 461 ++++++++++++++++++ 2 files changed, 508 insertions(+), 10 deletions(-) create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestTaskID.java diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskID.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskID.java index b9817dd16e3..3ddfbe97c84 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskID.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/TaskID.java @@ -63,6 +63,7 @@ public class TaskID extends org.apache.hadoop.mapred.ID { public static final String TASK_ID_REGEX = TASK + "_(\\d+)_(\\d+)_" + CharTaskTypeMaps.allTaskTypes + "_(\\d+)"; public static final Pattern taskIdPattern = Pattern.compile(TASK_ID_REGEX); + static { idFormat.setGroupingUsed(false); idFormat.setMinimumIntegerDigits(6); @@ -72,7 +73,8 @@ public class TaskID extends org.apache.hadoop.mapred.ID { private TaskType type; /** - * Constructs a TaskID object from given {@link JobID}. + * Constructs a TaskID object from given {@link JobID}. + * * @param jobId JobID that this tip belongs to * @param type the {@link TaskType} of the task * @param id the tip number @@ -88,6 +90,7 @@ public class TaskID extends org.apache.hadoop.mapred.ID { /** * Constructs a TaskInProgressId object from given parts. + * * @param jtIdentifier jobTracker identifier * @param jobId job number * @param type the TaskType @@ -99,6 +102,7 @@ public class TaskID extends org.apache.hadoop.mapred.ID { /** * Constructs a TaskID object from given {@link JobID}. + * * @param jobId JobID that this tip belongs to * @param isMap whether the tip is a map * @param id the tip number @@ -110,6 +114,7 @@ public class TaskID extends org.apache.hadoop.mapred.ID { /** * Constructs a TaskInProgressId object from given parts. + * * @param jtIdentifier jobTracker identifier * @param jobId job number * @param isMap whether the tip is a map @@ -120,23 +125,37 @@ public class TaskID extends org.apache.hadoop.mapred.ID { this(new JobID(jtIdentifier, jobId), isMap, id); } + /** + * Default constructor for Writable. Sets the task type to + * {@link TaskType#REDUCE}, the ID to 0, and the job ID to an empty job ID. + */ public TaskID() { - jobId = new JobID(); + this(new JobID(), TaskType.REDUCE, 0); } - /** Returns the {@link JobID} object that this tip belongs to */ + /** + * Returns the {@link JobID} object that this tip belongs to. + * + * @return the JobID object + */ public JobID getJobID() { return jobId; } - /**Returns whether this TaskID is a map ID */ + /** + * Returns whether this TaskID is a map ID. + * + * @return whether this TaskID is a map ID + */ @Deprecated public boolean isMap() { return type == TaskType.MAP; } /** - * Get the type of the task + * Get the type of the task. + * + * @return the type of the task */ public TaskType getTaskType() { return type; @@ -151,8 +170,14 @@ public class TaskID extends org.apache.hadoop.mapred.ID { return this.type == that.type && this.jobId.equals(that.jobId); } - /**Compare TaskInProgressIds by first jobIds, then by tip numbers. Reduces are - * defined as greater then maps.*/ + /** + * Compare TaskInProgressIds by first jobIds, then by tip numbers. + * Reducers are defined as greater than mappers. + * + * @param o the TaskID against which to compare + * @return 0 if equal, positive if this TaskID is greater, and negative if + * this TaskID is less + */ @Override public int compareTo(ID o) { TaskID that = (TaskID)o; @@ -174,6 +199,7 @@ public class TaskID extends org.apache.hadoop.mapred.ID { /** * Add the unique string to the given builder. + * * @param builder the builder to append to * @return the builder that was passed in */ @@ -204,7 +230,10 @@ public class TaskID extends org.apache.hadoop.mapred.ID { WritableUtils.writeEnum(out, type); } - /** Construct a TaskID object from given string + /** + * Construct a TaskID object from given string. + * + * @param str the target string * @return constructed TaskID object or null if the given String is null * @throws IllegalArgumentException if the given string is malformed */ @@ -224,7 +253,8 @@ public class TaskID extends org.apache.hadoop.mapred.ID { throw new IllegalArgumentException(exceptionMsg); } /** - * Gets the character representing the {@link TaskType} + * Gets the character representing the {@link TaskType}. + * * @param type the TaskType * @return the character */ @@ -232,7 +262,8 @@ public class TaskID extends org.apache.hadoop.mapred.ID { return CharTaskTypeMaps.getRepresentingCharacter(type); } /** - * Gets the {@link TaskType} corresponding to the character + * Gets the {@link TaskType} corresponding to the character. + * * @param c the character * @return the TaskType */ @@ -240,6 +271,12 @@ public class TaskID extends org.apache.hadoop.mapred.ID { return CharTaskTypeMaps.getTaskType(c); } + /** + * Returns a string of characters describing all possible {@link TaskType} + * values + * + * @return a string of all task type characters + */ public static String getAllTaskTypes() { return CharTaskTypeMaps.allTaskTypes; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestTaskID.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestTaskID.java new file mode 100644 index 00000000000..5531074dce8 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestTaskID.java @@ -0,0 +1,461 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.mapreduce; + +import org.apache.hadoop.io.DataInputByteBuffer; +import org.apache.hadoop.io.DataOutputByteBuffer; +import org.apache.hadoop.io.WritableUtils; +import org.junit.Test; +import static org.junit.Assert.*; + +/** + * Test the {@link TaskID} class. + */ +public class TestTaskID { + /** + * Test of getJobID method, of class TaskID. + */ + @Test + public void testGetJobID() { + JobID jobId = new JobID("1234", 0); + TaskID taskId = new TaskID(jobId, TaskType.MAP, 0); + + assertSame("TaskID did not store the JobID correctly", + jobId, taskId.getJobID()); + + taskId = new TaskID(); + + assertEquals("Job ID was set unexpectedly in default contsructor", + "", taskId.getJobID().getJtIdentifier()); + } + + /** + * Test of isMap method, of class TaskID. + */ + @Test + public void testIsMap() { + JobID jobId = new JobID("1234", 0); + + for (TaskType type : TaskType.values()) { + TaskID taskId = new TaskID(jobId, type, 0); + + if (type == TaskType.MAP) { + assertTrue("TaskID for map task did not correctly identify itself " + + "as a map task", taskId.isMap()); + } else { + assertFalse("TaskID for " + type + " task incorrectly identified " + + "itself as a map task", taskId.isMap()); + } + } + + TaskID taskId = new TaskID(); + + assertFalse("TaskID of default type incorrectly identified itself as a " + + "map task", taskId.isMap()); + } + + /** + * Test of getTaskType method, of class TaskID. + */ + @Test + public void testGetTaskType_0args() { + JobID jobId = new JobID("1234", 0); + + for (TaskType type : TaskType.values()) { + TaskID taskId = new TaskID(jobId, type, 0); + + assertEquals("TaskID incorrectly reported its type", + type, taskId.getTaskType()); + } + + TaskID taskId = new TaskID(); + + assertEquals("TaskID of default type incorrectly reported its type", + TaskType.REDUCE, taskId.getTaskType()); + } + + /** + * Test of equals method, of class TaskID. + */ + @Test + public void testEquals() { + JobID jobId1 = new JobID("1234", 1); + JobID jobId2 = new JobID("2345", 2); + TaskID taskId1 = new TaskID(jobId1, TaskType.MAP, 0); + TaskID taskId2 = new TaskID(jobId1, TaskType.MAP, 0); + + assertTrue("The equals() method reported two equal task IDs were not equal", + taskId1.equals(taskId2)); + + taskId2 = new TaskID(jobId2, TaskType.MAP, 0); + + assertFalse("The equals() method reported two task IDs with different " + + "job IDs were equal", taskId1.equals(taskId2)); + + taskId2 = new TaskID(jobId1, TaskType.MAP, 1); + + assertFalse("The equals() method reported two task IDs with different IDs " + + "were equal", taskId1.equals(taskId2)); + + TaskType[] types = TaskType.values(); + + for (int i = 0; i < types.length; i++) { + for (int j = 0; j < types.length; j++) { + taskId1 = new TaskID(jobId1, types[i], 0); + taskId2 = new TaskID(jobId1, types[j], 0); + + if (i == j) { + assertTrue("The equals() method reported two equal task IDs were not " + + "equal", taskId1.equals(taskId2)); + } else { + assertFalse("The equals() method reported two task IDs with " + + "different types were equal", taskId1.equals(taskId2)); + } + } + } + + assertFalse("The equals() method matched against a JobID object", + taskId1.equals(jobId1)); + + assertFalse("The equals() method matched against a null object", + taskId1.equals(null)); + } + + /** + * Test of compareTo method, of class TaskID. + */ + @Test + public void testCompareTo() { + JobID jobId = new JobID("1234", 1); + TaskID taskId1 = new TaskID(jobId, TaskType.REDUCE, 0); + TaskID taskId2 = new TaskID(jobId, TaskType.REDUCE, 0); + + assertEquals("The compareTo() method returned non-zero for two equal " + + "task IDs", 0, taskId1.compareTo(taskId2)); + + taskId2 = new TaskID(jobId, TaskType.MAP, 1); + + assertTrue("The compareTo() method did not weigh task type more than task " + + "ID", taskId1.compareTo(taskId2) > 0); + + TaskType[] types = TaskType.values(); + + for (int i = 0; i < types.length; i++) { + for (int j = 0; j < types.length; j++) { + taskId1 = new TaskID(jobId, types[i], 0); + taskId2 = new TaskID(jobId, types[j], 0); + + if (i == j) { + assertEquals("The compareTo() method returned non-zero for two equal " + + "task IDs", 0, taskId1.compareTo(taskId2)); + } else if (i < j) { + assertTrue("The compareTo() method did not order " + types[i] + + " before " + types[j], taskId1.compareTo(taskId2) < 0); + } else { + assertTrue("The compareTo() method did not order " + types[i] + + " after " + types[j], taskId1.compareTo(taskId2) > 0); + } + } + } + + try { + taskId1.compareTo(jobId); + fail("The compareTo() method allowed comparison to a JobID object"); + } catch (ClassCastException ex) { + // Expected + } + + try { + taskId1.compareTo(null); + fail("The compareTo() method allowed comparison to a null object"); + } catch (NullPointerException ex) { + // Expected + } + } + + /** + * Test of toString method, of class TaskID. + */ + @Test + public void testToString() { + JobID jobId = new JobID("1234", 1); + + for (TaskType type : TaskType.values()) { + TaskID taskId = new TaskID(jobId, type, 0); + String str = String.format("task_1234_0001_%c_000000", + TaskID.getRepresentingCharacter(type)); + + assertEquals("The toString() method returned the wrong value", + str, taskId.toString()); + } + } + + /** + * Test of appendTo method, of class TaskID. + */ + @Test + public void testAppendTo() { + JobID jobId = new JobID("1234", 1); + StringBuilder builder = new StringBuilder(); + + for (TaskType type : TaskType.values()) { + builder.setLength(0); + TaskID taskId = new TaskID(jobId, type, 0); + String str = String.format("_1234_0001_%c_000000", + TaskID.getRepresentingCharacter(type)); + + assertEquals("The appendTo() method appended the wrong value", + str, taskId.appendTo(builder).toString()); + } + + try { + new TaskID().appendTo(null); + fail("The appendTo() method allowed a null builder"); + } catch (NullPointerException ex) { + // Expected + } + } + + /** + * Test of hashCode method, of class TaskID. + */ + @Test + public void testHashCode() { + TaskType[] types = TaskType.values(); + + for (int i = 0; i < types.length; i++) { + JobID jobId = new JobID("1234" + i, i); + TaskID taskId1 = new TaskID(jobId, types[i], i); + TaskID taskId2 = new TaskID(jobId, types[i], i); + + assertTrue("The hashcode() method gave unequal hash codes for two equal " + + "task IDs", taskId1.hashCode() == taskId2.hashCode()); + } + } + + /** + * Test of readFields method, of class TaskID. + */ + @Test + public void testReadFields() throws Exception { + DataOutputByteBuffer out = new DataOutputByteBuffer(); + + out.writeInt(0); + out.writeInt(1); + WritableUtils.writeVInt(out, 4); + out.write(new byte[] { 0x31, 0x32, 0x33, 0x34}); + WritableUtils.writeEnum(out, TaskType.REDUCE); + + DataInputByteBuffer in = new DataInputByteBuffer(); + + in.reset(out.getData()); + + TaskID instance = new TaskID(); + + instance.readFields(in); + + assertEquals("The readFields() method did not produce the expected task ID", + "task_1234_0001_r_000000", instance.toString()); + } + + /** + * Test of write method, of class TaskID. + */ + @Test + public void testWrite() throws Exception { + JobID jobId = new JobID("1234", 1); + TaskID taskId = new TaskID(jobId, TaskType.JOB_SETUP, 0); + DataOutputByteBuffer out = new DataOutputByteBuffer(); + + taskId.write(out); + + DataInputByteBuffer in = new DataInputByteBuffer(); + byte[] buffer = new byte[4]; + + in.reset(out.getData()); + + assertEquals("The write() method did not write the expected task ID", + 0, in.readInt()); + assertEquals("The write() method did not write the expected job ID", + 1, in.readInt()); + assertEquals("The write() method did not write the expected job " + + "identifier length", 4, WritableUtils.readVInt(in)); + in.readFully(buffer, 0, 4); + assertEquals("The write() method did not write the expected job " + + "identifier length", "1234", new String(buffer)); + assertEquals("The write() method did not write the expected task type", + TaskType.JOB_SETUP, WritableUtils.readEnum(in, TaskType.class)); + } + + /** + * Test of forName method, of class TaskID. + */ + @Test + public void testForName() { + assertEquals("The forName() method did not parse the task ID string " + + "correctly", "task_1_0001_m_000000", + TaskID.forName("task_1_0001_m_000").toString()); + assertEquals("The forName() method did not parse the task ID string " + + "correctly", "task_23_0002_r_000001", + TaskID.forName("task_23_0002_r_0001").toString()); + assertEquals("The forName() method did not parse the task ID string " + + "correctly", "task_345_0003_s_000002", + TaskID.forName("task_345_0003_s_00002").toString()); + assertEquals("The forName() method did not parse the task ID string " + + "correctly", "task_6789_0004_c_000003", + TaskID.forName("task_6789_0004_c_000003").toString()); + assertEquals("The forName() method did not parse the task ID string " + + "correctly", "task_12345_0005_t_4000000", + TaskID.forName("task_12345_0005_t_4000000").toString()); + + try { + TaskID.forName("tisk_12345_0005_t_4000000"); + fail("The forName() method parsed an invalid job ID: " + + "tisk_12345_0005_t_4000000"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("tisk_12345_0005_t_4000000"); + fail("The forName() method parsed an invalid job ID: " + + "tisk_12345_0005_t_4000000"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("task_abc_0005_t_4000000"); + fail("The forName() method parsed an invalid job ID: " + + "task_abc_0005_t_4000000"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("task_12345_xyz_t_4000000"); + fail("The forName() method parsed an invalid job ID: " + + "task_12345_xyz_t_4000000"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("task_12345_0005_x_4000000"); + fail("The forName() method parsed an invalid job ID: " + + "task_12345_0005_x_4000000"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("task_12345_0005_t_jkl"); + fail("The forName() method parsed an invalid job ID: " + + "task_12345_0005_t_jkl"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("task_12345_0005_t"); + fail("The forName() method parsed an invalid job ID: " + + "task_12345_0005_t"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("task_12345_0005_4000000"); + fail("The forName() method parsed an invalid job ID: " + + "task_12345_0005_4000000"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("task_12345_t_4000000"); + fail("The forName() method parsed an invalid job ID: " + + "task_12345_t_4000000"); + } catch (IllegalArgumentException ex) { + // Expected + } + + try { + TaskID.forName("12345_0005_t_4000000"); + fail("The forName() method parsed an invalid job ID: " + + "12345_0005_t_4000000"); + } catch (IllegalArgumentException ex) { + // Expected + } + } + + /** + * Test of getRepresentingCharacter method, of class TaskID. + */ + @Test + public void testGetRepresentingCharacter() { + assertEquals("The getRepresentingCharacter() method did not return the " + + "expected character", 'm', + TaskID.getRepresentingCharacter(TaskType.MAP)); + assertEquals("The getRepresentingCharacter() method did not return the " + + "expected character", 'r', + TaskID.getRepresentingCharacter(TaskType.REDUCE)); + assertEquals("The getRepresentingCharacter() method did not return the " + + "expected character", 's', + TaskID.getRepresentingCharacter(TaskType.JOB_SETUP)); + assertEquals("The getRepresentingCharacter() method did not return the " + + "expected character", 'c', + TaskID.getRepresentingCharacter(TaskType.JOB_CLEANUP)); + assertEquals("The getRepresentingCharacter() method did not return the " + + "expected character", 't', + TaskID.getRepresentingCharacter(TaskType.TASK_CLEANUP)); + } + + /** + * Test of getTaskType method, of class TaskID. + */ + @Test + public void testGetTaskType_char() { + assertEquals("The getTaskType() method did not return the expected type", + TaskType.MAP, + TaskID.getTaskType('m')); + assertEquals("The getTaskType() method did not return the expected type", + TaskType.REDUCE, + TaskID.getTaskType('r')); + assertEquals("The getTaskType() method did not return the expected type", + TaskType.JOB_SETUP, + TaskID.getTaskType('s')); + assertEquals("The getTaskType() method did not return the expected type", + TaskType.JOB_CLEANUP, + TaskID.getTaskType('c')); + assertEquals("The getTaskType() method did not return the expected type", + TaskType.TASK_CLEANUP, + TaskID.getTaskType('t')); + assertNull("The getTaskType() method did not return null for an unknown " + + "type", TaskID.getTaskType('x')); + } + + /** + * Test of getAllTaskTypes method, of class TaskID. + */ + @Test + public void testGetAllTaskTypes() { + assertEquals("The getAllTaskTypes method did not return the expected " + + "string", "(m|r|s|c|t)", TaskID.getAllTaskTypes()); + } +} From e8fc81f9c812b0c167411de7f1789a9a433a0d57 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 25 Mar 2016 20:15:49 +0000 Subject: [PATCH 27/43] YARN-4814. ATS 1.5 timelineclient impl call flush after every event write. Contributed by Xuan Gong --- .../hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java index 35d9970ba40..3fa8691a2d1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java @@ -379,6 +379,8 @@ public class FileSystemTimelineWriter extends TimelineWriter{ this.stream = createLogFileStream(fs, logPath); this.jsonGenerator = new JsonFactory().createJsonGenerator(stream); this.jsonGenerator.setPrettyPrinter(new MinimalPrettyPrinter("\n")); + this.jsonGenerator.configure( + JsonGenerator.Feature.FLUSH_PASSED_TO_STREAM, false); this.lastModifiedTime = Time.monotonicNow(); } From d4df7849a5caf749403bd89d29652f69c9c3f5a8 Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Fri, 25 Mar 2016 15:28:53 -0700 Subject: [PATCH 28/43] HADOOP-12962. KMS key names are incorrectly encoded when creating key. Contributed by Xiao Chen. --- .../hadoop/crypto/key/kms/server/KMS.java | 14 +++---- .../hadoop/crypto/key/kms/server/TestKMS.java | 42 +++++++++++++++++++ 2 files changed, 49 insertions(+), 7 deletions(-) diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java index 43b07fec63d..f069fcaebda 100644 --- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java +++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java @@ -41,10 +41,10 @@ import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriBuilder; import java.io.IOException; import java.net.URI; -import java.net.URISyntaxException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.LinkedList; @@ -89,9 +89,9 @@ public class KMS { keyVersion.getVersionName(), null); } - private static URI getKeyURI(String name) throws URISyntaxException { - return new URI(KMSRESTConstants.SERVICE_VERSION + "/" + - KMSRESTConstants.KEY_RESOURCE + "/" + name); + private static URI getKeyURI(String domain, String keyName) { + return UriBuilder.fromPath("{a}/{b}/{c}") + .build(domain, KMSRESTConstants.KEY_RESOURCE, keyName); } @POST @@ -151,9 +151,9 @@ public class KMS { String requestURL = KMSMDCFilter.getURL(); int idx = requestURL.lastIndexOf(KMSRESTConstants.KEYS_RESOURCE); requestURL = requestURL.substring(0, idx); - String keyURL = requestURL + KMSRESTConstants.KEY_RESOURCE + "/" + name; - return Response.created(getKeyURI(name)).type(MediaType.APPLICATION_JSON). - header("Location", keyURL).entity(json).build(); + return Response.created(getKeyURI(KMSRESTConstants.SERVICE_VERSION, name)) + .type(MediaType.APPLICATION_JSON) + .header("Location", getKeyURI(requestURL, name)).entity(json).build(); } @DELETE diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java index 9b75ee11a02..8094ae2e6d2 100644 --- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java +++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java @@ -39,11 +39,15 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AuthorizationException; import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.log4j.Level; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.security.auth.kerberos.KerberosPrincipal; import javax.security.auth.login.AppConfigurationEntry; @@ -69,12 +73,14 @@ import java.util.UUID; import java.util.concurrent.Callable; public class TestKMS { + private static final Logger LOG = LoggerFactory.getLogger(TestKMS.class); @Before public void cleanUp() { // resetting kerberos security Configuration conf = new Configuration(); UserGroupInformation.setConfiguration(conf); + GenericTestUtils.setLogLevel(LOG, Level.INFO); } public static File getTestDir() throws Exception { @@ -380,6 +386,42 @@ public class TestKMS { testStartStop(true, true); } + @Test(timeout = 30000) + public void testSpecialKeyNames() throws Exception { + final String specialKey = "key %^[\n{]}|\"<>\\"; + Configuration conf = new Configuration(); + conf.set("hadoop.security.authentication", "kerberos"); + UserGroupInformation.setConfiguration(conf); + File confDir = getTestDir(); + conf = createBaseKMSConf(confDir); + conf.set(KeyAuthorizationKeyProvider.KEY_ACL + specialKey + ".ALL", "*"); + writeConf(confDir, conf); + + runServer(null, null, confDir, new KMSCallable() { + @Override + public Void call() throws Exception { + Configuration conf = new Configuration(); + URI uri = createKMSUri(getKMSUrl()); + KeyProvider kp = createProvider(uri, conf); + Assert.assertTrue(kp.getKeys().isEmpty()); + Assert.assertEquals(0, kp.getKeysMetadata().length); + + KeyProvider.Options options = new KeyProvider.Options(conf); + options.setCipher("AES/CTR/NoPadding"); + options.setBitLength(128); + options.setDescription("l1"); + LOG.info("Creating key with name '{}'", specialKey); + + KeyProvider.KeyVersion kv0 = kp.createKey(specialKey, options); + Assert.assertNotNull(kv0); + Assert.assertEquals(specialKey, kv0.getName()); + Assert.assertNotNull(kv0.getVersionName()); + Assert.assertNotNull(kv0.getMaterial()); + return null; + } + }); + } + @Test public void testKMSProvider() throws Exception { Configuration conf = new Configuration(); From 00bebb7e58ba6899904e1619d151aa1b2f5b6acd Mon Sep 17 00:00:00 2001 From: Arun Suresh Date: Fri, 25 Mar 2016 15:54:38 -0700 Subject: [PATCH 29/43] YARN-4823. Refactor the nested reservation id field in listReservation to simple string field. (subru via asuresh) --- .../webapp/dao/ReservationIdInfo.java | 64 ------------------- .../webapp/dao/ReservationInfo.java | 7 +- .../webapp/TestRMWebServicesReservation.java | 15 ++--- .../src/site/markdown/ResourceManagerRest.md | 27 ++------ 4 files changed, 16 insertions(+), 97 deletions(-) delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationIdInfo.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationIdInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationIdInfo.java deleted file mode 100644 index 3a2596a1c84..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationIdInfo.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao; - -import org.apache.hadoop.yarn.api.records.ReservationId; - -import javax.xml.bind.annotation.XmlAccessType; -import javax.xml.bind.annotation.XmlAccessorType; -import javax.xml.bind.annotation.XmlElement; -import javax.xml.bind.annotation.XmlRootElement; - -/** - * Simple class that represent a reservation ID. - */ -@XmlRootElement -@XmlAccessorType(XmlAccessType.FIELD) -public class ReservationIdInfo { - @XmlElement(name = "cluster-timestamp") - private long clusterTimestamp; - - @XmlElement(name = "reservation-id") - private long reservationId; - - public ReservationIdInfo() { - this.clusterTimestamp = 0; - this.reservationId = 0; - } - - public ReservationIdInfo(ReservationId reservationId) { - this.clusterTimestamp = reservationId.getClusterTimestamp(); - this.reservationId = reservationId.getId(); - } - - public long getClusterTimestamp() { - return this.clusterTimestamp; - } - - public void setClusterTimestamp(long newClusterTimestamp) { - this.clusterTimestamp = newClusterTimestamp; - } - - public long getReservationId() { - return this.reservationId; - } - - public void setReservationId(long newReservationId) { - this.reservationId = newReservationId; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationInfo.java index 1a31a8b17d8..8b532add22c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationInfo.java @@ -44,7 +44,7 @@ public class ReservationInfo { private List resourceAllocations = new ArrayList<>(); @XmlElement(name = "reservation-id") - private ReservationIdInfo reservationId; + private String reservationId; @XmlElement(name = "reservation-definition") private ReservationDefinitionInfo reservationDefinition; @@ -55,7 +55,6 @@ public class ReservationInfo { user = ""; reservationDefinition = new ReservationDefinitionInfo(); - reservationId = new ReservationIdInfo(); } public ReservationInfo(ReservationAllocationState allocation, boolean @@ -74,7 +73,7 @@ public class ReservationInfo { } } - reservationId = new ReservationIdInfo(allocation.getReservationId()); + reservationId = allocation.getReservationId().toString(); reservationDefinition = new ReservationDefinitionInfo( allocation.getReservationDefinition()); } @@ -95,7 +94,7 @@ public class ReservationInfo { return resourceAllocations; } - public ReservationIdInfo getReservationId() { + public String getReservationId() { return reservationId; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java index 72705d4f4dd..a60cf17dc13 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java @@ -714,9 +714,8 @@ public class TestRMWebServicesReservation extends JerseyTestBase { testRDLHelper(reservations); - long reservationId = reservations.getJSONObject("reservation-id") - .getLong("reservation-id"); - assertEquals(id1.getId(), reservationId); + String reservationId = reservations.getString("reservation-id"); + assertEquals(id1.toString(), reservationId); rm.stop(); } @@ -769,9 +768,8 @@ public class TestRMWebServicesReservation extends JerseyTestBase { testRDLHelper(reservations); - long reservationId = reservations.getJSONObject("reservation-id") - .getLong("reservation-id"); - assertEquals(id1.getId(), reservationId); + String reservationId = reservations.getString("reservation-id"); + assertEquals(id1.toString(), reservationId); assertTrue(reservations.has("resource-allocations")); @@ -804,9 +802,8 @@ public class TestRMWebServicesReservation extends JerseyTestBase { testRDLHelper(reservations); - long reservationId = reservations.getJSONObject("reservation-id") - .getLong("reservation-id"); - assertEquals(id1.getId(), reservationId); + String reservationId = reservations.getString("reservation-id"); + assertEquals(id1.toString(), reservationId); assertTrue(!reservations.has("resource-allocations")); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md index 338778b41a4..c72b7f408a7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md @@ -3199,7 +3199,7 @@ The Cluster Reservation API can be used to list reservations. When listing reser |:---- |:---- |:---- | | acceptance-time | long | Time that the reservation was accepted | | resource-allocations | array of ResourceAllocationInfo(JSON) / zero or more ResourceAllocationInfo objects(XML) | Resource allocation information for the reservation | -| reservation-id | A single ReservationId object | The unique reservation identifier | +| reservation-id | A single ReservationId string | The unique reservation identifier | | reservation-definition | A single ReservationDefinition Object | A set of constraints representing the need for resources over time of a user | | user | string | User who made the reservation | @@ -3211,21 +3211,14 @@ The Cluster Reservation API can be used to list reservations. When listing reser | startTime | long | Start time that the resource is allocated for | | endTime | long | End time that the resource is allocated for | -### elements of the *resource* object +### Elements of the *resource* object | Item | Data Type | Description | |:---- |:---- |:---- | | memory | int | The memory allocated for the reservation allocation | | vCores | int | The number of cores allocated for the reservation allocation | -### Elements of the *reservation-id* object - -| Item | Data Type | Description | -|:---- |:---- |:---- | -| cluster-timestamp | long | Timestamp representing the time the reservation was created | -| reservation-id | long | The id of the reservation that was listed | - -Elements of the *reservation-definition* object +### Elements of the *reservation-definition* object | Item | Data Type | Description | |:---- |:---- |:---- | @@ -3234,14 +3227,14 @@ Elements of the *reservation-definition* object | reservation-name | string | A mnemonic name of the reservaiton (not a valid identifier). | | reservation-requests | object | A list of "stages" or phases of this reservation, each describing resource requirements and duration | -Elements of the *reservation-requests* object +### Elements of the *reservation-requests* object | Item | Data Type | Description | |:---- |:---- |:---- | | reservation-request-interpreter | int | A numeric choice of how to interpret the set of ReservationRequest: 0 is an ANY, 1 for ALL, 2 for ORDER, 3 for ORDER\_NO\_GAP | | reservation-request | object | The description of the resource and time capabilities for a phase/stage of this reservation | -Elements of the *reservation-request* object +### Elements of the *reservation-request* object | Item | Data Type | Description | |:---- |:---- |:---- | @@ -3305,10 +3298,7 @@ Response Body: "endTime": "1465542252000" } ], - "reservation-id": { - "cluster-timestamp": "1455133859510", - "reservation-id": "6" - }, + "reservation-id": "reservation_1458852875788_0002", "reservation-definition": { "arrival": "1465541532000", "deadline": "1465542252000", @@ -3370,10 +3360,7 @@ Response Body: 1465542251000 1465542252000 - - 1455228059846 - 1 - + reservation_1458852875788_0002 1465541532000 1465542252000 From 9a09200a1f5f752e266d4fb8e0c808073080bde8 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 25 Mar 2016 22:56:49 +0000 Subject: [PATCH 30/43] HADOOP-12958. PhantomReference for filesystem statistics can trigger OOM. Contributed by Sangjin Lee --- .../src/main/java/org/apache/hadoop/fs/FileSystem.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index a8a5c6d2dc7..e0ea7ac7368 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -20,7 +20,7 @@ package org.apache.hadoop.fs; import java.io.Closeable; import java.io.FileNotFoundException; import java.io.IOException; -import java.lang.ref.PhantomReference; +import java.lang.ref.WeakReference; import java.lang.ref.ReferenceQueue; import java.net.URI; import java.net.URISyntaxException; @@ -3133,7 +3133,7 @@ public abstract class FileSystem extends Configured implements Closeable { /** * Set of all thread-local data areas. Protected by the Statistics lock. - * The references to the statistics data are kept using phantom references + * The references to the statistics data are kept using weak references * to the associated threads. Proper clean-up is performed by the cleaner * thread when the threads are garbage collected. */ @@ -3186,11 +3186,11 @@ public abstract class FileSystem extends Configured implements Closeable { } /** - * A phantom reference to a thread that also includes the data associated + * A weak reference to a thread that also includes the data associated * with that thread. On the thread being garbage collected, it is enqueued * to the reference queue for clean-up. */ - private class StatisticsDataReference extends PhantomReference { + private class StatisticsDataReference extends WeakReference { private final StatisticsData data; public StatisticsDataReference(StatisticsData data, Thread thread) { From 4fcfea71bfb16295f3a661e712d66351a1edc55e Mon Sep 17 00:00:00 2001 From: Lei Xu Date: Fri, 25 Mar 2016 17:09:12 -0700 Subject: [PATCH 31/43] HDFS-9005. Provide support for upgrade domain script. (Ming Ma via Lei Xu) --- .../protocol/DatanodeAdminProperties.java | 100 +++++++ .../hdfs/util/CombinedHostsFileReader.java | 76 ++++++ .../hdfs/util/CombinedHostsFileWriter.java | 69 +++++ .../CombinedHostFileManager.java | 250 ++++++++++++++++++ .../blockmanagement/HostConfigManager.java | 80 ++++++ .../hdfs/server/blockmanagement/HostSet.java | 114 ++++++++ ...TestUpgradeDomainBlockPlacementPolicy.java | 169 ++++++++++++ .../hadoop/hdfs/util/HostsFileWriter.java | 122 +++++++++ .../util/TestCombinedHostsFileReader.java | 79 ++++++ .../src/test/resources/dfs.hosts.json | 5 + 10 files changed, 1064 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeAdminProperties.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostConfigManager.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostSet.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeAdminProperties.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeAdminProperties.java new file mode 100644 index 00000000000..9f7b98309d1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeAdminProperties.java @@ -0,0 +1,100 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; + +/** + * The class describes the configured admin properties for a datanode. + * + * It is the static configuration specified by administrators via dfsadmin + * command; different from the runtime state. CombinedHostFileManager uses + * the class to deserialize the configurations from json-based file format. + * + * To decommission a node, use AdminStates.DECOMMISSIONED. + */ +public class DatanodeAdminProperties { + private String hostName; + private int port; + private String upgradeDomain; + private AdminStates adminState = AdminStates.NORMAL; + + /** + * Return the host name of the datanode. + * @return the host name of the datanode. + */ + public String getHostName() { + return hostName; + } + + /** + * Set the host name of the datanode. + * @param hostName the host name of the datanode. + */ + public void setHostName(final String hostName) { + this.hostName = hostName; + } + + /** + * Get the port number of the datanode. + * @return the port number of the datanode. + */ + public int getPort() { + return port; + } + + /** + * Set the port number of the datanode. + * @param port the port number of the datanode. + */ + public void setPort(final int port) { + this.port = port; + } + + /** + * Get the upgrade domain of the datanode. + * @return the upgrade domain of the datanode. + */ + public String getUpgradeDomain() { + return upgradeDomain; + } + + /** + * Set the upgrade domain of the datanode. + * @param upgradeDomain the upgrade domain of the datanode. + */ + public void setUpgradeDomain(final String upgradeDomain) { + this.upgradeDomain = upgradeDomain; + } + + /** + * Get the admin state of the datanode. + * @return the admin state of the datanode. + */ + public AdminStates getAdminState() { + return adminState; + } + + /** + * Set the admin state of the datanode. + * @param adminState the admin state of the datanode. + */ + public void setAdminState(final AdminStates adminState) { + this.adminState = adminState; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java new file mode 100644 index 00000000000..33acb91f837 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.util; + +import java.io.FileInputStream; +import java.io.InputStreamReader; +import java.io.IOException; +import java.io.Reader; + +import java.util.Iterator; +import java.util.Set; +import java.util.HashSet; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.codehaus.jackson.JsonFactory; +import org.codehaus.jackson.map.ObjectMapper; + +import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties; + +/** + * Reader support for JSON based datanode configuration, an alternative + * to the exclude/include files configuration. + * The JSON file format is the array of elements where each element + * in the array describes the properties of a datanode. The properties of + * a datanode is defined in {@link DatanodeAdminProperties}. For example, + * + * {"hostName": "host1"} + * {"hostName": "host2", "port": 50, "upgradeDomain": "ud0"} + * {"hostName": "host3", "port": 0, "adminState": "DECOMMISSIONED"} + */ +@InterfaceAudience.LimitedPrivate({"HDFS"}) +@InterfaceStability.Unstable +public final class CombinedHostsFileReader { + private CombinedHostsFileReader() { + } + + /** + * Deserialize a set of DatanodeAdminProperties from a json file. + * @param hostsFile the input json file to read from. + * @return the set of DatanodeAdminProperties + * @throws IOException + */ + public static Set + readFile(final String hostsFile) throws IOException { + HashSet allDNs = new HashSet<>(); + ObjectMapper mapper = new ObjectMapper(); + try (Reader input = + new InputStreamReader(new FileInputStream(hostsFile), "UTF-8")) { + Iterator iterator = + mapper.readValues(new JsonFactory().createJsonParser(input), + DatanodeAdminProperties.class); + while (iterator.hasNext()) { + DatanodeAdminProperties properties = iterator.next(); + allDNs.add(properties); + } + } + return allDNs; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java new file mode 100644 index 00000000000..ea70be2eb70 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.util; + +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.Writer; + +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.codehaus.jackson.map.ObjectMapper; + +import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties; + +/** + * Writer support for JSON based datanode configuration, an alternative + * to the exclude/include files configuration. + * The JSON file format is the array of elements where each element + * in the array describes the properties of a datanode. The properties of + * a datanode is defined in {@link DatanodeAdminProperties}. For example, + * + * {"hostName": "host1"} + * {"hostName": "host2", "port": 50, "upgradeDomain": "ud0"} + * {"hostName": "host3", "port": 0, "adminState": "DECOMMISSIONED"} + */ +@InterfaceAudience.LimitedPrivate({"HDFS"}) +@InterfaceStability.Unstable +public final class CombinedHostsFileWriter { + private CombinedHostsFileWriter() { + } + + /** + * Serialize a set of DatanodeAdminProperties to a json file. + * @param hostsFile the json file name. + * @param allDNs the set of DatanodeAdminProperties + * @throws IOException + */ + public static void writeFile(final String hostsFile, + final Set allDNs) throws IOException { + StringBuilder configs = new StringBuilder(); + try (Writer output = + new OutputStreamWriter(new FileOutputStream(hostsFile), "UTF-8")) { + for (DatanodeAdminProperties datanodeAdminProperties: allDNs) { + ObjectMapper mapper = new ObjectMapper(); + configs.append(mapper.writeValueAsString(datanodeAdminProperties)); + } + output.write(configs.toString()); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java new file mode 100644 index 00000000000..3e913b93a25 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java @@ -0,0 +1,250 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.blockmanagement; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; +import com.google.common.collect.UnmodifiableIterator; +import com.google.common.collect.Iterables; +import com.google.common.collect.Collections2; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.protocol.DatanodeID; +import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; + +import java.io.IOException; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +import com.google.common.base.Predicate; + +import org.apache.hadoop.hdfs.util.CombinedHostsFileReader; + +/** + * This class manages datanode configuration using a json file. + * Please refer to {@link CombinedHostsFileReader} for the json format. + *

+ *

+ * Entries may or may not specify a port. If they don't, we consider + * them to apply to every DataNode on that host. The code canonicalizes the + * entries into IP addresses. + *

+ *

+ * The code ignores all entries that the DNS fails to resolve their IP + * addresses. This is okay because by default the NN rejects the registrations + * of DNs when it fails to do a forward and reverse lookup. Note that DNS + * resolutions are only done during the loading time to minimize the latency. + */ +public class CombinedHostFileManager extends HostConfigManager { + private static final Logger LOG = LoggerFactory.getLogger( + CombinedHostFileManager.class); + private Configuration conf; + private HostProperties hostProperties = new HostProperties(); + + static class HostProperties { + private Multimap allDNs = + HashMultimap.create(); + // optimization. If every node in the file isn't in service, it implies + // any node is allowed to register with nn. This is equivalent to having + // an empty "include" file. + private boolean emptyInServiceNodeLists = true; + synchronized void add(InetAddress addr, + DatanodeAdminProperties properties) { + allDNs.put(addr, properties); + if (properties.getAdminState().equals( + AdminStates.NORMAL)) { + emptyInServiceNodeLists = false; + } + } + + // If the includes list is empty, act as if everything is in the + // includes list. + synchronized boolean isIncluded(final InetSocketAddress address) { + return emptyInServiceNodeLists || Iterables.any( + allDNs.get(address.getAddress()), + new Predicate() { + public boolean apply(DatanodeAdminProperties input) { + return input.getPort() == 0 || + input.getPort() == address.getPort(); + } + }); + } + + synchronized boolean isExcluded(final InetSocketAddress address) { + return Iterables.any(allDNs.get(address.getAddress()), + new Predicate() { + public boolean apply(DatanodeAdminProperties input) { + return input.getAdminState().equals( + AdminStates.DECOMMISSIONED) && + (input.getPort() == 0 || + input.getPort() == address.getPort()); + } + }); + } + + synchronized String getUpgradeDomain(final InetSocketAddress address) { + Iterable datanode = Iterables.filter( + allDNs.get(address.getAddress()), + new Predicate() { + public boolean apply(DatanodeAdminProperties input) { + return (input.getPort() == 0 || + input.getPort() == address.getPort()); + } + }); + return datanode.iterator().hasNext() ? + datanode.iterator().next().getUpgradeDomain() : null; + } + + Iterable getIncludes() { + return new Iterable() { + @Override + public Iterator iterator() { + return new HostIterator(allDNs.entries()); + } + }; + } + + Iterable getExcludes() { + return new Iterable() { + @Override + public Iterator iterator() { + return new HostIterator( + Collections2.filter(allDNs.entries(), + new Predicate>() { + public boolean apply(java.util.Map.Entry entry) { + return entry.getValue().getAdminState().equals( + AdminStates.DECOMMISSIONED); + } + } + )); + } + }; + } + + static class HostIterator extends UnmodifiableIterator { + private final Iterator> it; + public HostIterator(Collection> nodes) { + this.it = nodes.iterator(); + } + @Override + public boolean hasNext() { + return it.hasNext(); + } + + @Override + public InetSocketAddress next() { + Map.Entry e = it.next(); + return new InetSocketAddress(e.getKey(), e.getValue().getPort()); + } + } + } + + @Override + public Iterable getIncludes() { + return hostProperties.getIncludes(); + } + + @Override + public Iterable getExcludes() { + return hostProperties.getExcludes(); + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void refresh() throws IOException { + refresh(conf.get(DFSConfigKeys.DFS_HOSTS, "")); + } + private void refresh(final String hostsFile) throws IOException { + HostProperties hostProps = new HostProperties(); + Set all = + CombinedHostsFileReader.readFile(hostsFile); + for(DatanodeAdminProperties properties : all) { + InetSocketAddress addr = parseEntry(hostsFile, + properties.getHostName(), properties.getPort()); + if (addr != null) { + hostProps.add(addr.getAddress(), properties); + } + } + refresh(hostProps); + } + + @VisibleForTesting + static InetSocketAddress parseEntry(final String fn, final String hostName, + final int port) { + InetSocketAddress addr = new InetSocketAddress(hostName, port); + if (addr.isUnresolved()) { + LOG.warn("Failed to resolve {} in {}. ", hostName, fn); + return null; + } + return addr; + } + + @Override + public synchronized boolean isIncluded(final DatanodeID dn) { + return hostProperties.isIncluded(dn.getResolvedAddress()); + } + + @Override + public synchronized boolean isExcluded(final DatanodeID dn) { + return isExcluded(dn.getResolvedAddress()); + } + + private boolean isExcluded(final InetSocketAddress address) { + return hostProperties.isExcluded(address); + } + + @Override + public synchronized String getUpgradeDomain(final DatanodeID dn) { + return hostProperties.getUpgradeDomain(dn.getResolvedAddress()); + } + + /** + * Set the properties lists by the new instances. The + * old instance is discarded. + * @param hostProperties the new properties list + */ + @VisibleForTesting + private void refresh(final HostProperties hostProperties) { + synchronized (this) { + this.hostProperties = hostProperties; + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostConfigManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostConfigManager.java new file mode 100644 index 00000000000..f28ed2997a1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostConfigManager.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.blockmanagement; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.hdfs.protocol.DatanodeID; + +import java.io.IOException; +import java.net.InetSocketAddress; + +/** + * This interface abstracts how datanode configuration is managed. + * + * Each implementation defines its own way to persist the configuration. + * For example, it can use one JSON file to store the configs for all + * datanodes; or it can use one file to store in-service datanodes and another + * file to store decommission-requested datanodes. + * + * These files control which DataNodes the NameNode expects to see in the + * cluster. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public abstract class HostConfigManager implements Configurable { + + /** + * Return all the datanodes that are allowed to connect to the namenode. + * @return Iterable of all datanodes + */ + public abstract Iterable getIncludes(); + + /** + * Return all datanodes that should be in decommissioned state. + * @return Iterable of those datanodes + */ + public abstract Iterable getExcludes(); + + /** + * Check if a datanode is allowed to connect the namenode. + * @param dn the DatanodeID of the datanode + * @return boolean if dn is allowed to connect the namenode. + */ + public abstract boolean isIncluded(DatanodeID dn); + + /** + * Check if a datanode needs to be decommissioned. + * @param dn the DatanodeID of the datanode + * @return boolean if dn needs to be decommissioned. + */ + public abstract boolean isExcluded(DatanodeID dn); + + /** + * Reload the configuration. + */ + public abstract void refresh() throws IOException; + + /** + * Get the upgrade domain of a datanode. + * @param dn the DatanodeID of the datanode + * @return the upgrade domain of dn. + */ + public abstract String getUpgradeDomain(DatanodeID dn); +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostSet.java new file mode 100644 index 00000000000..958557b4f8a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostSet.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.blockmanagement; + +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Iterators; +import com.google.common.collect.Multimap; +import com.google.common.collect.UnmodifiableIterator; + +import javax.annotation.Nullable; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; + + +/** + * The HostSet allows efficient queries on matching wildcard addresses. + *

+ * For InetSocketAddress A and B with the same host address, + * we define a partial order between A and B, A <= B iff A.getPort() == B + * .getPort() || B.getPort() == 0. + */ +public class HostSet implements Iterable { + // Host -> lists of ports + private final Multimap addrs = HashMultimap.create(); + + /** + * The function that checks whether there exists an entry foo in the set + * so that foo <= addr. + */ + boolean matchedBy(InetSocketAddress addr) { + Collection ports = addrs.get(addr.getAddress()); + return addr.getPort() == 0 ? !ports.isEmpty() : ports.contains(addr + .getPort()); + } + + /** + * The function that checks whether there exists an entry foo in the set + * so that addr <= foo. + */ + boolean match(InetSocketAddress addr) { + int port = addr.getPort(); + Collection ports = addrs.get(addr.getAddress()); + boolean exactMatch = ports.contains(port); + boolean genericMatch = ports.contains(0); + return exactMatch || genericMatch; + } + + boolean isEmpty() { + return addrs.isEmpty(); + } + + int size() { + return addrs.size(); + } + + void add(InetSocketAddress addr) { + Preconditions.checkArgument(!addr.isUnresolved()); + addrs.put(addr.getAddress(), addr.getPort()); + } + + @Override + public Iterator iterator() { + return new UnmodifiableIterator() { + private final Iterator> it = addrs.entries().iterator(); + + @Override + public boolean hasNext() { + return it.hasNext(); + } + + @Override + public InetSocketAddress next() { + Map.Entry e = it.next(); + return new InetSocketAddress(e.getKey(), e.getValue()); + } + }; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("HostSet("); + Joiner.on(",").appendTo(sb, Iterators.transform(iterator(), + new Function() { + @Override + public String apply(@Nullable InetSocketAddress addr) { + assert addr != null; + return addr.getAddress().getHostAddress() + ":" + addr.getPort(); + } + })); + return sb.append(")").toString(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java new file mode 100644 index 00000000000..cc14fcbb04b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java @@ -0,0 +1,169 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.permission.PermissionStatus; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties; +import org.apache.hadoop.hdfs.protocol.DatanodeID; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithUpgradeDomain; +import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager; +import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager; +import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; +import org.apache.hadoop.hdfs.util.HostsFileWriter; +import org.apache.hadoop.net.StaticMapping; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * End-to-end test case for upgrade domain + * The test configs upgrade domain for nodes via admin json + * config file and put some nodes to decommission state. + * The test then verifies replicas are placed on the nodes that + * satisfy the upgrade domain policy. + * + */ +public class TestUpgradeDomainBlockPlacementPolicy { + + private static final short REPLICATION_FACTOR = (short) 3; + private static final int DEFAULT_BLOCK_SIZE = 1024; + static final String[] racks = + { "/RACK1", "/RACK1", "/RACK1", "/RACK2", "/RACK2", "/RACK2" }; + /** + * Use host names that can be resolved ( + * InetSocketAddress#isUnresolved == false). Otherwise, + * CombinedHostFileManager won't allow those hosts. + */ + static final String[] hosts = + { "127.0.0.1", "127.0.0.1", "127.0.0.1", "127.0.0.1", + "127.0.0.1", "127.0.0.1" }; + static final String[] upgradeDomains = + { "ud1", "ud2", "ud3", "ud1", "ud2", "ud3" }; + static final Set expectedDatanodeIDs = new HashSet<>(); + private MiniDFSCluster cluster = null; + private NamenodeProtocols nameNodeRpc = null; + private FSNamesystem namesystem = null; + private PermissionStatus perm = null; + + @Before + public void setup() throws IOException { + StaticMapping.resetMap(); + Configuration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE); + conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE / 2); + conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, + BlockPlacementPolicyWithUpgradeDomain.class, + BlockPlacementPolicy.class); + conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY, + CombinedHostFileManager.class, HostConfigManager.class); + HostsFileWriter hostsFileWriter = new HostsFileWriter(); + hostsFileWriter.initialize(conf, "temp/upgradedomainpolicy"); + + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(6).racks(racks) + .hosts(hosts).build(); + cluster.waitActive(); + nameNodeRpc = cluster.getNameNodeRpc(); + namesystem = cluster.getNamesystem(); + perm = new PermissionStatus("TestDefaultBlockPlacementPolicy", null, + FsPermission.getDefault()); + refreshDatanodeAdminProperties(hostsFileWriter); + hostsFileWriter.cleanup(); + } + + @After + public void teardown() { + if (cluster != null) { + cluster.shutdown(); + cluster = null; + } + } + + /** + * Define admin properties for these datanodes as follows. + * dn0 and dn3 have upgrade domain ud1. + * dn1 and dn4 have upgrade domain ud2. + * dn2 and dn5 have upgrade domain ud3. + * dn0 and dn5 are decommissioned. + * Given dn0, dn1 and dn2 are on rack1 and dn3, dn4 and dn5 are on + * rack2. Then any block's replicas should be on either + * {dn1, dn2, d3} or {dn2, dn3, dn4}. + */ + private void refreshDatanodeAdminProperties(HostsFileWriter hostsFileWriter) + throws IOException { + DatanodeAdminProperties[] datanodes = new DatanodeAdminProperties[ + hosts.length]; + for (int i = 0; i < hosts.length; i++) { + datanodes[i] = new DatanodeAdminProperties(); + DatanodeID datanodeID = cluster.getDataNodes().get(i).getDatanodeId(); + datanodes[i].setHostName(datanodeID.getHostName()); + datanodes[i].setPort(datanodeID.getXferPort()); + datanodes[i].setUpgradeDomain(upgradeDomains[i]); + } + datanodes[0].setAdminState(DatanodeInfo.AdminStates.DECOMMISSIONED); + datanodes[5].setAdminState(DatanodeInfo.AdminStates.DECOMMISSIONED); + hostsFileWriter.initIncludeHosts(datanodes); + cluster.getFileSystem().refreshNodes(); + + expectedDatanodeIDs.add(cluster.getDataNodes().get(2).getDatanodeId()); + expectedDatanodeIDs.add(cluster.getDataNodes().get(3).getDatanodeId()); + } + + @Test + public void testPlacement() throws Exception { + String clientMachine = "127.0.0.1"; + for (int i = 0; i < 5; i++) { + String src = "/test-" + i; + // Create the file with client machine + HdfsFileStatus fileStatus = namesystem.startFile(src, perm, + clientMachine, clientMachine, EnumSet.of(CreateFlag.CREATE), true, + REPLICATION_FACTOR, DEFAULT_BLOCK_SIZE, null, false); + LocatedBlock locatedBlock = nameNodeRpc.addBlock(src, clientMachine, + null, null, fileStatus.getFileId(), null); + + assertEquals("Block should be allocated sufficient locations", + REPLICATION_FACTOR, locatedBlock.getLocations().length); + Set locs = new HashSet<>(Arrays.asList( + locatedBlock.getLocations())); + for (DatanodeID datanodeID : expectedDatanodeIDs) { + locs.contains(datanodeID); + } + + nameNodeRpc.abandonBlock(locatedBlock.getBlock(), fileStatus.getFileId(), + src, clientMachine); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java new file mode 100644 index 00000000000..cd5ae954971 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java @@ -0,0 +1,122 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.util; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; + + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager; +import org.apache.hadoop.hdfs.server.blockmanagement.HostFileManager; + +import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; + +import static org.junit.Assert.assertTrue; + +public class HostsFileWriter { + private FileSystem localFileSys; + private Path fullDir; + private Path excludeFile; + private Path includeFile; + private Path combinedFile; + private boolean isLegacyHostsFile = false; + + public void initialize(Configuration conf, String dir) throws IOException { + localFileSys = FileSystem.getLocal(conf); + Path workingDir = new Path(MiniDFSCluster.getBaseDirectory()); + this.fullDir = new Path(workingDir, dir); + assertTrue(localFileSys.mkdirs(this.fullDir)); + + if (conf.getClass( + DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY, + HostFileManager.class, HostConfigManager.class).equals( + HostFileManager.class)) { + isLegacyHostsFile = true; + } + if (isLegacyHostsFile) { + excludeFile = new Path(fullDir, "exclude"); + includeFile = new Path(fullDir, "include"); + DFSTestUtil.writeFile(localFileSys, excludeFile, ""); + DFSTestUtil.writeFile(localFileSys, includeFile, ""); + conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath()); + conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath()); + } else { + combinedFile = new Path(fullDir, "all"); + conf.set(DFSConfigKeys.DFS_HOSTS, combinedFile.toString()); + } + } + + public void initExcludeHost(String hostNameAndPort) throws IOException { + if (isLegacyHostsFile) { + DFSTestUtil.writeFile(localFileSys, excludeFile, hostNameAndPort); + } else { + DatanodeAdminProperties dn = new DatanodeAdminProperties(); + String [] hostAndPort = hostNameAndPort.split(":"); + dn.setHostName(hostAndPort[0]); + dn.setPort(Integer.parseInt(hostAndPort[1])); + dn.setAdminState(AdminStates.DECOMMISSIONED); + HashSet allDNs = new HashSet<>(); + allDNs.add(dn); + CombinedHostsFileWriter.writeFile(combinedFile.toString(), allDNs); + } + } + + public void initIncludeHosts(String[] hostNameAndPorts) throws IOException { + StringBuilder includeHosts = new StringBuilder(); + if (isLegacyHostsFile) { + for(String hostNameAndPort : hostNameAndPorts) { + includeHosts.append(hostNameAndPort).append("\n"); + } + DFSTestUtil.writeFile(localFileSys, includeFile, + includeHosts.toString()); + } else { + HashSet allDNs = new HashSet<>(); + for(String hostNameAndPort : hostNameAndPorts) { + String[] hostAndPort = hostNameAndPort.split(":"); + DatanodeAdminProperties dn = new DatanodeAdminProperties(); + dn.setHostName(hostAndPort[0]); + dn.setPort(Integer.parseInt(hostAndPort[1])); + allDNs.add(dn); + } + CombinedHostsFileWriter.writeFile(combinedFile.toString(), allDNs); + } + } + + public void initIncludeHosts(DatanodeAdminProperties[] datanodes) + throws IOException { + CombinedHostsFileWriter.writeFile(combinedFile.toString(), + new HashSet<>(Arrays.asList(datanodes))); + } + + public void cleanup() throws IOException { + if (localFileSys.exists(fullDir)) { + FileUtils.deleteQuietly(new File(fullDir.toUri().getPath())); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java new file mode 100644 index 00000000000..c3946e412b2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.util; + +import java.io.File; +import java.io.FileWriter; + +import java.util.Set; + +import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties; +import org.junit.Before; +import org.junit.After; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/* + * Test for JSON based HostsFileReader + */ +public class TestCombinedHostsFileReader { + + // Using /test/build/data/tmp directory to store temporary files + static final String HOSTS_TEST_DIR = new File(System.getProperty( + "test.build.data", "/tmp")).getAbsolutePath(); + File NEW_FILE = new File(HOSTS_TEST_DIR, "dfs.hosts.new.json"); + + static final String TEST_CACHE_DATA_DIR = + System.getProperty("test.cache.data", "build/test/cache"); + File EXISTING_FILE = new File(TEST_CACHE_DATA_DIR, "dfs.hosts.json"); + + @Before + public void setUp() throws Exception { + } + + @After + public void tearDown() throws Exception { + // Delete test file after running tests + NEW_FILE.delete(); + + } + + /* + * Load the existing test json file + */ + @Test + public void testLoadExistingJsonFile() throws Exception { + Set all = + CombinedHostsFileReader.readFile(EXISTING_FILE.getAbsolutePath()); + assertEquals(5, all.size()); + } + + /* + * Test empty json config file + */ + @Test + public void testEmptyCombinedHostsFileReader() throws Exception { + FileWriter hosts = new FileWriter(NEW_FILE); + hosts.write(""); + hosts.close(); + Set all = + CombinedHostsFileReader.readFile(NEW_FILE.getAbsolutePath()); + assertEquals(0, all.size()); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json new file mode 100644 index 00000000000..64fca48dbff --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json @@ -0,0 +1,5 @@ +{"hostName": "host1"} +{"hostName": "host2", "upgradeDomain": "ud0"} +{"hostName": "host3", "adminState": "DECOMMISSIONED"} +{"hostName": "host4", "upgradeDomain": "ud2", "adminState": "DECOMMISSIONED"} +{"hostName": "host5", "port": 8090} From fde8ac5d8514f5146f438f8d0794116aaef20416 Mon Sep 17 00:00:00 2001 From: Lei Xu Date: Fri, 25 Mar 2016 17:10:31 -0700 Subject: [PATCH 32/43] Add missing files from HDFS-9005. (lei) --- .../hadoop/hdfs/protocol/DatanodeID.java | 6 + .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 4 +- .../blockmanagement/DatanodeManager.java | 59 ++++--- .../blockmanagement/HostFileManager.java | 147 +++++------------- .../src/main/resources/hdfs-default.xml | 14 ++ .../src/site/markdown/HdfsUserGuide.md | 6 +- .../hadoop/hdfs/TestDatanodeReport.java | 57 ++++++- .../TestBlocksWithNotEnoughRacks.java | 34 ++-- .../blockmanagement/TestDatanodeManager.java | 8 +- .../blockmanagement/TestHostFileManager.java | 10 +- .../hdfs/server/namenode/TestHostsFiles.java | 70 ++++----- .../server/namenode/TestNameNodeMXBean.java | 25 ++- .../hdfs/server/namenode/TestStartup.java | 52 +------ 13 files changed, 226 insertions(+), 266 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java index 5fd845d9c73..af720c7d80b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java @@ -23,6 +23,8 @@ import org.apache.hadoop.classification.InterfaceStability; import com.google.common.annotations.VisibleForTesting; +import java.net.InetSocketAddress; + /** * This class represents the primary identifier for a Datanode. * Datanodes are identified by how they can be contacted (hostname @@ -274,4 +276,8 @@ public class DatanodeID implements Comparable { public int compareTo(DatanodeID that) { return getXferAddr().compareTo(that.getXferAddr()); } + + public InetSocketAddress getResolvedAddress() { + return new InetSocketAddress(this.getIpAddr(), this.getXferPort()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 844fec29843..94246625bec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -415,12 +415,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_METRICS_PERCENTILES_INTERVALS_KEY = "dfs.metrics.percentiles.intervals"; public static final String DFS_DATANODE_HOST_NAME_KEY = HdfsClientConfigKeys.DeprecatedKeys.DFS_DATANODE_HOST_NAME_KEY; - public static final String DFS_NAMENODE_HOSTS_KEY = "dfs.namenode.hosts"; - public static final String DFS_NAMENODE_HOSTS_EXCLUDE_KEY = "dfs.namenode.hosts.exclude"; public static final String DFS_NAMENODE_CHECKPOINT_DIR_KEY = HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY; public static final String DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY = HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY; + public static final String DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY = + "dfs.namenode.hosts.provider.classname"; public static final String DFS_HOSTS = "dfs.hosts"; public static final String DFS_HOSTS_EXCLUDE = "dfs.hosts.exclude"; public static final String DFS_NAMENODE_AUDIT_LOGGERS_KEY = "dfs.namenode.audit.loggers"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index 53c7c16862e..cd1bdaba693 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -111,7 +111,7 @@ public class DatanodeManager { private final int defaultIpcPort; /** Read include/exclude files. */ - private final HostFileManager hostFileManager = new HostFileManager(); + private HostConfigManager hostConfigManager; /** The period to wait for datanode heartbeat.*/ private long heartbeatExpireInterval; @@ -204,9 +204,11 @@ public class DatanodeManager { this.defaultIpcPort = NetUtils.createSocketAddr( conf.getTrimmed(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_DEFAULT)).getPort(); + this.hostConfigManager = ReflectionUtils.newInstance( + conf.getClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY, + HostFileManager.class, HostConfigManager.class), conf); try { - this.hostFileManager.refresh(conf.get(DFSConfigKeys.DFS_HOSTS, ""), - conf.get(DFSConfigKeys.DFS_HOSTS_EXCLUDE, "")); + this.hostConfigManager.refresh(); } catch (IOException e) { LOG.error("error reading hosts files: ", e); } @@ -224,7 +226,7 @@ public class DatanodeManager { // in the cache; so future calls to resolve will be fast. if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) { final ArrayList locations = new ArrayList<>(); - for (InetSocketAddress addr : hostFileManager.getIncludes()) { + for (InetSocketAddress addr : hostConfigManager.getIncludes()) { locations.add(addr.getAddress().getHostAddress()); } dnsToSwitchMapping.resolve(locations); @@ -337,8 +339,8 @@ public class DatanodeManager { return decomManager; } - HostFileManager getHostFileManager() { - return hostFileManager; + public HostConfigManager getHostConfigManager() { + return hostConfigManager; } @VisibleForTesting @@ -632,6 +634,7 @@ public class DatanodeManager { networktopology.add(node); // may throw InvalidTopologyException host2DatanodeMap.add(node); checkIfClusterIsNowMultiRack(node); + resolveUpgradeDomain(node); if (LOG.isDebugEnabled()) { LOG.debug(getClass().getSimpleName() + ".addDatanode: " @@ -704,7 +707,14 @@ public class DatanodeManager { return new HashMap<> (this.datanodesSoftwareVersions); } } - + + void resolveUpgradeDomain(DatanodeDescriptor node) { + String upgradeDomain = hostConfigManager.getUpgradeDomain(node); + if (upgradeDomain != null && upgradeDomain.length() > 0) { + node.setUpgradeDomain(upgradeDomain); + } + } + /** * Resolve a node's network location. If the DNS to switch mapping fails * then this method guarantees default rack location. @@ -831,7 +841,7 @@ public class DatanodeManager { */ void startDecommissioningIfExcluded(DatanodeDescriptor nodeReg) { // If the registered node is in exclude list, then decommission it - if (getHostFileManager().isExcluded(nodeReg)) { + if (getHostConfigManager().isExcluded(nodeReg)) { decomManager.startDecommission(nodeReg); } } @@ -871,7 +881,7 @@ public class DatanodeManager { // Checks if the node is not on the hosts list. If it is not, then // it will be disallowed from registering. - if (!hostFileManager.isIncluded(nodeReg)) { + if (!hostConfigManager.isIncluded(nodeReg)) { throw new DisallowedDatanodeException(nodeReg); } @@ -939,7 +949,8 @@ public class DatanodeManager { getNetworkDependenciesWithDefault(nodeS)); } getNetworkTopology().add(nodeS); - + resolveUpgradeDomain(nodeS); + // also treat the registration message as a heartbeat heartbeatManager.register(nodeS); incrementVersionCount(nodeS.getSoftwareVersion()); @@ -971,7 +982,8 @@ public class DatanodeManager { } networktopology.add(nodeDescr); nodeDescr.setSoftwareVersion(nodeReg.getSoftwareVersion()); - + resolveUpgradeDomain(nodeDescr); + // register new datanode addDatanode(nodeDescr); blockManager.getBlockReportLeaseManager().register(nodeDescr); @@ -1026,9 +1038,9 @@ public class DatanodeManager { // Update the file names and refresh internal includes and excludes list. if (conf == null) { conf = new HdfsConfiguration(); + this.hostConfigManager.setConf(conf); } - this.hostFileManager.refresh(conf.get(DFSConfigKeys.DFS_HOSTS, ""), - conf.get(DFSConfigKeys.DFS_HOSTS_EXCLUDE, "")); + this.hostConfigManager.refresh(); } /** @@ -1044,15 +1056,16 @@ public class DatanodeManager { } for (DatanodeDescriptor node : copy.values()) { // Check if not include. - if (!hostFileManager.isIncluded(node)) { + if (!hostConfigManager.isIncluded(node)) { node.setDisallowed(true); // case 2. } else { - if (hostFileManager.isExcluded(node)) { + if (hostConfigManager.isExcluded(node)) { decomManager.startDecommission(node); // case 3. } else { decomManager.stopDecommission(node); // case 4. } } + node.setUpgradeDomain(hostConfigManager.getUpgradeDomain(node)); } } @@ -1268,9 +1281,9 @@ public class DatanodeManager { type == DatanodeReportType.DECOMMISSIONING; ArrayList nodes; - final HostFileManager.HostSet foundNodes = new HostFileManager.HostSet(); - final HostFileManager.HostSet includedNodes = hostFileManager.getIncludes(); - final HostFileManager.HostSet excludedNodes = hostFileManager.getExcludes(); + final HostSet foundNodes = new HostSet(); + final Iterable includedNodes = + hostConfigManager.getIncludes(); synchronized(this) { nodes = new ArrayList<>(datanodeMap.size()); @@ -1281,11 +1294,11 @@ public class DatanodeManager { if (((listLiveNodes && !isDead) || (listDeadNodes && isDead) || (listDecommissioningNodes && isDecommissioning)) && - hostFileManager.isIncluded(dn)) { + hostConfigManager.isIncluded(dn)) { nodes.add(dn); } - foundNodes.add(HostFileManager.resolvedAddressFromDatanodeID(dn)); + foundNodes.add(dn.getResolvedAddress()); } } Collections.sort(nodes); @@ -1309,7 +1322,7 @@ public class DatanodeManager { addr.getPort() == 0 ? defaultXferPort : addr.getPort(), defaultInfoPort, defaultInfoSecurePort, defaultIpcPort)); setDatanodeDead(dn); - if (excludedNodes.match(addr)) { + if (hostConfigManager.isExcluded(dn)) { dn.setDecommissioned(); } nodes.add(dn); @@ -1318,8 +1331,8 @@ public class DatanodeManager { if (LOG.isDebugEnabled()) { LOG.debug("getDatanodeListForReport with " + - "includedNodes = " + hostFileManager.getIncludes() + - ", excludedNodes = " + hostFileManager.getExcludes() + + "includedNodes = " + hostConfigManager.getIncludes() + + ", excludedNodes = " + hostConfigManager.getExcludes() + ", foundNodes = " + foundNodes + ", nodes = " + nodes); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java index e05ef9a4047..bcfebf25de1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java @@ -18,28 +18,18 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Iterators; -import com.google.common.collect.Multimap; -import com.google.common.collect.UnmodifiableIterator; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.util.HostsFileReader; -import javax.annotation.Nullable; import java.io.IOException; -import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; -import java.util.Collection; import java.util.HashSet; -import java.util.Iterator; -import java.util.Map; /** * This class manages the include and exclude files for HDFS. @@ -59,11 +49,27 @@ import java.util.Map; * of DNs when it fails to do a forward and reverse lookup. Note that DNS * resolutions are only done during the loading time to minimize the latency. */ -class HostFileManager { +public class HostFileManager extends HostConfigManager { private static final Log LOG = LogFactory.getLog(HostFileManager.class); + private Configuration conf; private HostSet includes = new HostSet(); private HostSet excludes = new HostSet(); + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void refresh() throws IOException { + refresh(conf.get(DFSConfigKeys.DFS_HOSTS, ""), + conf.get(DFSConfigKeys.DFS_HOSTS_EXCLUDE, "")); + } private static HostSet readFile(String type, String filename) throws IOException { HostSet res = new HostSet(); @@ -99,31 +105,37 @@ class HostFileManager { return null; } - static InetSocketAddress resolvedAddressFromDatanodeID(DatanodeID id) { - return new InetSocketAddress(id.getIpAddr(), id.getXferPort()); - } - - synchronized HostSet getIncludes() { + @Override + public synchronized HostSet getIncludes() { return includes; } - synchronized HostSet getExcludes() { + @Override + public synchronized HostSet getExcludes() { return excludes; } // If the includes list is empty, act as if everything is in the // includes list. - synchronized boolean isIncluded(DatanodeID dn) { - return includes.isEmpty() || includes.match - (resolvedAddressFromDatanodeID(dn)); + @Override + public synchronized boolean isIncluded(DatanodeID dn) { + return includes.isEmpty() || includes.match(dn.getResolvedAddress()); } - synchronized boolean isExcluded(DatanodeID dn) { - return excludes.match(resolvedAddressFromDatanodeID(dn)); + @Override + public synchronized boolean isExcluded(DatanodeID dn) { + return isExcluded(dn.getResolvedAddress()); } - synchronized boolean hasIncludes() { - return !includes.isEmpty(); + private boolean isExcluded(InetSocketAddress address) { + return excludes.match(address); + } + + @Override + public synchronized String getUpgradeDomain(final DatanodeID dn) { + // The include/exclude files based config doesn't support upgrade domain + // config. + return null; } /** @@ -133,7 +145,8 @@ class HostFileManager { * @param excludeFile the path to the new excludes list * @throws IOException thrown if there is a problem reading one of the files */ - void refresh(String includeFile, String excludeFile) throws IOException { + private void refresh(String includeFile, String excludeFile) + throws IOException { HostSet newIncludes = readFile("included", includeFile); HostSet newExcludes = readFile("excluded", excludeFile); @@ -153,84 +166,4 @@ class HostFileManager { excludes = newExcludes; } } - - /** - * The HostSet allows efficient queries on matching wildcard addresses. - *

- * For InetSocketAddress A and B with the same host address, - * we define a partial order between A and B, A <= B iff A.getPort() == B - * .getPort() || B.getPort() == 0. - */ - static class HostSet implements Iterable { - // Host -> lists of ports - private final Multimap addrs = HashMultimap.create(); - - /** - * The function that checks whether there exists an entry foo in the set - * so that foo <= addr. - */ - boolean matchedBy(InetSocketAddress addr) { - Collection ports = addrs.get(addr.getAddress()); - return addr.getPort() == 0 ? !ports.isEmpty() : ports.contains(addr - .getPort()); - } - - /** - * The function that checks whether there exists an entry foo in the set - * so that addr <= foo. - */ - boolean match(InetSocketAddress addr) { - int port = addr.getPort(); - Collection ports = addrs.get(addr.getAddress()); - boolean exactMatch = ports.contains(port); - boolean genericMatch = ports.contains(0); - return exactMatch || genericMatch; - } - - boolean isEmpty() { - return addrs.isEmpty(); - } - - int size() { - return addrs.size(); - } - - void add(InetSocketAddress addr) { - Preconditions.checkArgument(!addr.isUnresolved()); - addrs.put(addr.getAddress(), addr.getPort()); - } - - @Override - public Iterator iterator() { - return new UnmodifiableIterator() { - private final Iterator> it = addrs.entries().iterator(); - - @Override - public boolean hasNext() { - return it.hasNext(); - } - - @Override - public InetSocketAddress next() { - Map.Entry e = it.next(); - return new InetSocketAddress(e.getKey(), e.getValue()); - } - }; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("HostSet("); - Joiner.on(",").appendTo(sb, Iterators.transform(iterator(), - new Function() { - @Override - public String apply(@Nullable InetSocketAddress addr) { - assert addr != null; - return addr.getAddress().getHostAddress() + ":" + addr.getPort(); - } - })); - return sb.append(")").toString(); - } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 1e87626afe7..35dce0e2825 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -3022,4 +3022,18 @@ retries or failovers for WebHDFS client. + + + dfs.namenode.hosts.provider.classname + org.apache.hadoop.hdfs.server.blockmanagement.HostFileManager + + The class that provides access for host files. + org.apache.hadoop.hdfs.server.blockmanagement.HostFileManager is used + by default which loads files specified by dfs.hosts and dfs.hosts.exclude. + If org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager is + used, it will load the JSON file defined in dfs.hosts. + To change class name, nn restart is required. "dfsadmin -refreshNodes" only + refreshes the configuration files used by the class. + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md index 24af67b8513..e85fb53e975 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md @@ -142,12 +142,16 @@ The `bin/hdfs dfsadmin` command supports a few HDFS administration related opera during last upgrade. * `-refreshNodes`: Updates the namenode with the set of datanodes - allowed to connect to the namenode. Namenodes re-read datanode + allowed to connect to the namenode. By default, Namenodes re-read datanode hostnames in the file defined by `dfs.hosts`, `dfs.hosts.exclude` Hosts defined in `dfs.hosts` are the datanodes that are part of the cluster. If there are entries in `dfs.hosts`, only the hosts in it are allowed to register with the namenode. Entries in `dfs.hosts.exclude` are datanodes that need to be decommissioned. + Alternatively if `dfs.namenode.hosts.provider.classname` is set to + `org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager`, + all include and exclude hosts are specified in the JSON file defined by + `dfs.hosts`. Datanodes complete decommissioning when all the replicas from them are replicated to other datanodes. Decommissioned nodes are not automatically shutdown and are not chosen for writing for new diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java index 1e6db21d8af..f23e53183a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java @@ -29,11 +29,16 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; +import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager; +import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager; import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport; +import org.apache.hadoop.hdfs.util.HostsFileWriter; import org.junit.Test; /** @@ -43,7 +48,57 @@ public class TestDatanodeReport { static final Log LOG = LogFactory.getLog(TestDatanodeReport.class); final static private Configuration conf = new HdfsConfiguration(); final static private int NUM_OF_DATANODES = 4; - + + /** + * This test verifies upgrade domain is set according to the JSON host file. + */ + @Test + public void testDatanodeReportWithUpgradeDomain() throws Exception { + conf.setInt( + DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500); // 0.5s + conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); + conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY, + CombinedHostFileManager.class, HostConfigManager.class); + HostsFileWriter hostsFileWriter = new HostsFileWriter(); + hostsFileWriter.initialize(conf, "temp/datanodeReport"); + + MiniDFSCluster cluster = + new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); + final DFSClient client = cluster.getFileSystem().dfs; + final String ud1 = "ud1"; + final String ud2 = "ud2"; + + try { + //wait until the cluster is up + cluster.waitActive(); + + DatanodeAdminProperties datanode = new DatanodeAdminProperties(); + datanode.setHostName(cluster.getDataNodes().get(0).getDatanodeId().getHostName()); + datanode.setUpgradeDomain(ud1); + hostsFileWriter.initIncludeHosts( + new DatanodeAdminProperties[]{datanode}); + client.refreshNodes(); + DatanodeInfo[] all = client.datanodeReport(DatanodeReportType.ALL); + assertEquals(all[0].getUpgradeDomain(), ud1); + + datanode.setUpgradeDomain(null); + hostsFileWriter.initIncludeHosts( + new DatanodeAdminProperties[]{datanode}); + client.refreshNodes(); + all = client.datanodeReport(DatanodeReportType.ALL); + assertEquals(all[0].getUpgradeDomain(), null); + + datanode.setUpgradeDomain(ud2); + hostsFileWriter.initIncludeHosts( + new DatanodeAdminProperties[]{datanode}); + client.refreshNodes(); + all = client.datanodeReport(DatanodeReportType.ALL); + assertEquals(all[0].getUpgradeDomain(), ud2); + } finally { + cluster.shutdown(); + } + } + /** * This test attempts to different types of datanode report. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java index d91155abf01..1901dc1d234 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertArrayEquals; import java.util.ArrayList; @@ -39,6 +38,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.hdfs.util.HostsFileWriter; import org.apache.hadoop.test.GenericTestUtils; import org.apache.log4j.Level; import org.junit.Test; @@ -385,17 +385,8 @@ public class TestBlocksWithNotEnoughRacks { short REPLICATION_FACTOR = 2; final Path filePath = new Path("/testFile"); - // Configure an excludes file - FileSystem localFileSys = FileSystem.getLocal(conf); - Path workingDir = new Path(MiniDFSCluster.getBaseDirectory()); - Path dir = new Path(workingDir, "temp/decommission"); - Path excludeFile = new Path(dir, "exclude"); - Path includeFile = new Path(dir, "include"); - assertTrue(localFileSys.mkdirs(dir)); - DFSTestUtil.writeFile(localFileSys, excludeFile, ""); - DFSTestUtil.writeFile(localFileSys, includeFile, ""); - conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath()); - conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath()); + HostsFileWriter hostsFileWriter = new HostsFileWriter(); + hostsFileWriter.initialize(conf, "temp/decommission"); // Two blocks and four racks String racks[] = {"/rack1", "/rack1", "/rack2", "/rack2"}; @@ -416,7 +407,7 @@ public class TestBlocksWithNotEnoughRacks { BlockLocation locs[] = fs.getFileBlockLocations( fs.getFileStatus(filePath), 0, Long.MAX_VALUE); String name = locs[0].getNames()[0]; - DFSTestUtil.writeFile(localFileSys, excludeFile, name); + hostsFileWriter.initExcludeHost(name); ns.getBlockManager().getDatanodeManager().refreshNodes(conf); DFSTestUtil.waitForDecommission(fs, name); @@ -424,6 +415,7 @@ public class TestBlocksWithNotEnoughRacks { DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0); } finally { cluster.shutdown(); + hostsFileWriter.cleanup(); } } @@ -438,17 +430,8 @@ public class TestBlocksWithNotEnoughRacks { short REPLICATION_FACTOR = 5; final Path filePath = new Path("/testFile"); - // Configure an excludes file - FileSystem localFileSys = FileSystem.getLocal(conf); - Path workingDir = new Path(MiniDFSCluster.getBaseDirectory()); - Path dir = new Path(workingDir, "temp/decommission"); - Path excludeFile = new Path(dir, "exclude"); - Path includeFile = new Path(dir, "include"); - assertTrue(localFileSys.mkdirs(dir)); - DFSTestUtil.writeFile(localFileSys, excludeFile, ""); - DFSTestUtil.writeFile(localFileSys, includeFile, ""); - conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath()); - conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath()); + HostsFileWriter hostsFileWriter = new HostsFileWriter(); + hostsFileWriter.initialize(conf, "temp/decommission"); // All hosts are on two racks, only one host on /rack2 String racks[] = {"/rack1", "/rack2", "/rack1", "/rack1", "/rack1"}; @@ -474,7 +457,7 @@ public class TestBlocksWithNotEnoughRacks { for (String top : locs[0].getTopologyPaths()) { if (!top.startsWith("/rack2")) { String name = top.substring("/rack1".length()+1); - DFSTestUtil.writeFile(localFileSys, excludeFile, name); + hostsFileWriter.initExcludeHost(name); ns.getBlockManager().getDatanodeManager().refreshNodes(conf); DFSTestUtil.waitForDecommission(fs, name); break; @@ -486,6 +469,7 @@ public class TestBlocksWithNotEnoughRacks { DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0); } finally { cluster.shutdown(); + hostsFileWriter.cleanup(); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java index b55a716e69c..be8a0f06b40 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java @@ -383,9 +383,9 @@ public class TestDatanodeManager { DatanodeManager dm = mockDatanodeManager(fsn, new Configuration()); HostFileManager hm = new HostFileManager(); - HostFileManager.HostSet noNodes = new HostFileManager.HostSet(); - HostFileManager.HostSet oneNode = new HostFileManager.HostSet(); - HostFileManager.HostSet twoNodes = new HostFileManager.HostSet(); + HostSet noNodes = new HostSet(); + HostSet oneNode = new HostSet(); + HostSet twoNodes = new HostSet(); DatanodeRegistration dr1 = new DatanodeRegistration( new DatanodeID("127.0.0.1", "127.0.0.1", "someStorageID-123", 12345, 12345, 12345, 12345), @@ -402,7 +402,7 @@ public class TestDatanodeManager { oneNode.add(entry("127.0.0.1:23456")); hm.refresh(twoNodes, noNodes); - Whitebox.setInternalState(dm, "hostFileManager", hm); + Whitebox.setInternalState(dm, "hostConfigManager", hm); // Register two data nodes to simulate them coming up. // We need to add two nodes, because if we have only one node, removing it diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java index 6f17040ef68..e6be7cb1727 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java @@ -40,7 +40,7 @@ public class TestHostFileManager { @Test public void testDeduplication() { - HostFileManager.HostSet s = new HostFileManager.HostSet(); + HostSet s = new HostSet(); // These entries will be de-duped, since they refer to the same IP // address + port combo. s.add(entry("127.0.0.1:12345")); @@ -60,7 +60,7 @@ public class TestHostFileManager { @Test public void testRelation() { - HostFileManager.HostSet s = new HostFileManager.HostSet(); + HostSet s = new HostSet(); s.add(entry("127.0.0.1:123")); Assert.assertTrue(s.match(entry("127.0.0.1:123"))); Assert.assertFalse(s.match(entry("127.0.0.1:12"))); @@ -105,8 +105,8 @@ public class TestHostFileManager { FSNamesystem fsn = mock(FSNamesystem.class); Configuration conf = new Configuration(); HostFileManager hm = new HostFileManager(); - HostFileManager.HostSet includedNodes = new HostFileManager.HostSet(); - HostFileManager.HostSet excludedNodes = new HostFileManager.HostSet(); + HostSet includedNodes = new HostSet(); + HostSet excludedNodes = new HostSet(); includedNodes.add(entry("127.0.0.1:12345")); includedNodes.add(entry("localhost:12345")); @@ -122,7 +122,7 @@ public class TestHostFileManager { hm.refresh(includedNodes, excludedNodes); DatanodeManager dm = new DatanodeManager(bm, fsn, conf); - Whitebox.setInternalState(dm, "hostFileManager", hm); + Whitebox.setInternalState(dm, "hostConfigManager", hm); Map dnMap = (Map) Whitebox.getInternalState(dm, "datanodeMap"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java index 08ad38b9033..d35b8a76442 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java @@ -20,11 +20,10 @@ package org.apache.hadoop.hdfs.server.namenode; import static org.junit.Assert.assertTrue; import java.lang.management.ManagementFactory; -import java.io.File; +import java.util.Arrays; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FileSystem; @@ -34,7 +33,13 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager; +import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager; +import org.apache.hadoop.hdfs.server.blockmanagement.HostFileManager; +import org.apache.hadoop.hdfs.util.HostsFileWriter; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import javax.management.MBeanServer; import javax.management.ObjectName; @@ -43,9 +48,21 @@ import javax.management.ObjectName; * DFS_HOSTS and DFS_HOSTS_EXCLUDE tests * */ +@RunWith(Parameterized.class) public class TestHostsFiles { private static final Log LOG = LogFactory.getLog(TestHostsFiles.class.getName()); + private Class hostFileMgrClass; + + public TestHostsFiles(Class hostFileMgrClass) { + this.hostFileMgrClass = hostFileMgrClass; + } + + @Parameterized.Parameters + public static Iterable data() { + return Arrays.asList(new Object[][]{ + {HostFileManager.class}, {CombinedHostFileManager.class}}); + } /* * Return a configuration object with low timeouts for testing and @@ -72,6 +89,10 @@ public class TestHostsFiles { // Indicates we have multiple racks conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, "xyz"); + + // Host file manager + conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY, + hostFileMgrClass, HostConfigManager.class); return conf; } @@ -80,18 +101,8 @@ public class TestHostsFiles { Configuration conf = getConf(); short REPLICATION_FACTOR = 2; final Path filePath = new Path("/testFile"); - - // Configure an excludes file - FileSystem localFileSys = FileSystem.getLocal(conf); - Path workingDir = new Path(MiniDFSCluster.getBaseDirectory()); - Path dir = new Path(workingDir, "temp/decommission"); - Path excludeFile = new Path(dir, "exclude"); - Path includeFile = new Path(dir, "include"); - assertTrue(localFileSys.mkdirs(dir)); - DFSTestUtil.writeFile(localFileSys, excludeFile, ""); - DFSTestUtil.writeFile(localFileSys, includeFile, ""); - conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath()); - conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath()); + HostsFileWriter hostsFileWriter = new HostsFileWriter(); + hostsFileWriter.initialize(conf, "temp/decommission"); // Two blocks and four racks String racks[] = {"/rack1", "/rack1", "/rack2", "/rack2"}; @@ -112,9 +123,8 @@ public class TestHostsFiles { BlockLocation locs[] = fs.getFileBlockLocations( fs.getFileStatus(filePath), 0, Long.MAX_VALUE); String name = locs[0].getNames()[0]; - String names = name + "\n" + "localhost:42\n"; - LOG.info("adding '" + names + "' to exclude file " + excludeFile.toUri().getPath()); - DFSTestUtil.writeFile(localFileSys, excludeFile, name); + LOG.info("adding '" + name + "' to decommission"); + hostsFileWriter.initExcludeHost(name); ns.getBlockManager().getDatanodeManager().refreshNodes(conf); DFSTestUtil.waitForDecommission(fs, name); @@ -131,9 +141,7 @@ public class TestHostsFiles { if (cluster != null) { cluster.shutdown(); } - if (localFileSys.exists(dir)) { - FileUtils.deleteQuietly(new File(dir.toUri().getPath())); - } + hostsFileWriter.cleanup(); } } @@ -141,20 +149,10 @@ public class TestHostsFiles { public void testHostsIncludeForDeadCount() throws Exception { Configuration conf = getConf(); - // Configure an excludes file - FileSystem localFileSys = FileSystem.getLocal(conf); - Path workingDir = new Path(MiniDFSCluster.getBaseDirectory()); - Path dir = new Path(workingDir, "temp/decommission"); - Path excludeFile = new Path(dir, "exclude"); - Path includeFile = new Path(dir, "include"); - assertTrue(localFileSys.mkdirs(dir)); - StringBuilder includeHosts = new StringBuilder(); - includeHosts.append("localhost:52").append("\n").append("127.0.0.1:7777") - .append("\n"); - DFSTestUtil.writeFile(localFileSys, excludeFile, ""); - DFSTestUtil.writeFile(localFileSys, includeFile, includeHosts.toString()); - conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath()); - conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath()); + HostsFileWriter hostsFileWriter = new HostsFileWriter(); + hostsFileWriter.initialize(conf, "temp/decommission"); + hostsFileWriter.initIncludeHosts(new String[] + {"localhost:52","127.0.0.1:7777"}); MiniDFSCluster cluster = null; try { @@ -174,9 +172,7 @@ public class TestHostsFiles { if (cluster != null) { cluster.shutdown(); } - if (localFileSys.exists(dir)) { - FileUtils.deleteQuietly(new File(dir.toUri().getPath())); - } + hostsFileWriter.cleanup(); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java index 2219aa6bd0a..78cf4569873 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil; import org.apache.hadoop.hdfs.server.namenode.top.TopConf; +import org.apache.hadoop.hdfs.util.HostsFileWriter; import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator; import org.apache.hadoop.net.ServerSocketUtil; @@ -44,9 +45,9 @@ import org.mortbay.util.ajax.JSON; import javax.management.MBeanServer; import javax.management.ObjectName; import java.io.File; -import java.io.IOException; import java.lang.management.ManagementFactory; import java.net.URI; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; @@ -236,8 +237,8 @@ public class TestNameNodeMXBean { conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1); MiniDFSCluster cluster = null; - FileSystem localFileSys = null; - Path dir = null; + HostsFileWriter hostsFileWriter = new HostsFileWriter(); + hostsFileWriter.initialize(conf, "temp/TestNameNodeMXBean"); try { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); @@ -249,18 +250,12 @@ public class TestNameNodeMXBean { ObjectName mxbeanName = new ObjectName( "Hadoop:service=NameNode,name=NameNodeInfo"); - // Define include file to generate deadNodes metrics - localFileSys = FileSystem.getLocal(conf); - Path workingDir = localFileSys.getWorkingDirectory(); - dir = new Path(workingDir,"build/test/data/temp/TestNameNodeMXBean"); - Path includeFile = new Path(dir, "include"); - assertTrue(localFileSys.mkdirs(dir)); - StringBuilder includeHosts = new StringBuilder(); + List hosts = new ArrayList<>(); for(DataNode dn : cluster.getDataNodes()) { - includeHosts.append(dn.getDisplayName()).append("\n"); + hosts.add(dn.getDisplayName()); } - DFSTestUtil.writeFile(localFileSys, includeFile, includeHosts.toString()); - conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath()); + hostsFileWriter.initIncludeHosts(hosts.toArray( + new String[hosts.size()])); fsn.getBlockManager().getDatanodeManager().refreshNodes(conf); cluster.stopDataNode(0); @@ -282,12 +277,10 @@ public class TestNameNodeMXBean { assertTrue(deadNode.containsKey("xferaddr")); } } finally { - if ((localFileSys != null) && localFileSys.exists(dir)) { - FileUtils.deleteQuietly(new File(dir.toUri().getPath())); - } if (cluster != null) { cluster.shutdown(); } + hostsFileWriter.cleanup(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java index 1815078c19e..4b6c0bd971b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java @@ -29,14 +29,12 @@ import java.io.IOException; import java.lang.management.ManagementFactory; import java.net.InetAddress; import java.net.URI; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; @@ -56,6 +54,7 @@ import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; +import org.apache.hadoop.hdfs.util.HostsFileWriter; import org.apache.hadoop.hdfs.util.MD5FileUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.test.GenericTestUtils; @@ -568,27 +567,15 @@ public class TestStartup { @Test public void testNNRestart() throws IOException, InterruptedException { MiniDFSCluster cluster = null; - FileSystem localFileSys; - Path hostsFile; - Path excludeFile; int HEARTBEAT_INTERVAL = 1; // heartbeat interval in seconds - // Set up the hosts/exclude files. - localFileSys = FileSystem.getLocal(config); - Path workingDir = localFileSys.getWorkingDirectory(); - Path dir = new Path(workingDir, "build/test/data/work-dir/restartnn"); - hostsFile = new Path(dir, "hosts"); - excludeFile = new Path(dir, "exclude"); - // Setup conf - config.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath()); - writeConfigFile(localFileSys, excludeFile, null); - config.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath()); - // write into hosts file - ArrayListlist = new ArrayList(); + HostsFileWriter hostsFileWriter = new HostsFileWriter(); + hostsFileWriter.initialize(config, "work-dir/restartnn"); + byte b[] = {127, 0, 0, 1}; InetAddress inetAddress = InetAddress.getByAddress(b); - list.add(inetAddress.getHostName()); - writeConfigFile(localFileSys, hostsFile, list); + hostsFileWriter.initIncludeHosts(new String[] {inetAddress.getHostName()}); + int numDatanodes = 1; try { @@ -613,37 +600,12 @@ public class TestStartup { fail(StringUtils.stringifyException(e)); throw e; } finally { - cleanupFile(localFileSys, excludeFile.getParent()); if (cluster != null) { cluster.shutdown(); } + hostsFileWriter.cleanup(); } } - - private void writeConfigFile(FileSystem localFileSys, Path name, - ArrayList nodes) throws IOException { - // delete if it already exists - if (localFileSys.exists(name)) { - localFileSys.delete(name, true); - } - - FSDataOutputStream stm = localFileSys.create(name); - if (nodes != null) { - for (Iterator it = nodes.iterator(); it.hasNext();) { - String node = it.next(); - stm.writeBytes(node); - stm.writeBytes("\n"); - } - } - stm.close(); - } - - private void cleanupFile(FileSystem fileSys, Path name) throws IOException { - assertTrue(fileSys.exists(name)); - fileSys.delete(name, true); - assertTrue(!fileSys.exists(name)); - } - @Test(timeout = 120000) public void testXattrConfiguration() throws Exception { From e5ff0ea7ba087984262f1f27200ae5bb40d9b838 Mon Sep 17 00:00:00 2001 From: Uma Maheswara Rao G Date: Sat, 26 Mar 2016 00:52:50 -0700 Subject: [PATCH 33/43] HDFS-9694. Make existing DFSClient#getFileChecksum() work for striped blocks. Contributed by Kai Zheng --- .../dev-support/findbugsExcludeFile.xml | 1 + .../org/apache/hadoop/hdfs/DFSClient.java | 11 +- .../hadoop/hdfs/FileChecksumHelper.java | 187 ++++++++++-- .../datatransfer/DataTransferProtocol.java | 16 +- .../hadoop/hdfs/protocol/datatransfer/Op.java | 1 + .../hdfs/protocol/datatransfer/Sender.java | 19 ++ .../hdfs/protocolPB/PBHelperClient.java | 42 ++- .../hadoop/hdfs/util/StripedBlockUtil.java | 12 + .../src/main/proto/datatransfer.proto | 9 +- .../hdfs/protocol/datatransfer/Receiver.java | 28 ++ .../server/datanode/BlockChecksumHelper.java | 284 ++++++++++++++---- .../hdfs/server/datanode/DataXceiver.java | 43 +++ 12 files changed, 570 insertions(+), 83 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml index 2c3329e47a9..9d6ab9a1029 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml @@ -8,6 +8,7 @@ + diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 3506d3ac70a..88bd21909d2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1704,7 +1704,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, /** * Get the checksum of the whole file or a range of the file. Note that the - * range always starts from the beginning of the file. + * range always starts from the beginning of the file. The file can be + * in replicated form, or striped mode. It can be used to checksum and compare + * two replicated files, or two striped files, but not applicable for two + * files of different block layout forms. * @param src The file path * @param length the length of the range, i.e., the range is [0, length] * @return The checksum @@ -1717,7 +1720,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, LocatedBlocks blockLocations = getBlockLocations(src, length); - FileChecksumHelper.FileChecksumComputer maker = + FileChecksumHelper.FileChecksumComputer maker; + ErasureCodingPolicy ecPolicy = blockLocations.getErasureCodingPolicy(); + maker = ecPolicy != null ? + new FileChecksumHelper.StripedFileNonStripedChecksumComputer(src, + length, blockLocations, namenode, this, ecPolicy) : new FileChecksumHelper.ReplicatedFileChecksumComputer(src, length, blockLocations, namenode, this); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java index d15db9f11c9..dfd939397b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java @@ -22,10 +22,13 @@ import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum; import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Op; @@ -75,7 +78,7 @@ final class FileChecksumHelper { private int bytesPerCRC = -1; private DataChecksum.Type crcType = DataChecksum.Type.DEFAULT; private long crcPerBlock = 0; - private boolean refetchBlocks = false; + private boolean isRefetchBlocks = false; private int lastRetriedIndex = -1; /** @@ -127,8 +130,11 @@ final class FileChecksumHelper { return blockLocations; } - void setBlockLocations(LocatedBlocks blockLocations) { - this.blockLocations = blockLocations; + void refetchBlocks() throws IOException { + this.blockLocations = getClient().getBlockLocations(getSrc(), + getLength()); + this.locatedBlocks = getBlockLocations().getLocatedBlocks(); + this.isRefetchBlocks = false; } int getTimeout() { @@ -143,10 +149,6 @@ final class FileChecksumHelper { return locatedBlocks; } - void setLocatedBlocks(List locatedBlocks) { - this.locatedBlocks = locatedBlocks; - } - long getRemaining() { return remaining; } @@ -180,11 +182,11 @@ final class FileChecksumHelper { } boolean isRefetchBlocks() { - return refetchBlocks; + return isRefetchBlocks; } void setRefetchBlocks(boolean refetchBlocks) { - this.refetchBlocks = refetchBlocks; + this.isRefetchBlocks = refetchBlocks; } int getLastRetriedIndex() { @@ -278,10 +280,7 @@ final class FileChecksumHelper { blockIdx < getLocatedBlocks().size() && getRemaining() >= 0; blockIdx++) { if (isRefetchBlocks()) { // refetch to get fresh tokens - setBlockLocations(getClient().getBlockLocations(getSrc(), - getLength())); - setLocatedBlocks(getBlockLocations().getLocatedBlocks()); - setRefetchBlocks(false); + refetchBlocks(); } LocatedBlock locatedBlock = getLocatedBlocks().get(blockIdx); @@ -380,15 +379,13 @@ final class FileChecksumHelper { } //read md5 - final MD5Hash md5 = new MD5Hash( - checksumData.getMd5().toByteArray()); + final MD5Hash md5 = new MD5Hash(checksumData.getMd5().toByteArray()); md5.write(getMd5out()); // read crc-type final DataChecksum.Type ct; if (checksumData.hasCrcType()) { - ct = PBHelperClient.convert(checksumData - .getCrcType()); + ct = PBHelperClient.convert(checksumData.getCrcType()); } else { LOG.debug("Retrieving checksum from an earlier-version DataNode: " + "inferring checksum by reading first byte"); @@ -413,4 +410,160 @@ final class FileChecksumHelper { } } } + + /** + * Striped file checksum computing. + */ + static class StripedFileNonStripedChecksumComputer + extends FileChecksumComputer { + private final ErasureCodingPolicy ecPolicy; + private int bgIdx; + + StripedFileNonStripedChecksumComputer(String src, long length, + LocatedBlocks blockLocations, + ClientProtocol namenode, + DFSClient client, + ErasureCodingPolicy ecPolicy) + throws IOException { + super(src, length, blockLocations, namenode, client); + + this.ecPolicy = ecPolicy; + } + + @Override + void checksumBlocks() throws IOException { + int tmpTimeout = 3000 * 1 + getClient().getConf().getSocketTimeout(); + setTimeout(tmpTimeout); + + for (bgIdx = 0; + bgIdx < getLocatedBlocks().size() && getRemaining() >= 0; bgIdx++) { + if (isRefetchBlocks()) { // refetch to get fresh tokens + refetchBlocks(); + } + + LocatedBlock locatedBlock = getLocatedBlocks().get(bgIdx); + LocatedStripedBlock blockGroup = (LocatedStripedBlock) locatedBlock; + + if (!checksumBlockGroup(blockGroup)) { + throw new IOException("Fail to get block MD5 for " + locatedBlock); + } + } + } + + + private boolean checksumBlockGroup( + LocatedStripedBlock blockGroup) throws IOException { + ExtendedBlock block = blockGroup.getBlock(); + if (getRemaining() < block.getNumBytes()) { + block.setNumBytes(getRemaining()); + } + setRemaining(getRemaining() - block.getNumBytes()); + + StripedBlockInfo stripedBlockInfo = new StripedBlockInfo(block, + blockGroup.getLocations(), blockGroup.getBlockTokens(), ecPolicy); + DatanodeInfo[] datanodes = blockGroup.getLocations(); + + //try each datanode in the block group. + boolean done = false; + for (int j = 0; !done && j < datanodes.length; j++) { + try { + tryDatanode(blockGroup, stripedBlockInfo, datanodes[j]); + done = true; + } catch (InvalidBlockTokenException ibte) { + if (bgIdx > getLastRetriedIndex()) { + LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM " + + "for file {} for block {} from datanode {}. Will retry " + + "the block once.", + getSrc(), block, datanodes[j]); + setLastRetriedIndex(bgIdx); + done = true; // actually it's not done; but we'll retry + bgIdx--; // repeat at bgIdx-th block + setRefetchBlocks(true); + } + } catch (IOException ie) { + LOG.warn("src={}" + ", datanodes[{}]={}", + getSrc(), j, datanodes[j], ie); + } + } + + return done; + } + + /** + * Return true when sounds good to continue or retry, false when severe + * condition or totally failed. + */ + private void tryDatanode(LocatedStripedBlock blockGroup, + StripedBlockInfo stripedBlockInfo, + DatanodeInfo datanode) throws IOException { + + try (IOStreamPair pair = getClient().connectToDN(datanode, + getTimeout(), blockGroup.getBlockToken())) { + + LOG.debug("write to {}: {}, blockGroup={}", + datanode, Op.BLOCK_GROUP_CHECKSUM, blockGroup); + + // get block MD5 + createSender(pair).blockGroupChecksum(stripedBlockInfo, + blockGroup.getBlockToken()); + + BlockOpResponseProto reply = BlockOpResponseProto.parseFrom( + PBHelperClient.vintPrefixed(pair.in)); + + String logInfo = "for blockGroup " + blockGroup + + " from datanode " + datanode; + DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo); + + OpBlockChecksumResponseProto checksumData = reply.getChecksumResponse(); + + //read byte-per-checksum + final int bpc = checksumData.getBytesPerCrc(); + if (bgIdx == 0) { //first block + setBytesPerCRC(bpc); + } else { + if (bpc != getBytesPerCRC()) { + throw new IOException("Byte-per-checksum not matched: bpc=" + bpc + + " but bytesPerCRC=" + getBytesPerCRC()); + } + } + + //read crc-per-block + final long cpb = checksumData.getCrcPerBlock(); + if (getLocatedBlocks().size() > 1 && bgIdx == 0) { // first block + setCrcPerBlock(cpb); + } + + //read md5 + final MD5Hash md5 = new MD5Hash( + checksumData.getMd5().toByteArray()); + md5.write(getMd5out()); + + // read crc-type + final DataChecksum.Type ct; + if (checksumData.hasCrcType()) { + ct = PBHelperClient.convert(checksumData.getCrcType()); + } else { + LOG.debug("Retrieving checksum from an earlier-version DataNode: " + + "inferring checksum by reading first byte"); + ct = getClient().inferChecksumTypeByReading(blockGroup, datanode); + } + + if (bgIdx == 0) { + setCrcType(ct); + } else if (getCrcType() != DataChecksum.Type.MIXED && + getCrcType() != ct) { + // if crc types are mixed in a file + setCrcType(DataChecksum.Type.MIXED); + } + + if (LOG.isDebugEnabled()) { + if (bgIdx == 0) { + LOG.debug("set bytesPerCRC=" + getBytesPerCRC() + + ", crcPerBlock=" + getCrcPerBlock()); + } + LOG.debug("got reply from " + datanode + ": md5=" + md5); + } + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java index 4aa545b3338..ad3f2ad90a1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId; @@ -197,6 +198,17 @@ public interface DataTransferProtocol { * @param blockToken security token for accessing the block. * @throws IOException */ - void blockChecksum(final ExtendedBlock blk, - final Token blockToken) throws IOException; + void blockChecksum(ExtendedBlock blk, + Token blockToken) throws IOException; + + + /** + * Get striped block group checksum (MD5 of CRC32). + * + * @param stripedBlockInfo a striped block info. + * @param blockToken security token for accessing the block. + * @throws IOException + */ + void blockGroupChecksum(StripedBlockInfo stripedBlockInfo, + Token blockToken) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java index 511574c4145..94250e5e7f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java @@ -38,6 +38,7 @@ public enum Op { REQUEST_SHORT_CIRCUIT_FDS((byte)87), RELEASE_SHORT_CIRCUIT_FDS((byte)88), REQUEST_SHORT_CIRCUIT_SHM((byte)89), + BLOCK_GROUP_CHECKSUM((byte)90), CUSTOM((byte)127); /** The code for this operation. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java index 65456815aed..585ed99b1ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java @@ -28,11 +28,13 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockGroupChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; @@ -261,4 +263,21 @@ public class Sender implements DataTransferProtocol { send(out, Op.BLOCK_CHECKSUM, proto); } + + @Override + public void blockGroupChecksum(StripedBlockInfo stripedBlockInfo, + Token blockToken) throws IOException { + OpBlockGroupChecksumProto proto = OpBlockGroupChecksumProto.newBuilder() + .setHeader(DataTransferProtoUtil.buildBaseHeader( + stripedBlockInfo.getBlock(), blockToken)) + .setDatanodes(PBHelperClient.convertToProto( + stripedBlockInfo.getDatanodes())) + .addAllBlockTokens(PBHelperClient.convert( + stripedBlockInfo.getBlockTokens())) + .setEcPolicy(PBHelperClient.convertErasureCodingPolicy( + stripedBlockInfo.getErasureCodingPolicy())) + .build(); + + send(out, Op.BLOCK_GROUP_CHECKSUM, proto); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java index 38e875c012d..47593732123 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java @@ -553,10 +553,8 @@ public class PBHelperClient { proto.getCorrupt(), cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()])); List tokenProtos = proto.getBlockTokensList(); - Token[] blockTokens = new Token[indices.length]; - for (int i = 0; i < indices.length; i++) { - blockTokens[i] = convert(tokenProtos.get(i)); - } + Token[] blockTokens = + convertTokens(tokenProtos); ((LocatedStripedBlock) lb).setBlockTokens(blockTokens); } lb.setBlockToken(convert(proto.getBlockToken())); @@ -564,6 +562,18 @@ public class PBHelperClient { return lb; } + static public Token[] convertTokens( + List tokenProtos) { + + @SuppressWarnings("unchecked") + Token[] blockTokens = new Token[tokenProtos.size()]; + for (int i = 0; i < blockTokens.length; i++) { + blockTokens[i] = convert(tokenProtos.get(i)); + } + + return blockTokens; + } + static public DatanodeInfo convert(DatanodeInfoProto di) { if (di == null) return null; return new DatanodeInfo( @@ -815,9 +825,7 @@ public class PBHelperClient { byte[] indices = sb.getBlockIndices(); builder.setBlockIndices(PBHelperClient.getByteString(indices)); Token[] blockTokens = sb.getBlockTokens(); - for (int i = 0; i < indices.length; i++) { - builder.addBlockTokens(PBHelperClient.convert(blockTokens[i])); - } + builder.addAllBlockTokens(convert(blockTokens)); } return builder.setB(PBHelperClient.convert(b.getBlock())) @@ -825,6 +833,16 @@ public class PBHelperClient { .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build(); } + public static List convert( + Token[] blockTokens) { + List results = new ArrayList<>(blockTokens.length); + for (Token bt : blockTokens) { + results.add(convert(bt)); + } + + return results; + } + public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) { List cList = proto.getCreationPolicy() .getStorageTypesList(); @@ -2500,4 +2518,14 @@ public class PBHelperClient { .setId(policy.getId()); return builder.build(); } + + public static HdfsProtos.DatanodeInfosProto convertToProto( + DatanodeInfo[] datanodeInfos) { + HdfsProtos.DatanodeInfosProto.Builder builder = + HdfsProtos.DatanodeInfosProto.newBuilder(); + for (DatanodeInfo datanodeInfo : datanodeInfos) { + builder.addDatanodes(PBHelperClient.convert(datanodeInfo)); + } + return builder.build(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 43772e2efbd..0819376aac0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -75,6 +75,18 @@ public class StripedBlockUtil { public static final Logger LOG = LoggerFactory.getLogger(StripedBlockUtil.class); + /** + * Parses a striped block group into individual blocks. + * @param bg The striped block group + * @param ecPolicy The erasure coding policy + * @return An array of the blocks in the group + */ + public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, + ErasureCodingPolicy ecPolicy) { + return parseStripedBlockGroup(bg, ecPolicy.getCellSize(), + ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits()); + } + /** * This method parses a striped block group into individual blocks. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto index a091d417d7c..522ee06b68b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto @@ -74,7 +74,6 @@ message OpReadBlockProto { optional CachingStrategyProto cachingStrategy = 5; } - message ChecksumProto { required ChecksumTypeProto type = 1; required uint32 bytesPerChecksum = 2; @@ -149,6 +148,14 @@ message OpBlockChecksumProto { required BaseHeaderProto header = 1; } +message OpBlockGroupChecksumProto { + required BaseHeaderProto header = 1; + required DatanodeInfosProto datanodes = 2; + // each internal block has a block token + repeated hadoop.common.TokenProto blockTokens = 3; + required ErasureCodingPolicyProto ecPolicy = 4; +} + /** * An ID uniquely identifying a shared memory segment. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index e0401574ce6..b2f26f8d3cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -26,11 +26,13 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockGroupChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; @@ -111,6 +113,9 @@ public abstract class Receiver implements DataTransferProtocol { case BLOCK_CHECKSUM: opBlockChecksum(in); break; + case BLOCK_GROUP_CHECKSUM: + opStripedBlockChecksum(in); + break; case TRANSFER_BLOCK: opTransferBlock(in); break; @@ -290,4 +295,27 @@ public abstract class Receiver implements DataTransferProtocol { if (traceScope != null) traceScope.close(); } } + + /** Receive OP_STRIPED_BLOCK_CHECKSUM. */ + private void opStripedBlockChecksum(DataInputStream dis) throws IOException { + OpBlockGroupChecksumProto proto = + OpBlockGroupChecksumProto.parseFrom(vintPrefixed(dis)); + TraceScope traceScope = continueTraceSpan(proto.getHeader(), + proto.getClass().getSimpleName()); + StripedBlockInfo stripedBlockInfo = new StripedBlockInfo( + PBHelperClient.convert(proto.getHeader().getBlock()), + PBHelperClient.convert(proto.getDatanodes()), + PBHelperClient.convertTokens(proto.getBlockTokensList()), + PBHelperClient.convertErasureCodingPolicy(proto.getEcPolicy()) + ); + + try { + blockGroupChecksum(stripedBlockInfo, + PBHelperClient.convert(proto.getHeader().getToken())); + } finally { + if (traceScope != null) { + traceScope.close(); + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java index 9a5552db3a7..1f1a25c4184 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java @@ -19,16 +19,30 @@ package org.apache.hadoop.hdfs.server.datanode; import com.google.common.base.Preconditions; import org.apache.hadoop.hdfs.DFSUtilClient; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; +import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; +import org.apache.hadoop.hdfs.protocol.datatransfer.Op; +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos; +import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.BufferedInputStream; import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.security.MessageDigest; @@ -41,13 +55,87 @@ final class BlockChecksumHelper { static final Logger LOG = LoggerFactory.getLogger(BlockChecksumHelper.class); - private BlockChecksumHelper() {} + private BlockChecksumHelper() { + } /** * The abstract base block checksum computer. */ - static abstract class BlockChecksumComputer { + static abstract class AbstractBlockChecksumComputer { private final DataNode datanode; + + private byte[] outBytes; + private int bytesPerCRC = -1; + private DataChecksum.Type crcType = null; + private long crcPerBlock = -1; + private int checksumSize = -1; + + AbstractBlockChecksumComputer(DataNode datanode) throws IOException { + this.datanode = datanode; + } + + abstract void compute() throws IOException; + + Sender createSender(IOStreamPair pair) { + DataOutputStream out = (DataOutputStream) pair.out; + return new Sender(out); + } + + DataNode getDatanode() { + return datanode; + } + + InputStream getBlockInputStream(ExtendedBlock block, long seekOffset) + throws IOException { + return datanode.data.getBlockInputStream(block, seekOffset); + } + + void setOutBytes(byte[] bytes) { + this.outBytes = bytes; + } + + byte[] getOutBytes() { + return outBytes; + } + + int getBytesPerCRC() { + return bytesPerCRC; + } + + public void setBytesPerCRC(int bytesPerCRC) { + this.bytesPerCRC = bytesPerCRC; + } + + public void setCrcType(DataChecksum.Type crcType) { + this.crcType = crcType; + } + + public void setCrcPerBlock(long crcPerBlock) { + this.crcPerBlock = crcPerBlock; + } + + public void setChecksumSize(int checksumSize) { + this.checksumSize = checksumSize; + } + + DataChecksum.Type getCrcType() { + return crcType; + } + + long getCrcPerBlock() { + return crcPerBlock; + } + + int getChecksumSize() { + return checksumSize; + } + } + + /** + * The abstract base block checksum computer. + */ + static abstract class BlockChecksumComputer + extends AbstractBlockChecksumComputer { private final ExtendedBlock block; // client side now can specify a range of the block for checksum private final long requestLength; @@ -56,17 +144,12 @@ final class BlockChecksumHelper { private final long visibleLength; private final boolean partialBlk; - private byte[] outBytes; - private int bytesPerCRC = -1; - private DataChecksum.Type crcType = null; - private long crcPerBlock = -1; - private int checksumSize = -1; private BlockMetadataHeader header; private DataChecksum checksum; BlockChecksumComputer(DataNode datanode, ExtendedBlock block) throws IOException { - this.datanode = datanode; + super(datanode); this.block = block; this.requestLength = block.getNumBytes(); Preconditions.checkArgument(requestLength >= 0); @@ -81,98 +164,80 @@ final class BlockChecksumHelper { new BufferedInputStream(metadataIn, ioFileBufferSize)); } - protected DataNode getDatanode() { - return datanode; + Sender createSender(IOStreamPair pair) { + DataOutputStream out = (DataOutputStream) pair.out; + return new Sender(out); } - protected ExtendedBlock getBlock() { + + ExtendedBlock getBlock() { return block; } - protected long getRequestLength() { + long getRequestLength() { return requestLength; } - protected LengthInputStream getMetadataIn() { + LengthInputStream getMetadataIn() { return metadataIn; } - protected DataInputStream getChecksumIn() { + DataInputStream getChecksumIn() { return checksumIn; } - protected long getVisibleLength() { + long getVisibleLength() { return visibleLength; } - protected boolean isPartialBlk() { + boolean isPartialBlk() { return partialBlk; } - protected void setOutBytes(byte[] bytes) { - this.outBytes = bytes; - } - - protected byte[] getOutBytes() { - return outBytes; - } - - protected int getBytesPerCRC() { - return bytesPerCRC; - } - - protected DataChecksum.Type getCrcType() { - return crcType; - } - - protected long getCrcPerBlock() { - return crcPerBlock; - } - - protected int getChecksumSize() { - return checksumSize; - } - - protected BlockMetadataHeader getHeader() { + BlockMetadataHeader getHeader() { return header; } - protected DataChecksum getChecksum() { + DataChecksum getChecksum() { return checksum; } /** * Perform the block checksum computing. + * * @throws IOException */ abstract void compute() throws IOException; /** * Read block metadata header. + * * @throws IOException */ - protected void readHeader() throws IOException { + void readHeader() throws IOException { //read metadata file header = BlockMetadataHeader.readHeader(checksumIn); checksum = header.getChecksum(); - checksumSize = checksum.getChecksumSize(); - bytesPerCRC = checksum.getBytesPerChecksum(); - crcPerBlock = checksumSize <= 0 ? 0 : + setChecksumSize(checksum.getChecksumSize()); + setBytesPerCRC(checksum.getBytesPerChecksum()); + long crcPerBlock = checksum.getChecksumSize() <= 0 ? 0 : (metadataIn.getLength() - - BlockMetadataHeader.getHeaderSize()) / checksumSize; - crcType = checksum.getChecksumType(); + BlockMetadataHeader.getHeaderSize()) / checksum.getChecksumSize(); + setCrcPerBlock(crcPerBlock); + setCrcType(checksum.getChecksumType()); } /** * Calculate partial block checksum. + * * @return * @throws IOException */ - protected byte[] crcPartialBlock() throws IOException { - int partialLength = (int) (requestLength % bytesPerCRC); + byte[] crcPartialBlock() throws IOException { + int partialLength = (int) (requestLength % getBytesPerCRC()); if (partialLength > 0) { byte[] buf = new byte[partialLength]; - final InputStream blockIn = datanode.data.getBlockInputStream(block, + final InputStream blockIn = getBlockInputStream(block, requestLength - partialLength); try { // Get the CRC of the partialLength. @@ -181,7 +246,7 @@ final class BlockChecksumHelper { IOUtils.closeStream(blockIn); } checksum.update(buf, 0, partialLength); - byte[] partialCrc = new byte[checksumSize]; + byte[] partialCrc = new byte[getChecksumSize()]; checksum.writeValue(partialCrc, 0, true); return partialCrc; } @@ -229,7 +294,7 @@ final class BlockChecksumHelper { } private MD5Hash checksumPartialBlock() throws IOException { - byte[] buffer = new byte[4*1024]; + byte[] buffer = new byte[4 * 1024]; MessageDigest digester = MD5Hash.getDigester(); long remaining = (getRequestLength() / getBytesPerCRC()) @@ -251,4 +316,115 @@ final class BlockChecksumHelper { return new MD5Hash(digester.digest()); } } -} + + /** + * Non-striped block group checksum computer for striped blocks. + */ + static class BlockGroupNonStripedChecksumComputer + extends AbstractBlockChecksumComputer { + + private final ExtendedBlock blockGroup; + private final ErasureCodingPolicy ecPolicy; + private final DatanodeInfo[] datanodes; + private final Token[] blockTokens; + + private final DataOutputBuffer md5writer = new DataOutputBuffer(); + + BlockGroupNonStripedChecksumComputer(DataNode datanode, + StripedBlockInfo stripedBlockInfo) + throws IOException { + super(datanode); + this.blockGroup = stripedBlockInfo.getBlock(); + this.ecPolicy = stripedBlockInfo.getErasureCodingPolicy(); + this.datanodes = stripedBlockInfo.getDatanodes(); + this.blockTokens = stripedBlockInfo.getBlockTokens(); + } + + @Override + void compute() throws IOException { + for (int idx = 0; idx < ecPolicy.getNumDataUnits(); idx++) { + ExtendedBlock block = + StripedBlockUtil.constructInternalBlock(blockGroup, + ecPolicy.getCellSize(), ecPolicy.getNumDataUnits(), idx); + DatanodeInfo targetDatanode = datanodes[idx]; + Token blockToken = blockTokens[idx]; + checksumBlock(block, idx, blockToken, targetDatanode); + } + + MD5Hash md5out = MD5Hash.digest(md5writer.getData()); + setOutBytes(md5out.getDigest()); + } + + private void checksumBlock(ExtendedBlock block, int blockIdx, + Token blockToken, + DatanodeInfo targetDatanode) throws IOException { + int timeout = 3000; + try (IOStreamPair pair = getDatanode().connectToDN(targetDatanode, + timeout, block, blockToken)) { + + LOG.debug("write to {}: {}, block={}", + getDatanode(), Op.BLOCK_CHECKSUM, block); + + // get block MD5 + createSender(pair).blockChecksum(block, blockToken); + + final DataTransferProtos.BlockOpResponseProto reply = + DataTransferProtos.BlockOpResponseProto.parseFrom( + PBHelperClient.vintPrefixed(pair.in)); + + String logInfo = "for block " + block + + " from datanode " + targetDatanode; + DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo); + + DataTransferProtos.OpBlockChecksumResponseProto checksumData = + reply.getChecksumResponse(); + + //read byte-per-checksum + final int bpc = checksumData.getBytesPerCrc(); + if (blockIdx == 0) { //first block + setBytesPerCRC(bpc); + } else if (bpc != getBytesPerCRC()) { + throw new IOException("Byte-per-checksum not matched: bpc=" + bpc + + " but bytesPerCRC=" + getBytesPerCRC()); + } + + //read crc-per-block + final long cpb = checksumData.getCrcPerBlock(); + if (blockIdx == 0) { + setCrcPerBlock(cpb); + } + + //read md5 + final MD5Hash md5 = new MD5Hash( + checksumData.getMd5().toByteArray()); + md5.write(md5writer); + + // read crc-type + final DataChecksum.Type ct; + if (checksumData.hasCrcType()) { + ct = PBHelperClient.convert(checksumData.getCrcType()); + } else { + LOG.debug("Retrieving checksum from an earlier-version DataNode: " + + "inferring checksum by reading first byte"); + ct = DataChecksum.Type.DEFAULT; + } + + if (blockIdx == 0) { // first block + setCrcType(ct); + } else if (getCrcType() != DataChecksum.Type.MIXED && + getCrcType() != ct) { + // if crc types are mixed in a file + setCrcType(DataChecksum.Type.MIXED); + } + + if (LOG.isDebugEnabled()) { + if (blockIdx == 0) { + LOG.debug("set bytesPerCRC=" + getBytesPerCRC() + + ", crcPerBlock=" + getCrcPerBlock()); + } + LOG.debug("got reply from " + targetDatanode + ": md5=" + md5); + } + } + } + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index 1d4a79ac88d..63bf5ae5363 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; @@ -46,7 +47,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.BlockChecksumComputer; +import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.AbstractBlockChecksumComputer; import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.ReplicatedBlockChecksumComputer; +import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.BlockGroupNonStripedChecksumComputer; import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException; import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException; import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.NewShmInfo; @@ -923,6 +926,46 @@ class DataXceiver extends Receiver implements Runnable { datanode.metrics.addBlockChecksumOp(elapsed()); } + @Override + public void blockGroupChecksum(final StripedBlockInfo stripedBlockInfo, + final Token blockToken) + throws IOException { + updateCurrentThreadName("Getting checksum for block group" + + stripedBlockInfo.getBlock()); + final DataOutputStream out = new DataOutputStream(getOutputStream()); + checkAccess(out, true, stripedBlockInfo.getBlock(), blockToken, + Op.BLOCK_GROUP_CHECKSUM, BlockTokenIdentifier.AccessMode.READ); + + AbstractBlockChecksumComputer maker = + new BlockGroupNonStripedChecksumComputer(datanode, stripedBlockInfo); + + try { + maker.compute(); + + //write reply + BlockOpResponseProto.newBuilder() + .setStatus(SUCCESS) + .setChecksumResponse(OpBlockChecksumResponseProto.newBuilder() + .setBytesPerCrc(maker.getBytesPerCRC()) + .setCrcPerBlock(maker.getCrcPerBlock()) + .setMd5(ByteString.copyFrom(maker.getOutBytes())) + .setCrcType(PBHelperClient.convert(maker.getCrcType()))) + .build() + .writeDelimitedTo(out); + out.flush(); + } catch (IOException ioe) { + LOG.info("blockChecksum " + stripedBlockInfo.getBlock() + + " received exception " + ioe); + incrDatanodeNetworkErrors(); + throw ioe; + } finally { + IOUtils.closeStream(out); + } + + //update metrics + datanode.metrics.addBlockChecksumOp(elapsed()); + } + @Override public void copyBlock(final ExtendedBlock block, final Token blockToken) throws IOException { From a337ceb74e984991dbf976236d2e785cf5921b16 Mon Sep 17 00:00:00 2001 From: Arpit Agarwal Date: Sat, 26 Mar 2016 09:20:01 -0700 Subject: [PATCH 34/43] Revert "HDFS-9694. Make existing DFSClient#getFileChecksum() work for striped blocks. Contributed by Kai Zheng" This reverts commit e5ff0ea7ba087984262f1f27200ae5bb40d9b838. --- .../dev-support/findbugsExcludeFile.xml | 1 - .../org/apache/hadoop/hdfs/DFSClient.java | 11 +- .../hadoop/hdfs/FileChecksumHelper.java | 187 ++---------- .../datatransfer/DataTransferProtocol.java | 16 +- .../hadoop/hdfs/protocol/datatransfer/Op.java | 1 - .../hdfs/protocol/datatransfer/Sender.java | 19 -- .../hdfs/protocolPB/PBHelperClient.java | 42 +-- .../hadoop/hdfs/util/StripedBlockUtil.java | 12 - .../src/main/proto/datatransfer.proto | 9 +- .../hdfs/protocol/datatransfer/Receiver.java | 28 -- .../server/datanode/BlockChecksumHelper.java | 284 ++++-------------- .../hdfs/server/datanode/DataXceiver.java | 43 --- 12 files changed, 83 insertions(+), 570 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml index 9d6ab9a1029..2c3329e47a9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml @@ -8,7 +8,6 @@ - diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 88bd21909d2..3506d3ac70a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1704,10 +1704,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, /** * Get the checksum of the whole file or a range of the file. Note that the - * range always starts from the beginning of the file. The file can be - * in replicated form, or striped mode. It can be used to checksum and compare - * two replicated files, or two striped files, but not applicable for two - * files of different block layout forms. + * range always starts from the beginning of the file. * @param src The file path * @param length the length of the range, i.e., the range is [0, length] * @return The checksum @@ -1720,11 +1717,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, LocatedBlocks blockLocations = getBlockLocations(src, length); - FileChecksumHelper.FileChecksumComputer maker; - ErasureCodingPolicy ecPolicy = blockLocations.getErasureCodingPolicy(); - maker = ecPolicy != null ? - new FileChecksumHelper.StripedFileNonStripedChecksumComputer(src, - length, blockLocations, namenode, this, ecPolicy) : + FileChecksumHelper.FileChecksumComputer maker = new FileChecksumHelper.ReplicatedFileChecksumComputer(src, length, blockLocations, namenode, this); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java index dfd939397b1..d15db9f11c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java @@ -22,13 +22,10 @@ import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum; import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; -import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Op; @@ -78,7 +75,7 @@ final class FileChecksumHelper { private int bytesPerCRC = -1; private DataChecksum.Type crcType = DataChecksum.Type.DEFAULT; private long crcPerBlock = 0; - private boolean isRefetchBlocks = false; + private boolean refetchBlocks = false; private int lastRetriedIndex = -1; /** @@ -130,11 +127,8 @@ final class FileChecksumHelper { return blockLocations; } - void refetchBlocks() throws IOException { - this.blockLocations = getClient().getBlockLocations(getSrc(), - getLength()); - this.locatedBlocks = getBlockLocations().getLocatedBlocks(); - this.isRefetchBlocks = false; + void setBlockLocations(LocatedBlocks blockLocations) { + this.blockLocations = blockLocations; } int getTimeout() { @@ -149,6 +143,10 @@ final class FileChecksumHelper { return locatedBlocks; } + void setLocatedBlocks(List locatedBlocks) { + this.locatedBlocks = locatedBlocks; + } + long getRemaining() { return remaining; } @@ -182,11 +180,11 @@ final class FileChecksumHelper { } boolean isRefetchBlocks() { - return isRefetchBlocks; + return refetchBlocks; } void setRefetchBlocks(boolean refetchBlocks) { - this.isRefetchBlocks = refetchBlocks; + this.refetchBlocks = refetchBlocks; } int getLastRetriedIndex() { @@ -280,7 +278,10 @@ final class FileChecksumHelper { blockIdx < getLocatedBlocks().size() && getRemaining() >= 0; blockIdx++) { if (isRefetchBlocks()) { // refetch to get fresh tokens - refetchBlocks(); + setBlockLocations(getClient().getBlockLocations(getSrc(), + getLength())); + setLocatedBlocks(getBlockLocations().getLocatedBlocks()); + setRefetchBlocks(false); } LocatedBlock locatedBlock = getLocatedBlocks().get(blockIdx); @@ -379,13 +380,15 @@ final class FileChecksumHelper { } //read md5 - final MD5Hash md5 = new MD5Hash(checksumData.getMd5().toByteArray()); + final MD5Hash md5 = new MD5Hash( + checksumData.getMd5().toByteArray()); md5.write(getMd5out()); // read crc-type final DataChecksum.Type ct; if (checksumData.hasCrcType()) { - ct = PBHelperClient.convert(checksumData.getCrcType()); + ct = PBHelperClient.convert(checksumData + .getCrcType()); } else { LOG.debug("Retrieving checksum from an earlier-version DataNode: " + "inferring checksum by reading first byte"); @@ -410,160 +413,4 @@ final class FileChecksumHelper { } } } - - /** - * Striped file checksum computing. - */ - static class StripedFileNonStripedChecksumComputer - extends FileChecksumComputer { - private final ErasureCodingPolicy ecPolicy; - private int bgIdx; - - StripedFileNonStripedChecksumComputer(String src, long length, - LocatedBlocks blockLocations, - ClientProtocol namenode, - DFSClient client, - ErasureCodingPolicy ecPolicy) - throws IOException { - super(src, length, blockLocations, namenode, client); - - this.ecPolicy = ecPolicy; - } - - @Override - void checksumBlocks() throws IOException { - int tmpTimeout = 3000 * 1 + getClient().getConf().getSocketTimeout(); - setTimeout(tmpTimeout); - - for (bgIdx = 0; - bgIdx < getLocatedBlocks().size() && getRemaining() >= 0; bgIdx++) { - if (isRefetchBlocks()) { // refetch to get fresh tokens - refetchBlocks(); - } - - LocatedBlock locatedBlock = getLocatedBlocks().get(bgIdx); - LocatedStripedBlock blockGroup = (LocatedStripedBlock) locatedBlock; - - if (!checksumBlockGroup(blockGroup)) { - throw new IOException("Fail to get block MD5 for " + locatedBlock); - } - } - } - - - private boolean checksumBlockGroup( - LocatedStripedBlock blockGroup) throws IOException { - ExtendedBlock block = blockGroup.getBlock(); - if (getRemaining() < block.getNumBytes()) { - block.setNumBytes(getRemaining()); - } - setRemaining(getRemaining() - block.getNumBytes()); - - StripedBlockInfo stripedBlockInfo = new StripedBlockInfo(block, - blockGroup.getLocations(), blockGroup.getBlockTokens(), ecPolicy); - DatanodeInfo[] datanodes = blockGroup.getLocations(); - - //try each datanode in the block group. - boolean done = false; - for (int j = 0; !done && j < datanodes.length; j++) { - try { - tryDatanode(blockGroup, stripedBlockInfo, datanodes[j]); - done = true; - } catch (InvalidBlockTokenException ibte) { - if (bgIdx > getLastRetriedIndex()) { - LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM " - + "for file {} for block {} from datanode {}. Will retry " - + "the block once.", - getSrc(), block, datanodes[j]); - setLastRetriedIndex(bgIdx); - done = true; // actually it's not done; but we'll retry - bgIdx--; // repeat at bgIdx-th block - setRefetchBlocks(true); - } - } catch (IOException ie) { - LOG.warn("src={}" + ", datanodes[{}]={}", - getSrc(), j, datanodes[j], ie); - } - } - - return done; - } - - /** - * Return true when sounds good to continue or retry, false when severe - * condition or totally failed. - */ - private void tryDatanode(LocatedStripedBlock blockGroup, - StripedBlockInfo stripedBlockInfo, - DatanodeInfo datanode) throws IOException { - - try (IOStreamPair pair = getClient().connectToDN(datanode, - getTimeout(), blockGroup.getBlockToken())) { - - LOG.debug("write to {}: {}, blockGroup={}", - datanode, Op.BLOCK_GROUP_CHECKSUM, blockGroup); - - // get block MD5 - createSender(pair).blockGroupChecksum(stripedBlockInfo, - blockGroup.getBlockToken()); - - BlockOpResponseProto reply = BlockOpResponseProto.parseFrom( - PBHelperClient.vintPrefixed(pair.in)); - - String logInfo = "for blockGroup " + blockGroup + - " from datanode " + datanode; - DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo); - - OpBlockChecksumResponseProto checksumData = reply.getChecksumResponse(); - - //read byte-per-checksum - final int bpc = checksumData.getBytesPerCrc(); - if (bgIdx == 0) { //first block - setBytesPerCRC(bpc); - } else { - if (bpc != getBytesPerCRC()) { - throw new IOException("Byte-per-checksum not matched: bpc=" + bpc - + " but bytesPerCRC=" + getBytesPerCRC()); - } - } - - //read crc-per-block - final long cpb = checksumData.getCrcPerBlock(); - if (getLocatedBlocks().size() > 1 && bgIdx == 0) { // first block - setCrcPerBlock(cpb); - } - - //read md5 - final MD5Hash md5 = new MD5Hash( - checksumData.getMd5().toByteArray()); - md5.write(getMd5out()); - - // read crc-type - final DataChecksum.Type ct; - if (checksumData.hasCrcType()) { - ct = PBHelperClient.convert(checksumData.getCrcType()); - } else { - LOG.debug("Retrieving checksum from an earlier-version DataNode: " + - "inferring checksum by reading first byte"); - ct = getClient().inferChecksumTypeByReading(blockGroup, datanode); - } - - if (bgIdx == 0) { - setCrcType(ct); - } else if (getCrcType() != DataChecksum.Type.MIXED && - getCrcType() != ct) { - // if crc types are mixed in a file - setCrcType(DataChecksum.Type.MIXED); - } - - if (LOG.isDebugEnabled()) { - if (bgIdx == 0) { - LOG.debug("set bytesPerCRC=" + getBytesPerCRC() - + ", crcPerBlock=" + getCrcPerBlock()); - } - LOG.debug("got reply from " + datanode + ": md5=" + md5); - } - } - } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java index ad3f2ad90a1..4aa545b3338 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java @@ -24,7 +24,6 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId; @@ -198,17 +197,6 @@ public interface DataTransferProtocol { * @param blockToken security token for accessing the block. * @throws IOException */ - void blockChecksum(ExtendedBlock blk, - Token blockToken) throws IOException; - - - /** - * Get striped block group checksum (MD5 of CRC32). - * - * @param stripedBlockInfo a striped block info. - * @param blockToken security token for accessing the block. - * @throws IOException - */ - void blockGroupChecksum(StripedBlockInfo stripedBlockInfo, - Token blockToken) throws IOException; + void blockChecksum(final ExtendedBlock blk, + final Token blockToken) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java index 94250e5e7f6..511574c4145 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java @@ -38,7 +38,6 @@ public enum Op { REQUEST_SHORT_CIRCUIT_FDS((byte)87), RELEASE_SHORT_CIRCUIT_FDS((byte)88), REQUEST_SHORT_CIRCUIT_SHM((byte)89), - BLOCK_GROUP_CHECKSUM((byte)90), CUSTOM((byte)127); /** The code for this operation. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java index 585ed99b1ed..65456815aed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java @@ -28,13 +28,11 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockGroupChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; @@ -263,21 +261,4 @@ public class Sender implements DataTransferProtocol { send(out, Op.BLOCK_CHECKSUM, proto); } - - @Override - public void blockGroupChecksum(StripedBlockInfo stripedBlockInfo, - Token blockToken) throws IOException { - OpBlockGroupChecksumProto proto = OpBlockGroupChecksumProto.newBuilder() - .setHeader(DataTransferProtoUtil.buildBaseHeader( - stripedBlockInfo.getBlock(), blockToken)) - .setDatanodes(PBHelperClient.convertToProto( - stripedBlockInfo.getDatanodes())) - .addAllBlockTokens(PBHelperClient.convert( - stripedBlockInfo.getBlockTokens())) - .setEcPolicy(PBHelperClient.convertErasureCodingPolicy( - stripedBlockInfo.getErasureCodingPolicy())) - .build(); - - send(out, Op.BLOCK_GROUP_CHECKSUM, proto); - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java index 47593732123..38e875c012d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java @@ -553,8 +553,10 @@ public class PBHelperClient { proto.getCorrupt(), cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()])); List tokenProtos = proto.getBlockTokensList(); - Token[] blockTokens = - convertTokens(tokenProtos); + Token[] blockTokens = new Token[indices.length]; + for (int i = 0; i < indices.length; i++) { + blockTokens[i] = convert(tokenProtos.get(i)); + } ((LocatedStripedBlock) lb).setBlockTokens(blockTokens); } lb.setBlockToken(convert(proto.getBlockToken())); @@ -562,18 +564,6 @@ public class PBHelperClient { return lb; } - static public Token[] convertTokens( - List tokenProtos) { - - @SuppressWarnings("unchecked") - Token[] blockTokens = new Token[tokenProtos.size()]; - for (int i = 0; i < blockTokens.length; i++) { - blockTokens[i] = convert(tokenProtos.get(i)); - } - - return blockTokens; - } - static public DatanodeInfo convert(DatanodeInfoProto di) { if (di == null) return null; return new DatanodeInfo( @@ -825,7 +815,9 @@ public class PBHelperClient { byte[] indices = sb.getBlockIndices(); builder.setBlockIndices(PBHelperClient.getByteString(indices)); Token[] blockTokens = sb.getBlockTokens(); - builder.addAllBlockTokens(convert(blockTokens)); + for (int i = 0; i < indices.length; i++) { + builder.addBlockTokens(PBHelperClient.convert(blockTokens[i])); + } } return builder.setB(PBHelperClient.convert(b.getBlock())) @@ -833,16 +825,6 @@ public class PBHelperClient { .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build(); } - public static List convert( - Token[] blockTokens) { - List results = new ArrayList<>(blockTokens.length); - for (Token bt : blockTokens) { - results.add(convert(bt)); - } - - return results; - } - public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) { List cList = proto.getCreationPolicy() .getStorageTypesList(); @@ -2518,14 +2500,4 @@ public class PBHelperClient { .setId(policy.getId()); return builder.build(); } - - public static HdfsProtos.DatanodeInfosProto convertToProto( - DatanodeInfo[] datanodeInfos) { - HdfsProtos.DatanodeInfosProto.Builder builder = - HdfsProtos.DatanodeInfosProto.newBuilder(); - for (DatanodeInfo datanodeInfo : datanodeInfos) { - builder.addDatanodes(PBHelperClient.convert(datanodeInfo)); - } - return builder.build(); - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 0819376aac0..43772e2efbd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -75,18 +75,6 @@ public class StripedBlockUtil { public static final Logger LOG = LoggerFactory.getLogger(StripedBlockUtil.class); - /** - * Parses a striped block group into individual blocks. - * @param bg The striped block group - * @param ecPolicy The erasure coding policy - * @return An array of the blocks in the group - */ - public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, - ErasureCodingPolicy ecPolicy) { - return parseStripedBlockGroup(bg, ecPolicy.getCellSize(), - ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits()); - } - /** * This method parses a striped block group into individual blocks. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto index 522ee06b68b..a091d417d7c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto @@ -74,6 +74,7 @@ message OpReadBlockProto { optional CachingStrategyProto cachingStrategy = 5; } + message ChecksumProto { required ChecksumTypeProto type = 1; required uint32 bytesPerChecksum = 2; @@ -148,14 +149,6 @@ message OpBlockChecksumProto { required BaseHeaderProto header = 1; } -message OpBlockGroupChecksumProto { - required BaseHeaderProto header = 1; - required DatanodeInfosProto datanodes = 2; - // each internal block has a block token - repeated hadoop.common.TokenProto blockTokens = 3; - required ErasureCodingPolicyProto ecPolicy = 4; -} - /** * An ID uniquely identifying a shared memory segment. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index b2f26f8d3cc..e0401574ce6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -26,13 +26,11 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockGroupChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; @@ -113,9 +111,6 @@ public abstract class Receiver implements DataTransferProtocol { case BLOCK_CHECKSUM: opBlockChecksum(in); break; - case BLOCK_GROUP_CHECKSUM: - opStripedBlockChecksum(in); - break; case TRANSFER_BLOCK: opTransferBlock(in); break; @@ -295,27 +290,4 @@ public abstract class Receiver implements DataTransferProtocol { if (traceScope != null) traceScope.close(); } } - - /** Receive OP_STRIPED_BLOCK_CHECKSUM. */ - private void opStripedBlockChecksum(DataInputStream dis) throws IOException { - OpBlockGroupChecksumProto proto = - OpBlockGroupChecksumProto.parseFrom(vintPrefixed(dis)); - TraceScope traceScope = continueTraceSpan(proto.getHeader(), - proto.getClass().getSimpleName()); - StripedBlockInfo stripedBlockInfo = new StripedBlockInfo( - PBHelperClient.convert(proto.getHeader().getBlock()), - PBHelperClient.convert(proto.getDatanodes()), - PBHelperClient.convertTokens(proto.getBlockTokensList()), - PBHelperClient.convertErasureCodingPolicy(proto.getEcPolicy()) - ); - - try { - blockGroupChecksum(stripedBlockInfo, - PBHelperClient.convert(proto.getHeader().getToken())); - } finally { - if (traceScope != null) { - traceScope.close(); - } - } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java index 1f1a25c4184..9a5552db3a7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java @@ -19,30 +19,16 @@ package org.apache.hadoop.hdfs.server.datanode; import com.google.common.base.Preconditions; import org.apache.hadoop.hdfs.DFSUtilClient; -import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; -import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; -import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; -import org.apache.hadoop.hdfs.protocol.datatransfer.Op; -import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos; -import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; -import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream; -import org.apache.hadoop.hdfs.util.StripedBlockUtil; -import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.BufferedInputStream; import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.security.MessageDigest; @@ -55,87 +41,13 @@ final class BlockChecksumHelper { static final Logger LOG = LoggerFactory.getLogger(BlockChecksumHelper.class); - private BlockChecksumHelper() { - } + private BlockChecksumHelper() {} /** * The abstract base block checksum computer. */ - static abstract class AbstractBlockChecksumComputer { + static abstract class BlockChecksumComputer { private final DataNode datanode; - - private byte[] outBytes; - private int bytesPerCRC = -1; - private DataChecksum.Type crcType = null; - private long crcPerBlock = -1; - private int checksumSize = -1; - - AbstractBlockChecksumComputer(DataNode datanode) throws IOException { - this.datanode = datanode; - } - - abstract void compute() throws IOException; - - Sender createSender(IOStreamPair pair) { - DataOutputStream out = (DataOutputStream) pair.out; - return new Sender(out); - } - - DataNode getDatanode() { - return datanode; - } - - InputStream getBlockInputStream(ExtendedBlock block, long seekOffset) - throws IOException { - return datanode.data.getBlockInputStream(block, seekOffset); - } - - void setOutBytes(byte[] bytes) { - this.outBytes = bytes; - } - - byte[] getOutBytes() { - return outBytes; - } - - int getBytesPerCRC() { - return bytesPerCRC; - } - - public void setBytesPerCRC(int bytesPerCRC) { - this.bytesPerCRC = bytesPerCRC; - } - - public void setCrcType(DataChecksum.Type crcType) { - this.crcType = crcType; - } - - public void setCrcPerBlock(long crcPerBlock) { - this.crcPerBlock = crcPerBlock; - } - - public void setChecksumSize(int checksumSize) { - this.checksumSize = checksumSize; - } - - DataChecksum.Type getCrcType() { - return crcType; - } - - long getCrcPerBlock() { - return crcPerBlock; - } - - int getChecksumSize() { - return checksumSize; - } - } - - /** - * The abstract base block checksum computer. - */ - static abstract class BlockChecksumComputer - extends AbstractBlockChecksumComputer { private final ExtendedBlock block; // client side now can specify a range of the block for checksum private final long requestLength; @@ -144,12 +56,17 @@ final class BlockChecksumHelper { private final long visibleLength; private final boolean partialBlk; + private byte[] outBytes; + private int bytesPerCRC = -1; + private DataChecksum.Type crcType = null; + private long crcPerBlock = -1; + private int checksumSize = -1; private BlockMetadataHeader header; private DataChecksum checksum; BlockChecksumComputer(DataNode datanode, ExtendedBlock block) throws IOException { - super(datanode); + this.datanode = datanode; this.block = block; this.requestLength = block.getNumBytes(); Preconditions.checkArgument(requestLength >= 0); @@ -164,80 +81,98 @@ final class BlockChecksumHelper { new BufferedInputStream(metadataIn, ioFileBufferSize)); } - Sender createSender(IOStreamPair pair) { - DataOutputStream out = (DataOutputStream) pair.out; - return new Sender(out); + protected DataNode getDatanode() { + return datanode; } - - ExtendedBlock getBlock() { + protected ExtendedBlock getBlock() { return block; } - long getRequestLength() { + protected long getRequestLength() { return requestLength; } - LengthInputStream getMetadataIn() { + protected LengthInputStream getMetadataIn() { return metadataIn; } - DataInputStream getChecksumIn() { + protected DataInputStream getChecksumIn() { return checksumIn; } - long getVisibleLength() { + protected long getVisibleLength() { return visibleLength; } - boolean isPartialBlk() { + protected boolean isPartialBlk() { return partialBlk; } - BlockMetadataHeader getHeader() { + protected void setOutBytes(byte[] bytes) { + this.outBytes = bytes; + } + + protected byte[] getOutBytes() { + return outBytes; + } + + protected int getBytesPerCRC() { + return bytesPerCRC; + } + + protected DataChecksum.Type getCrcType() { + return crcType; + } + + protected long getCrcPerBlock() { + return crcPerBlock; + } + + protected int getChecksumSize() { + return checksumSize; + } + + protected BlockMetadataHeader getHeader() { return header; } - DataChecksum getChecksum() { + protected DataChecksum getChecksum() { return checksum; } /** * Perform the block checksum computing. - * * @throws IOException */ abstract void compute() throws IOException; /** * Read block metadata header. - * * @throws IOException */ - void readHeader() throws IOException { + protected void readHeader() throws IOException { //read metadata file header = BlockMetadataHeader.readHeader(checksumIn); checksum = header.getChecksum(); - setChecksumSize(checksum.getChecksumSize()); - setBytesPerCRC(checksum.getBytesPerChecksum()); - long crcPerBlock = checksum.getChecksumSize() <= 0 ? 0 : + checksumSize = checksum.getChecksumSize(); + bytesPerCRC = checksum.getBytesPerChecksum(); + crcPerBlock = checksumSize <= 0 ? 0 : (metadataIn.getLength() - - BlockMetadataHeader.getHeaderSize()) / checksum.getChecksumSize(); - setCrcPerBlock(crcPerBlock); - setCrcType(checksum.getChecksumType()); + BlockMetadataHeader.getHeaderSize()) / checksumSize; + crcType = checksum.getChecksumType(); } /** * Calculate partial block checksum. - * * @return * @throws IOException */ - byte[] crcPartialBlock() throws IOException { - int partialLength = (int) (requestLength % getBytesPerCRC()); + protected byte[] crcPartialBlock() throws IOException { + int partialLength = (int) (requestLength % bytesPerCRC); if (partialLength > 0) { byte[] buf = new byte[partialLength]; - final InputStream blockIn = getBlockInputStream(block, + final InputStream blockIn = datanode.data.getBlockInputStream(block, requestLength - partialLength); try { // Get the CRC of the partialLength. @@ -246,7 +181,7 @@ final class BlockChecksumHelper { IOUtils.closeStream(blockIn); } checksum.update(buf, 0, partialLength); - byte[] partialCrc = new byte[getChecksumSize()]; + byte[] partialCrc = new byte[checksumSize]; checksum.writeValue(partialCrc, 0, true); return partialCrc; } @@ -294,7 +229,7 @@ final class BlockChecksumHelper { } private MD5Hash checksumPartialBlock() throws IOException { - byte[] buffer = new byte[4 * 1024]; + byte[] buffer = new byte[4*1024]; MessageDigest digester = MD5Hash.getDigester(); long remaining = (getRequestLength() / getBytesPerCRC()) @@ -316,115 +251,4 @@ final class BlockChecksumHelper { return new MD5Hash(digester.digest()); } } - - /** - * Non-striped block group checksum computer for striped blocks. - */ - static class BlockGroupNonStripedChecksumComputer - extends AbstractBlockChecksumComputer { - - private final ExtendedBlock blockGroup; - private final ErasureCodingPolicy ecPolicy; - private final DatanodeInfo[] datanodes; - private final Token[] blockTokens; - - private final DataOutputBuffer md5writer = new DataOutputBuffer(); - - BlockGroupNonStripedChecksumComputer(DataNode datanode, - StripedBlockInfo stripedBlockInfo) - throws IOException { - super(datanode); - this.blockGroup = stripedBlockInfo.getBlock(); - this.ecPolicy = stripedBlockInfo.getErasureCodingPolicy(); - this.datanodes = stripedBlockInfo.getDatanodes(); - this.blockTokens = stripedBlockInfo.getBlockTokens(); - } - - @Override - void compute() throws IOException { - for (int idx = 0; idx < ecPolicy.getNumDataUnits(); idx++) { - ExtendedBlock block = - StripedBlockUtil.constructInternalBlock(blockGroup, - ecPolicy.getCellSize(), ecPolicy.getNumDataUnits(), idx); - DatanodeInfo targetDatanode = datanodes[idx]; - Token blockToken = blockTokens[idx]; - checksumBlock(block, idx, blockToken, targetDatanode); - } - - MD5Hash md5out = MD5Hash.digest(md5writer.getData()); - setOutBytes(md5out.getDigest()); - } - - private void checksumBlock(ExtendedBlock block, int blockIdx, - Token blockToken, - DatanodeInfo targetDatanode) throws IOException { - int timeout = 3000; - try (IOStreamPair pair = getDatanode().connectToDN(targetDatanode, - timeout, block, blockToken)) { - - LOG.debug("write to {}: {}, block={}", - getDatanode(), Op.BLOCK_CHECKSUM, block); - - // get block MD5 - createSender(pair).blockChecksum(block, blockToken); - - final DataTransferProtos.BlockOpResponseProto reply = - DataTransferProtos.BlockOpResponseProto.parseFrom( - PBHelperClient.vintPrefixed(pair.in)); - - String logInfo = "for block " + block - + " from datanode " + targetDatanode; - DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo); - - DataTransferProtos.OpBlockChecksumResponseProto checksumData = - reply.getChecksumResponse(); - - //read byte-per-checksum - final int bpc = checksumData.getBytesPerCrc(); - if (blockIdx == 0) { //first block - setBytesPerCRC(bpc); - } else if (bpc != getBytesPerCRC()) { - throw new IOException("Byte-per-checksum not matched: bpc=" + bpc - + " but bytesPerCRC=" + getBytesPerCRC()); - } - - //read crc-per-block - final long cpb = checksumData.getCrcPerBlock(); - if (blockIdx == 0) { - setCrcPerBlock(cpb); - } - - //read md5 - final MD5Hash md5 = new MD5Hash( - checksumData.getMd5().toByteArray()); - md5.write(md5writer); - - // read crc-type - final DataChecksum.Type ct; - if (checksumData.hasCrcType()) { - ct = PBHelperClient.convert(checksumData.getCrcType()); - } else { - LOG.debug("Retrieving checksum from an earlier-version DataNode: " + - "inferring checksum by reading first byte"); - ct = DataChecksum.Type.DEFAULT; - } - - if (blockIdx == 0) { // first block - setCrcType(ct); - } else if (getCrcType() != DataChecksum.Type.MIXED && - getCrcType() != ct) { - // if crc types are mixed in a file - setCrcType(DataChecksum.Type.MIXED); - } - - if (LOG.isDebugEnabled()) { - if (blockIdx == 0) { - LOG.debug("set bytesPerCRC=" + getBytesPerCRC() - + ", crcPerBlock=" + getCrcPerBlock()); - } - LOG.debug("got reply from " + targetDatanode + ": md5=" + md5); - } - } - } - } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index 63bf5ae5363..1d4a79ac88d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; @@ -47,9 +46,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.BlockChecksumComputer; -import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.AbstractBlockChecksumComputer; import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.ReplicatedBlockChecksumComputer; -import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.BlockGroupNonStripedChecksumComputer; import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException; import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException; import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.NewShmInfo; @@ -926,46 +923,6 @@ class DataXceiver extends Receiver implements Runnable { datanode.metrics.addBlockChecksumOp(elapsed()); } - @Override - public void blockGroupChecksum(final StripedBlockInfo stripedBlockInfo, - final Token blockToken) - throws IOException { - updateCurrentThreadName("Getting checksum for block group" + - stripedBlockInfo.getBlock()); - final DataOutputStream out = new DataOutputStream(getOutputStream()); - checkAccess(out, true, stripedBlockInfo.getBlock(), blockToken, - Op.BLOCK_GROUP_CHECKSUM, BlockTokenIdentifier.AccessMode.READ); - - AbstractBlockChecksumComputer maker = - new BlockGroupNonStripedChecksumComputer(datanode, stripedBlockInfo); - - try { - maker.compute(); - - //write reply - BlockOpResponseProto.newBuilder() - .setStatus(SUCCESS) - .setChecksumResponse(OpBlockChecksumResponseProto.newBuilder() - .setBytesPerCrc(maker.getBytesPerCRC()) - .setCrcPerBlock(maker.getCrcPerBlock()) - .setMd5(ByteString.copyFrom(maker.getOutBytes())) - .setCrcType(PBHelperClient.convert(maker.getCrcType()))) - .build() - .writeDelimitedTo(out); - out.flush(); - } catch (IOException ioe) { - LOG.info("blockChecksum " + stripedBlockInfo.getBlock() + - " received exception " + ioe); - incrDatanodeNetworkErrors(); - throw ioe; - } finally { - IOUtils.closeStream(out); - } - - //update metrics - datanode.metrics.addBlockChecksumOp(elapsed()); - } - @Override public void copyBlock(final ExtendedBlock block, final Token blockToken) throws IOException { From 3a4ff7776e8fab6cc87932b9aa8fb48f7b69c720 Mon Sep 17 00:00:00 2001 From: Uma Maheswara Rao G Date: Sat, 26 Mar 2016 19:58:09 -0700 Subject: [PATCH 35/43] HDFS-9694. Make existing DFSClient#getFileChecksum() work for striped blocks. Contributed by Kai Zheng --- .../dev-support/findbugsExcludeFile.xml | 1 + .../org/apache/hadoop/hdfs/DFSClient.java | 11 +- .../hadoop/hdfs/FileChecksumHelper.java | 187 ++++++++++-- .../hdfs/protocol/StripedBlockInfo.java | 61 ++++ .../datatransfer/DataTransferProtocol.java | 16 +- .../hadoop/hdfs/protocol/datatransfer/Op.java | 1 + .../hdfs/protocol/datatransfer/Sender.java | 19 ++ .../hdfs/protocolPB/PBHelperClient.java | 42 ++- .../hadoop/hdfs/util/StripedBlockUtil.java | 12 + .../src/main/proto/datatransfer.proto | 9 +- .../hdfs/protocol/datatransfer/Receiver.java | 28 ++ .../server/datanode/BlockChecksumHelper.java | 284 ++++++++++++++---- .../hdfs/server/datanode/DataXceiver.java | 43 +++ .../apache/hadoop/hdfs/TestFileChecksum.java | 247 +++++++++++++++ 14 files changed, 878 insertions(+), 83 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/StripedBlockInfo.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml index 2c3329e47a9..9d6ab9a1029 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml @@ -8,6 +8,7 @@ + diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 3506d3ac70a..88bd21909d2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -1704,7 +1704,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, /** * Get the checksum of the whole file or a range of the file. Note that the - * range always starts from the beginning of the file. + * range always starts from the beginning of the file. The file can be + * in replicated form, or striped mode. It can be used to checksum and compare + * two replicated files, or two striped files, but not applicable for two + * files of different block layout forms. * @param src The file path * @param length the length of the range, i.e., the range is [0, length] * @return The checksum @@ -1717,7 +1720,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, LocatedBlocks blockLocations = getBlockLocations(src, length); - FileChecksumHelper.FileChecksumComputer maker = + FileChecksumHelper.FileChecksumComputer maker; + ErasureCodingPolicy ecPolicy = blockLocations.getErasureCodingPolicy(); + maker = ecPolicy != null ? + new FileChecksumHelper.StripedFileNonStripedChecksumComputer(src, + length, blockLocations, namenode, this, ecPolicy) : new FileChecksumHelper.ReplicatedFileChecksumComputer(src, length, blockLocations, namenode, this); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java index d15db9f11c9..dfd939397b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java @@ -22,10 +22,13 @@ import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum; import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; import org.apache.hadoop.hdfs.protocol.datatransfer.Op; @@ -75,7 +78,7 @@ final class FileChecksumHelper { private int bytesPerCRC = -1; private DataChecksum.Type crcType = DataChecksum.Type.DEFAULT; private long crcPerBlock = 0; - private boolean refetchBlocks = false; + private boolean isRefetchBlocks = false; private int lastRetriedIndex = -1; /** @@ -127,8 +130,11 @@ final class FileChecksumHelper { return blockLocations; } - void setBlockLocations(LocatedBlocks blockLocations) { - this.blockLocations = blockLocations; + void refetchBlocks() throws IOException { + this.blockLocations = getClient().getBlockLocations(getSrc(), + getLength()); + this.locatedBlocks = getBlockLocations().getLocatedBlocks(); + this.isRefetchBlocks = false; } int getTimeout() { @@ -143,10 +149,6 @@ final class FileChecksumHelper { return locatedBlocks; } - void setLocatedBlocks(List locatedBlocks) { - this.locatedBlocks = locatedBlocks; - } - long getRemaining() { return remaining; } @@ -180,11 +182,11 @@ final class FileChecksumHelper { } boolean isRefetchBlocks() { - return refetchBlocks; + return isRefetchBlocks; } void setRefetchBlocks(boolean refetchBlocks) { - this.refetchBlocks = refetchBlocks; + this.isRefetchBlocks = refetchBlocks; } int getLastRetriedIndex() { @@ -278,10 +280,7 @@ final class FileChecksumHelper { blockIdx < getLocatedBlocks().size() && getRemaining() >= 0; blockIdx++) { if (isRefetchBlocks()) { // refetch to get fresh tokens - setBlockLocations(getClient().getBlockLocations(getSrc(), - getLength())); - setLocatedBlocks(getBlockLocations().getLocatedBlocks()); - setRefetchBlocks(false); + refetchBlocks(); } LocatedBlock locatedBlock = getLocatedBlocks().get(blockIdx); @@ -380,15 +379,13 @@ final class FileChecksumHelper { } //read md5 - final MD5Hash md5 = new MD5Hash( - checksumData.getMd5().toByteArray()); + final MD5Hash md5 = new MD5Hash(checksumData.getMd5().toByteArray()); md5.write(getMd5out()); // read crc-type final DataChecksum.Type ct; if (checksumData.hasCrcType()) { - ct = PBHelperClient.convert(checksumData - .getCrcType()); + ct = PBHelperClient.convert(checksumData.getCrcType()); } else { LOG.debug("Retrieving checksum from an earlier-version DataNode: " + "inferring checksum by reading first byte"); @@ -413,4 +410,160 @@ final class FileChecksumHelper { } } } + + /** + * Striped file checksum computing. + */ + static class StripedFileNonStripedChecksumComputer + extends FileChecksumComputer { + private final ErasureCodingPolicy ecPolicy; + private int bgIdx; + + StripedFileNonStripedChecksumComputer(String src, long length, + LocatedBlocks blockLocations, + ClientProtocol namenode, + DFSClient client, + ErasureCodingPolicy ecPolicy) + throws IOException { + super(src, length, blockLocations, namenode, client); + + this.ecPolicy = ecPolicy; + } + + @Override + void checksumBlocks() throws IOException { + int tmpTimeout = 3000 * 1 + getClient().getConf().getSocketTimeout(); + setTimeout(tmpTimeout); + + for (bgIdx = 0; + bgIdx < getLocatedBlocks().size() && getRemaining() >= 0; bgIdx++) { + if (isRefetchBlocks()) { // refetch to get fresh tokens + refetchBlocks(); + } + + LocatedBlock locatedBlock = getLocatedBlocks().get(bgIdx); + LocatedStripedBlock blockGroup = (LocatedStripedBlock) locatedBlock; + + if (!checksumBlockGroup(blockGroup)) { + throw new IOException("Fail to get block MD5 for " + locatedBlock); + } + } + } + + + private boolean checksumBlockGroup( + LocatedStripedBlock blockGroup) throws IOException { + ExtendedBlock block = blockGroup.getBlock(); + if (getRemaining() < block.getNumBytes()) { + block.setNumBytes(getRemaining()); + } + setRemaining(getRemaining() - block.getNumBytes()); + + StripedBlockInfo stripedBlockInfo = new StripedBlockInfo(block, + blockGroup.getLocations(), blockGroup.getBlockTokens(), ecPolicy); + DatanodeInfo[] datanodes = blockGroup.getLocations(); + + //try each datanode in the block group. + boolean done = false; + for (int j = 0; !done && j < datanodes.length; j++) { + try { + tryDatanode(blockGroup, stripedBlockInfo, datanodes[j]); + done = true; + } catch (InvalidBlockTokenException ibte) { + if (bgIdx > getLastRetriedIndex()) { + LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM " + + "for file {} for block {} from datanode {}. Will retry " + + "the block once.", + getSrc(), block, datanodes[j]); + setLastRetriedIndex(bgIdx); + done = true; // actually it's not done; but we'll retry + bgIdx--; // repeat at bgIdx-th block + setRefetchBlocks(true); + } + } catch (IOException ie) { + LOG.warn("src={}" + ", datanodes[{}]={}", + getSrc(), j, datanodes[j], ie); + } + } + + return done; + } + + /** + * Return true when sounds good to continue or retry, false when severe + * condition or totally failed. + */ + private void tryDatanode(LocatedStripedBlock blockGroup, + StripedBlockInfo stripedBlockInfo, + DatanodeInfo datanode) throws IOException { + + try (IOStreamPair pair = getClient().connectToDN(datanode, + getTimeout(), blockGroup.getBlockToken())) { + + LOG.debug("write to {}: {}, blockGroup={}", + datanode, Op.BLOCK_GROUP_CHECKSUM, blockGroup); + + // get block MD5 + createSender(pair).blockGroupChecksum(stripedBlockInfo, + blockGroup.getBlockToken()); + + BlockOpResponseProto reply = BlockOpResponseProto.parseFrom( + PBHelperClient.vintPrefixed(pair.in)); + + String logInfo = "for blockGroup " + blockGroup + + " from datanode " + datanode; + DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo); + + OpBlockChecksumResponseProto checksumData = reply.getChecksumResponse(); + + //read byte-per-checksum + final int bpc = checksumData.getBytesPerCrc(); + if (bgIdx == 0) { //first block + setBytesPerCRC(bpc); + } else { + if (bpc != getBytesPerCRC()) { + throw new IOException("Byte-per-checksum not matched: bpc=" + bpc + + " but bytesPerCRC=" + getBytesPerCRC()); + } + } + + //read crc-per-block + final long cpb = checksumData.getCrcPerBlock(); + if (getLocatedBlocks().size() > 1 && bgIdx == 0) { // first block + setCrcPerBlock(cpb); + } + + //read md5 + final MD5Hash md5 = new MD5Hash( + checksumData.getMd5().toByteArray()); + md5.write(getMd5out()); + + // read crc-type + final DataChecksum.Type ct; + if (checksumData.hasCrcType()) { + ct = PBHelperClient.convert(checksumData.getCrcType()); + } else { + LOG.debug("Retrieving checksum from an earlier-version DataNode: " + + "inferring checksum by reading first byte"); + ct = getClient().inferChecksumTypeByReading(blockGroup, datanode); + } + + if (bgIdx == 0) { + setCrcType(ct); + } else if (getCrcType() != DataChecksum.Type.MIXED && + getCrcType() != ct) { + // if crc types are mixed in a file + setCrcType(DataChecksum.Type.MIXED); + } + + if (LOG.isDebugEnabled()) { + if (bgIdx == 0) { + LOG.debug("set bytesPerCRC=" + getBytesPerCRC() + + ", crcPerBlock=" + getCrcPerBlock()); + } + LOG.debug("got reply from " + datanode + ": md5=" + md5); + } + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/StripedBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/StripedBlockInfo.java new file mode 100644 index 00000000000..74e80810d4f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/StripedBlockInfo.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocol; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.security.token.Token; + +/** + * Striped block info that can be sent elsewhere to do block group level things, + * like checksum, and etc. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class StripedBlockInfo { + private final ExtendedBlock block; + private final DatanodeInfo[] datanodes; + private final Token[] blockTokens; + private final ErasureCodingPolicy ecPolicy; + + public StripedBlockInfo(ExtendedBlock block, DatanodeInfo[] datanodes, + Token[] blockTokens, + ErasureCodingPolicy ecPolicy) { + this.block = block; + this.datanodes = datanodes; + this.blockTokens = blockTokens; + this.ecPolicy = ecPolicy; + } + + public ExtendedBlock getBlock() { + return block; + } + + public DatanodeInfo[] getDatanodes() { + return datanodes; + } + + public Token[] getBlockTokens() { + return blockTokens; + } + + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java index 4aa545b3338..ad3f2ad90a1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId; @@ -197,6 +198,17 @@ public interface DataTransferProtocol { * @param blockToken security token for accessing the block. * @throws IOException */ - void blockChecksum(final ExtendedBlock blk, - final Token blockToken) throws IOException; + void blockChecksum(ExtendedBlock blk, + Token blockToken) throws IOException; + + + /** + * Get striped block group checksum (MD5 of CRC32). + * + * @param stripedBlockInfo a striped block info. + * @param blockToken security token for accessing the block. + * @throws IOException + */ + void blockGroupChecksum(StripedBlockInfo stripedBlockInfo, + Token blockToken) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java index 511574c4145..94250e5e7f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java @@ -38,6 +38,7 @@ public enum Op { REQUEST_SHORT_CIRCUIT_FDS((byte)87), RELEASE_SHORT_CIRCUIT_FDS((byte)88), REQUEST_SHORT_CIRCUIT_SHM((byte)89), + BLOCK_GROUP_CHECKSUM((byte)90), CUSTOM((byte)127); /** The code for this operation. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java index 65456815aed..585ed99b1ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java @@ -28,11 +28,13 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockGroupChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; @@ -261,4 +263,21 @@ public class Sender implements DataTransferProtocol { send(out, Op.BLOCK_CHECKSUM, proto); } + + @Override + public void blockGroupChecksum(StripedBlockInfo stripedBlockInfo, + Token blockToken) throws IOException { + OpBlockGroupChecksumProto proto = OpBlockGroupChecksumProto.newBuilder() + .setHeader(DataTransferProtoUtil.buildBaseHeader( + stripedBlockInfo.getBlock(), blockToken)) + .setDatanodes(PBHelperClient.convertToProto( + stripedBlockInfo.getDatanodes())) + .addAllBlockTokens(PBHelperClient.convert( + stripedBlockInfo.getBlockTokens())) + .setEcPolicy(PBHelperClient.convertErasureCodingPolicy( + stripedBlockInfo.getErasureCodingPolicy())) + .build(); + + send(out, Op.BLOCK_GROUP_CHECKSUM, proto); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java index 38e875c012d..47593732123 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java @@ -553,10 +553,8 @@ public class PBHelperClient { proto.getCorrupt(), cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()])); List tokenProtos = proto.getBlockTokensList(); - Token[] blockTokens = new Token[indices.length]; - for (int i = 0; i < indices.length; i++) { - blockTokens[i] = convert(tokenProtos.get(i)); - } + Token[] blockTokens = + convertTokens(tokenProtos); ((LocatedStripedBlock) lb).setBlockTokens(blockTokens); } lb.setBlockToken(convert(proto.getBlockToken())); @@ -564,6 +562,18 @@ public class PBHelperClient { return lb; } + static public Token[] convertTokens( + List tokenProtos) { + + @SuppressWarnings("unchecked") + Token[] blockTokens = new Token[tokenProtos.size()]; + for (int i = 0; i < blockTokens.length; i++) { + blockTokens[i] = convert(tokenProtos.get(i)); + } + + return blockTokens; + } + static public DatanodeInfo convert(DatanodeInfoProto di) { if (di == null) return null; return new DatanodeInfo( @@ -815,9 +825,7 @@ public class PBHelperClient { byte[] indices = sb.getBlockIndices(); builder.setBlockIndices(PBHelperClient.getByteString(indices)); Token[] blockTokens = sb.getBlockTokens(); - for (int i = 0; i < indices.length; i++) { - builder.addBlockTokens(PBHelperClient.convert(blockTokens[i])); - } + builder.addAllBlockTokens(convert(blockTokens)); } return builder.setB(PBHelperClient.convert(b.getBlock())) @@ -825,6 +833,16 @@ public class PBHelperClient { .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build(); } + public static List convert( + Token[] blockTokens) { + List results = new ArrayList<>(blockTokens.length); + for (Token bt : blockTokens) { + results.add(convert(bt)); + } + + return results; + } + public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) { List cList = proto.getCreationPolicy() .getStorageTypesList(); @@ -2500,4 +2518,14 @@ public class PBHelperClient { .setId(policy.getId()); return builder.build(); } + + public static HdfsProtos.DatanodeInfosProto convertToProto( + DatanodeInfo[] datanodeInfos) { + HdfsProtos.DatanodeInfosProto.Builder builder = + HdfsProtos.DatanodeInfosProto.newBuilder(); + for (DatanodeInfo datanodeInfo : datanodeInfos) { + builder.addDatanodes(PBHelperClient.convert(datanodeInfo)); + } + return builder.build(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 43772e2efbd..0819376aac0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -75,6 +75,18 @@ public class StripedBlockUtil { public static final Logger LOG = LoggerFactory.getLogger(StripedBlockUtil.class); + /** + * Parses a striped block group into individual blocks. + * @param bg The striped block group + * @param ecPolicy The erasure coding policy + * @return An array of the blocks in the group + */ + public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, + ErasureCodingPolicy ecPolicy) { + return parseStripedBlockGroup(bg, ecPolicy.getCellSize(), + ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits()); + } + /** * This method parses a striped block group into individual blocks. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto index a091d417d7c..522ee06b68b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto @@ -74,7 +74,6 @@ message OpReadBlockProto { optional CachingStrategyProto cachingStrategy = 5; } - message ChecksumProto { required ChecksumTypeProto type = 1; required uint32 bytesPerChecksum = 2; @@ -149,6 +148,14 @@ message OpBlockChecksumProto { required BaseHeaderProto header = 1; } +message OpBlockGroupChecksumProto { + required BaseHeaderProto header = 1; + required DatanodeInfosProto datanodes = 2; + // each internal block has a block token + repeated hadoop.common.TokenProto blockTokens = 3; + required ErasureCodingPolicyProto ecPolicy = 4; +} + /** * An ID uniquely identifying a shared memory segment. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index e0401574ce6..b2f26f8d3cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -26,11 +26,13 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockGroupChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto; @@ -111,6 +113,9 @@ public abstract class Receiver implements DataTransferProtocol { case BLOCK_CHECKSUM: opBlockChecksum(in); break; + case BLOCK_GROUP_CHECKSUM: + opStripedBlockChecksum(in); + break; case TRANSFER_BLOCK: opTransferBlock(in); break; @@ -290,4 +295,27 @@ public abstract class Receiver implements DataTransferProtocol { if (traceScope != null) traceScope.close(); } } + + /** Receive OP_STRIPED_BLOCK_CHECKSUM. */ + private void opStripedBlockChecksum(DataInputStream dis) throws IOException { + OpBlockGroupChecksumProto proto = + OpBlockGroupChecksumProto.parseFrom(vintPrefixed(dis)); + TraceScope traceScope = continueTraceSpan(proto.getHeader(), + proto.getClass().getSimpleName()); + StripedBlockInfo stripedBlockInfo = new StripedBlockInfo( + PBHelperClient.convert(proto.getHeader().getBlock()), + PBHelperClient.convert(proto.getDatanodes()), + PBHelperClient.convertTokens(proto.getBlockTokensList()), + PBHelperClient.convertErasureCodingPolicy(proto.getEcPolicy()) + ); + + try { + blockGroupChecksum(stripedBlockInfo, + PBHelperClient.convert(proto.getHeader().getToken())); + } finally { + if (traceScope != null) { + traceScope.close(); + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java index 9a5552db3a7..1f1a25c4184 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockChecksumHelper.java @@ -19,16 +19,30 @@ package org.apache.hadoop.hdfs.server.datanode; import com.google.common.base.Preconditions; import org.apache.hadoop.hdfs.DFSUtilClient; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; +import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; +import org.apache.hadoop.hdfs.protocol.datatransfer.Op; +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos; +import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.BufferedInputStream; import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.security.MessageDigest; @@ -41,13 +55,87 @@ final class BlockChecksumHelper { static final Logger LOG = LoggerFactory.getLogger(BlockChecksumHelper.class); - private BlockChecksumHelper() {} + private BlockChecksumHelper() { + } /** * The abstract base block checksum computer. */ - static abstract class BlockChecksumComputer { + static abstract class AbstractBlockChecksumComputer { private final DataNode datanode; + + private byte[] outBytes; + private int bytesPerCRC = -1; + private DataChecksum.Type crcType = null; + private long crcPerBlock = -1; + private int checksumSize = -1; + + AbstractBlockChecksumComputer(DataNode datanode) throws IOException { + this.datanode = datanode; + } + + abstract void compute() throws IOException; + + Sender createSender(IOStreamPair pair) { + DataOutputStream out = (DataOutputStream) pair.out; + return new Sender(out); + } + + DataNode getDatanode() { + return datanode; + } + + InputStream getBlockInputStream(ExtendedBlock block, long seekOffset) + throws IOException { + return datanode.data.getBlockInputStream(block, seekOffset); + } + + void setOutBytes(byte[] bytes) { + this.outBytes = bytes; + } + + byte[] getOutBytes() { + return outBytes; + } + + int getBytesPerCRC() { + return bytesPerCRC; + } + + public void setBytesPerCRC(int bytesPerCRC) { + this.bytesPerCRC = bytesPerCRC; + } + + public void setCrcType(DataChecksum.Type crcType) { + this.crcType = crcType; + } + + public void setCrcPerBlock(long crcPerBlock) { + this.crcPerBlock = crcPerBlock; + } + + public void setChecksumSize(int checksumSize) { + this.checksumSize = checksumSize; + } + + DataChecksum.Type getCrcType() { + return crcType; + } + + long getCrcPerBlock() { + return crcPerBlock; + } + + int getChecksumSize() { + return checksumSize; + } + } + + /** + * The abstract base block checksum computer. + */ + static abstract class BlockChecksumComputer + extends AbstractBlockChecksumComputer { private final ExtendedBlock block; // client side now can specify a range of the block for checksum private final long requestLength; @@ -56,17 +144,12 @@ final class BlockChecksumHelper { private final long visibleLength; private final boolean partialBlk; - private byte[] outBytes; - private int bytesPerCRC = -1; - private DataChecksum.Type crcType = null; - private long crcPerBlock = -1; - private int checksumSize = -1; private BlockMetadataHeader header; private DataChecksum checksum; BlockChecksumComputer(DataNode datanode, ExtendedBlock block) throws IOException { - this.datanode = datanode; + super(datanode); this.block = block; this.requestLength = block.getNumBytes(); Preconditions.checkArgument(requestLength >= 0); @@ -81,98 +164,80 @@ final class BlockChecksumHelper { new BufferedInputStream(metadataIn, ioFileBufferSize)); } - protected DataNode getDatanode() { - return datanode; + Sender createSender(IOStreamPair pair) { + DataOutputStream out = (DataOutputStream) pair.out; + return new Sender(out); } - protected ExtendedBlock getBlock() { + + ExtendedBlock getBlock() { return block; } - protected long getRequestLength() { + long getRequestLength() { return requestLength; } - protected LengthInputStream getMetadataIn() { + LengthInputStream getMetadataIn() { return metadataIn; } - protected DataInputStream getChecksumIn() { + DataInputStream getChecksumIn() { return checksumIn; } - protected long getVisibleLength() { + long getVisibleLength() { return visibleLength; } - protected boolean isPartialBlk() { + boolean isPartialBlk() { return partialBlk; } - protected void setOutBytes(byte[] bytes) { - this.outBytes = bytes; - } - - protected byte[] getOutBytes() { - return outBytes; - } - - protected int getBytesPerCRC() { - return bytesPerCRC; - } - - protected DataChecksum.Type getCrcType() { - return crcType; - } - - protected long getCrcPerBlock() { - return crcPerBlock; - } - - protected int getChecksumSize() { - return checksumSize; - } - - protected BlockMetadataHeader getHeader() { + BlockMetadataHeader getHeader() { return header; } - protected DataChecksum getChecksum() { + DataChecksum getChecksum() { return checksum; } /** * Perform the block checksum computing. + * * @throws IOException */ abstract void compute() throws IOException; /** * Read block metadata header. + * * @throws IOException */ - protected void readHeader() throws IOException { + void readHeader() throws IOException { //read metadata file header = BlockMetadataHeader.readHeader(checksumIn); checksum = header.getChecksum(); - checksumSize = checksum.getChecksumSize(); - bytesPerCRC = checksum.getBytesPerChecksum(); - crcPerBlock = checksumSize <= 0 ? 0 : + setChecksumSize(checksum.getChecksumSize()); + setBytesPerCRC(checksum.getBytesPerChecksum()); + long crcPerBlock = checksum.getChecksumSize() <= 0 ? 0 : (metadataIn.getLength() - - BlockMetadataHeader.getHeaderSize()) / checksumSize; - crcType = checksum.getChecksumType(); + BlockMetadataHeader.getHeaderSize()) / checksum.getChecksumSize(); + setCrcPerBlock(crcPerBlock); + setCrcType(checksum.getChecksumType()); } /** * Calculate partial block checksum. + * * @return * @throws IOException */ - protected byte[] crcPartialBlock() throws IOException { - int partialLength = (int) (requestLength % bytesPerCRC); + byte[] crcPartialBlock() throws IOException { + int partialLength = (int) (requestLength % getBytesPerCRC()); if (partialLength > 0) { byte[] buf = new byte[partialLength]; - final InputStream blockIn = datanode.data.getBlockInputStream(block, + final InputStream blockIn = getBlockInputStream(block, requestLength - partialLength); try { // Get the CRC of the partialLength. @@ -181,7 +246,7 @@ final class BlockChecksumHelper { IOUtils.closeStream(blockIn); } checksum.update(buf, 0, partialLength); - byte[] partialCrc = new byte[checksumSize]; + byte[] partialCrc = new byte[getChecksumSize()]; checksum.writeValue(partialCrc, 0, true); return partialCrc; } @@ -229,7 +294,7 @@ final class BlockChecksumHelper { } private MD5Hash checksumPartialBlock() throws IOException { - byte[] buffer = new byte[4*1024]; + byte[] buffer = new byte[4 * 1024]; MessageDigest digester = MD5Hash.getDigester(); long remaining = (getRequestLength() / getBytesPerCRC()) @@ -251,4 +316,115 @@ final class BlockChecksumHelper { return new MD5Hash(digester.digest()); } } -} + + /** + * Non-striped block group checksum computer for striped blocks. + */ + static class BlockGroupNonStripedChecksumComputer + extends AbstractBlockChecksumComputer { + + private final ExtendedBlock blockGroup; + private final ErasureCodingPolicy ecPolicy; + private final DatanodeInfo[] datanodes; + private final Token[] blockTokens; + + private final DataOutputBuffer md5writer = new DataOutputBuffer(); + + BlockGroupNonStripedChecksumComputer(DataNode datanode, + StripedBlockInfo stripedBlockInfo) + throws IOException { + super(datanode); + this.blockGroup = stripedBlockInfo.getBlock(); + this.ecPolicy = stripedBlockInfo.getErasureCodingPolicy(); + this.datanodes = stripedBlockInfo.getDatanodes(); + this.blockTokens = stripedBlockInfo.getBlockTokens(); + } + + @Override + void compute() throws IOException { + for (int idx = 0; idx < ecPolicy.getNumDataUnits(); idx++) { + ExtendedBlock block = + StripedBlockUtil.constructInternalBlock(blockGroup, + ecPolicy.getCellSize(), ecPolicy.getNumDataUnits(), idx); + DatanodeInfo targetDatanode = datanodes[idx]; + Token blockToken = blockTokens[idx]; + checksumBlock(block, idx, blockToken, targetDatanode); + } + + MD5Hash md5out = MD5Hash.digest(md5writer.getData()); + setOutBytes(md5out.getDigest()); + } + + private void checksumBlock(ExtendedBlock block, int blockIdx, + Token blockToken, + DatanodeInfo targetDatanode) throws IOException { + int timeout = 3000; + try (IOStreamPair pair = getDatanode().connectToDN(targetDatanode, + timeout, block, blockToken)) { + + LOG.debug("write to {}: {}, block={}", + getDatanode(), Op.BLOCK_CHECKSUM, block); + + // get block MD5 + createSender(pair).blockChecksum(block, blockToken); + + final DataTransferProtos.BlockOpResponseProto reply = + DataTransferProtos.BlockOpResponseProto.parseFrom( + PBHelperClient.vintPrefixed(pair.in)); + + String logInfo = "for block " + block + + " from datanode " + targetDatanode; + DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo); + + DataTransferProtos.OpBlockChecksumResponseProto checksumData = + reply.getChecksumResponse(); + + //read byte-per-checksum + final int bpc = checksumData.getBytesPerCrc(); + if (blockIdx == 0) { //first block + setBytesPerCRC(bpc); + } else if (bpc != getBytesPerCRC()) { + throw new IOException("Byte-per-checksum not matched: bpc=" + bpc + + " but bytesPerCRC=" + getBytesPerCRC()); + } + + //read crc-per-block + final long cpb = checksumData.getCrcPerBlock(); + if (blockIdx == 0) { + setCrcPerBlock(cpb); + } + + //read md5 + final MD5Hash md5 = new MD5Hash( + checksumData.getMd5().toByteArray()); + md5.write(md5writer); + + // read crc-type + final DataChecksum.Type ct; + if (checksumData.hasCrcType()) { + ct = PBHelperClient.convert(checksumData.getCrcType()); + } else { + LOG.debug("Retrieving checksum from an earlier-version DataNode: " + + "inferring checksum by reading first byte"); + ct = DataChecksum.Type.DEFAULT; + } + + if (blockIdx == 0) { // first block + setCrcType(ct); + } else if (getCrcType() != DataChecksum.Type.MIXED && + getCrcType() != ct) { + // if crc types are mixed in a file + setCrcType(DataChecksum.Type.MIXED); + } + + if (LOG.isDebugEnabled()) { + if (blockIdx == 0) { + LOG.debug("set bytesPerCRC=" + getBytesPerCRC() + + ", crcPerBlock=" + getCrcPerBlock()); + } + LOG.debug("got reply from " + targetDatanode + ": md5=" + md5); + } + } + } + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index 1d4a79ac88d..63bf5ae5363 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.StripedBlockInfo; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; @@ -46,7 +47,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.BlockChecksumComputer; +import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.AbstractBlockChecksumComputer; import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.ReplicatedBlockChecksumComputer; +import org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper.BlockGroupNonStripedChecksumComputer; import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException; import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException; import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.NewShmInfo; @@ -923,6 +926,46 @@ class DataXceiver extends Receiver implements Runnable { datanode.metrics.addBlockChecksumOp(elapsed()); } + @Override + public void blockGroupChecksum(final StripedBlockInfo stripedBlockInfo, + final Token blockToken) + throws IOException { + updateCurrentThreadName("Getting checksum for block group" + + stripedBlockInfo.getBlock()); + final DataOutputStream out = new DataOutputStream(getOutputStream()); + checkAccess(out, true, stripedBlockInfo.getBlock(), blockToken, + Op.BLOCK_GROUP_CHECKSUM, BlockTokenIdentifier.AccessMode.READ); + + AbstractBlockChecksumComputer maker = + new BlockGroupNonStripedChecksumComputer(datanode, stripedBlockInfo); + + try { + maker.compute(); + + //write reply + BlockOpResponseProto.newBuilder() + .setStatus(SUCCESS) + .setChecksumResponse(OpBlockChecksumResponseProto.newBuilder() + .setBytesPerCrc(maker.getBytesPerCRC()) + .setCrcPerBlock(maker.getCrcPerBlock()) + .setMd5(ByteString.copyFrom(maker.getOutBytes())) + .setCrcType(PBHelperClient.convert(maker.getCrcType()))) + .build() + .writeDelimitedTo(out); + out.flush(); + } catch (IOException ioe) { + LOG.info("blockChecksum " + stripedBlockInfo.getBlock() + + " received exception " + ioe); + incrDatanodeNetworkErrors(); + throw ioe; + } finally { + IOUtils.closeStream(out); + } + + //update metrics + datanode.metrics.addBlockChecksumOp(elapsed()); + } + @Override public void copyBlock(final ExtendedBlock block, final Token blockToken) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java new file mode 100644 index 00000000000..7cee3440143 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java @@ -0,0 +1,247 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileChecksum; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +/** + * This test serves a prototype to demo the idea proposed so far. It creates two + * files using the same data, one is in replica mode, the other is in stripped + * layout. For simple, it assumes 6 data blocks in both files and the block size + * are the same. + */ +public class TestFileChecksum { + public static final Log LOG = LogFactory.getLog(TestFileChecksum.class); + + private int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private Configuration conf; + private DFSClient client; + + private int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private int stripesPerBlock = 6; + private int blockSize = cellSize * stripesPerBlock; + private int numBlockGroups = 10; + private int stripSize = cellSize * dataBlocks; + private int blockGroupSize = stripesPerBlock * stripSize; + private int fileSize = numBlockGroups * blockGroupSize; + + private String ecDir = "/striped"; + private String stripedFile1 = ecDir + "/stripedFileChecksum1"; + private String stripedFile2 = ecDir + "/stripedFileChecksum2"; + private String replicatedFile = "/replicatedFileChecksum"; + + @Before + public void setup() throws IOException { + int numDNs = dataBlocks + parityBlocks + 2; + conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, + false); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + Path ecPath = new Path(ecDir); + cluster.getFileSystem().mkdir(ecPath, FsPermission.getDirDefault()); + cluster.getFileSystem().getClient().setErasureCodingPolicy(ecDir, null); + fs = cluster.getFileSystem(); + client = fs.getClient(); + + prepareTestFiles(); + + getDataNodeToKill(stripedFile1); + getDataNodeToKill(replicatedFile); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + cluster = null; + } + } + + @Test + public void testStripedFileChecksum1() throws Exception { + int length = 0; + testStripedFileChecksum(length, length + 10); + } + + @Test + public void testStripedFileChecksum2() throws Exception { + int length = stripSize - 1; + testStripedFileChecksum(length, length - 10); + } + + @Test + public void testStripedFileChecksum3() throws Exception { + int length = stripSize; + testStripedFileChecksum(length, length - 10); + } + + @Test + public void testStripedFileChecksum4() throws Exception { + int length = stripSize + cellSize * 2; + testStripedFileChecksum(length, length - 10); + } + + @Test + public void testStripedFileChecksum5() throws Exception { + int length = blockGroupSize; + testStripedFileChecksum(length, length - 10); + } + + @Test + public void testStripedFileChecksum6() throws Exception { + int length = blockGroupSize + blockSize; + testStripedFileChecksum(length, length - 10); + } + + @Test + public void testStripedFileChecksum7() throws Exception { + int length = -1; // whole file + testStripedFileChecksum(length, fileSize); + } + + void testStripedFileChecksum(int range1, int range2) throws Exception { + FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1, + range1, false); + FileChecksum stripedFileChecksum2 = getFileChecksum(stripedFile2, + range1, false); + FileChecksum stripedFileChecksum3 = getFileChecksum(stripedFile2, + range2, false); + + LOG.info("stripedFileChecksum1:" + stripedFileChecksum1); + LOG.info("stripedFileChecksum2:" + stripedFileChecksum2); + LOG.info("stripedFileChecksum3:" + stripedFileChecksum3); + + Assert.assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2)); + if (range1 >=0 && range1 != range2) { + Assert.assertFalse(stripedFileChecksum1.equals(stripedFileChecksum3)); + } + } + + @Test + public void testStripedAndReplicatedFileChecksum() throws Exception { + FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1, + 10, false); + FileChecksum replicatedFileChecksum = getFileChecksum(replicatedFile, + 10, false); + + Assert.assertFalse(stripedFileChecksum1.equals(replicatedFileChecksum)); + } + + /* + // TODO: allow datanode failure, HDFS-9833 + @Test + public void testStripedAndReplicatedWithFailure() throws Exception { + FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1, + 10, true); + FileChecksum replicatedFileChecksum = getFileChecksum(replicatedFile, + 10, true); + + Assert.assertFalse(stripedFileChecksum1.equals(replicatedFileChecksum)); + }*/ + + private FileChecksum getFileChecksum(String filePath, int range, + boolean killDn) throws Exception { + int dnIdxToDie = -1; + if (killDn) { + dnIdxToDie = getDataNodeToKill(filePath); + DataNode dnToDie = cluster.getDataNodes().get(dnIdxToDie); + shutdownDataNode(dnToDie); + } + + Path testPath = new Path(filePath); + FileChecksum fc; + + if (range >= 0) { + fc = fs.getFileChecksum(testPath, range); + } else { + fc = fs.getFileChecksum(testPath); + } + + if (dnIdxToDie != -1) { + cluster.restartDataNode(dnIdxToDie, true); + } + + return fc; + } + + void prepareTestFiles() throws IOException { + byte[] fileData = StripedFileTestUtil.generateBytes(fileSize); + + String[] filePaths = new String[] { + stripedFile1, stripedFile2, replicatedFile + }; + + for (String filePath : filePaths) { + Path testPath = new Path(filePath); + DFSTestUtil.writeFile(fs, testPath, fileData); + } + } + + void shutdownDataNode(DataNode dataNode) throws IOException { + /* + * Kill the datanode which contains one replica + * We need to make sure it dead in namenode: clear its update time and + * trigger NN to check heartbeat. + */ + dataNode.shutdown(); + cluster.setDataNodeDead(dataNode.getDatanodeId()); + } + + /** + * Determine the datanode that hosts the first block of the file. For simple + * this just returns the first datanode as it's firstly tried. + */ + int getDataNodeToKill(String filePath) throws IOException { + LocatedBlocks locatedBlocks = client.getLocatedBlocks(filePath, 0); + + LocatedBlock locatedBlock = locatedBlocks.get(0); + DatanodeInfo[] datanodes = locatedBlock.getLocations(); + DatanodeInfo chosenDn = datanodes[0]; + + int idx = 0; + for (DataNode dn : cluster.getDataNodes()) { + if (dn.getInfoPort() == chosenDn.getInfoPort()) { + return idx; + } + idx++; + } + + return -1; + } +} From 49ff54c8609431f5d1a6bee84df3b52fec445379 Mon Sep 17 00:00:00 2001 From: Karthik Kambatla Date: Sat, 26 Mar 2016 21:45:13 -0700 Subject: [PATCH 36/43] YARN-4805. Don't go through all schedulers in ParameterizedTestBase. (kasha) --- .../ParameterizedSchedulerTestBase.java | 36 ++++++------------- .../yarn/server/resourcemanager/TestRM.java | 4 --- .../server/resourcemanager/TestRMRestart.java | 4 --- .../TestWorkPreservingRMRestart.java | 9 ----- .../reservation/TestReservationSystem.java | 4 --- .../scheduler/TestAbstractYarnScheduler.java | 6 ---- .../security/TestClientToAMTokens.java | 4 --- 7 files changed, 11 insertions(+), 56 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ParameterizedSchedulerTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ParameterizedSchedulerTestBase.java index b0998368bf3..3f2bfc9f941 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ParameterizedSchedulerTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ParameterizedSchedulerTestBase.java @@ -23,19 +23,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; - import org.junit.Before; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.io.File; import java.io.FileWriter; import java.io.IOException; import java.io.PrintWriter; -import java.util.Arrays; -import java.util.Collection; -@RunWith(Parameterized.class) public abstract class ParameterizedSchedulerTestBase { protected final static String TEST_DIR = new File(System.getProperty("test.build.data", "/tmp")).getAbsolutePath(); @@ -49,31 +43,23 @@ public abstract class ParameterizedSchedulerTestBase { CAPACITY, FAIR } - public ParameterizedSchedulerTestBase(SchedulerType type) { - schedulerType = type; - } - public YarnConfiguration getConf() { return conf; } - @Parameterized.Parameters - public static Collection getParameters() { - return Arrays.asList(new SchedulerType[][]{ - {SchedulerType.CAPACITY}, {SchedulerType.FAIR}}); - } - @Before - public void configureScheduler() throws IOException { + public void configureScheduler() throws IOException, ClassNotFoundException { conf = new YarnConfiguration(); - switch (schedulerType) { - case CAPACITY: - conf.set(YarnConfiguration.RM_SCHEDULER, - CapacityScheduler.class.getName()); - break; - case FAIR: - configureFairScheduler(conf); - break; + + Class schedulerClass = + conf.getClass(YarnConfiguration.RM_SCHEDULER, + Class.forName(YarnConfiguration.DEFAULT_RM_SCHEDULER)); + + if (schedulerClass == FairScheduler.class) { + schedulerType = SchedulerType.FAIR; + configureFairScheduler(conf); + } else if (schedulerClass == CapacityScheduler.class) { + schedulerType = SchedulerType.CAPACITY; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java index 77d8cdf5243..b04b1731c82 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java @@ -91,10 +91,6 @@ public class TestRM extends ParameterizedSchedulerTestBase { private YarnConfiguration conf; - public TestRM(SchedulerType type) { - super(type); - } - @Before public void setup() { conf = getConf(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java index 3057615496b..96066877236 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java @@ -136,10 +136,6 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase { private static InetSocketAddress rmAddr; private List rms = new ArrayList(); - public TestRMRestart(SchedulerType type) { - super(type); - } - @Before public void setup() throws IOException { conf = getConf(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java index b32dfbf1056..370cbfefce9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java @@ -89,7 +89,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.Dom import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer; import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.Records; -import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -100,23 +99,15 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import com.google.common.base.Supplier; - @SuppressWarnings({"rawtypes", "unchecked"}) -@RunWith(value = Parameterized.class) public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase { private YarnConfiguration conf; MockRM rm1 = null; MockRM rm2 = null; - public TestWorkPreservingRMRestart(SchedulerType type) { - super(type); - } - @Before public void setup() throws UnknownHostException { Logger rootLogger = LogManager.getRootLogger(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystem.java index 7fcf4bcae5d..a7b7e32ff9c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystem.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystem.java @@ -53,10 +53,6 @@ public class TestReservationSystem extends private Configuration conf; private RMContext mockRMContext; - public TestReservationSystem(SchedulerType type) { - super(type); - } - @Before public void setUp() throws IOException { scheduler = initializeScheduler(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java index 81c8fe6d738..4042a2993bb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java @@ -83,10 +83,6 @@ import org.mockito.Mockito; @SuppressWarnings("unchecked") public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase { - public TestAbstractYarnScheduler(SchedulerType type) { - super(type); - } - @Test public void testMaximimumAllocationMemory() throws Exception { final int node1MaxMemory = 15 * 1024; @@ -268,7 +264,6 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase { Resource configuredMaximumResource = Resource.newInstance (configuredMaxMemory, configuredMaxVCores); - configureScheduler(); YarnConfiguration conf = getConf(); conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, configuredMaxVCores); @@ -323,7 +318,6 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase { Resource configuredMaximumResource = Resource.newInstance (configuredMaxMemory, configuredMaxVCores); - configureScheduler(); YarnConfiguration conf = getConf(); conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, configuredMaxVCores); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java index 499b4d147ca..d36fb9f6827 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java @@ -84,10 +84,6 @@ import org.junit.Test; public class TestClientToAMTokens extends ParameterizedSchedulerTestBase { private YarnConfiguration conf; - public TestClientToAMTokens(SchedulerType type) { - super(type); - } - @Before public void setup() { conf = getConf(); From 55ae1439233e8585d624b2872e1e4753ef63eebb Mon Sep 17 00:00:00 2001 From: Jian He Date: Sun, 27 Mar 2016 20:22:12 -0700 Subject: [PATCH 37/43] YARN-4117. End to end unit test with mini YARN cluster for AMRMProxy Service. Contributed by Giovanni Matteo Fumarola --- .../amrmproxy/AMRMProxyService.java | 14 +++- .../amrmproxy/DefaultRequestInterceptor.java | 7 ++ .../ContainerManagerImpl.java | 49 +++++++----- .../hadoop/yarn/server/MiniYARNCluster.java | 75 ++++++++++++++++++- 4 files changed, 122 insertions(+), 23 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java index bd6538c99f9..038c697b983 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.io.DataOutputBuffer; @@ -512,6 +513,16 @@ public class AMRMProxyService extends AbstractService implements return null; } + @Private + public InetSocketAddress getBindAddress() { + return this.listenerEndpoint; + } + + @Private + public AMRMProxyTokenSecretManager getSecretManager() { + return this.secretManager; + } + /** * Private class for handling application stop events. * @@ -546,7 +557,8 @@ public class AMRMProxyService extends AbstractService implements * ApplicationAttemptId instances. * */ - private static class RequestInterceptorChainWrapper { + @Private + public static class RequestInterceptorChainWrapper { private RequestInterceptor rootInterceptor; private ApplicationAttemptId applicationAttemptId; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java index 2c7939b0097..4457dd8f443 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java @@ -39,6 +39,8 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; + /** * Extends the AbstractRequestInterceptor class and provides an implementation * that simply forwards the AM requests to the cluster resource manager. @@ -135,4 +137,9 @@ public final class DefaultRequestInterceptor extends user.addToken(amrmToken); amrmToken.setService(ClientRMProxy.getAMRMTokenService(getConf())); } + + @VisibleForTesting + public void setRMClient(ApplicationMasterProtocol rmClient) { + this.rmClient = rmClient; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java index 94d5c1e6ede..8d09aa75bcc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java @@ -183,7 +183,7 @@ public class ContainerManagerImpl extends CompositeService implements private final ReadLock readLock; private final WriteLock writeLock; private AMRMProxyService amrmProxyService; - private boolean amrmProxyEnabled = false; + protected boolean amrmProxyEnabled = false; private long waitForContainersOnShutdownMillis; @@ -247,19 +247,7 @@ public class ContainerManagerImpl extends CompositeService implements addService(sharedCacheUploader); dispatcher.register(SharedCacheUploadEventType.class, sharedCacheUploader); - amrmProxyEnabled = - conf.getBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, - YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED); - - if (amrmProxyEnabled) { - LOG.info("AMRMProxyService is enabled. " - + "All the AM->RM requests will be intercepted by the proxy"); - this.amrmProxyService = - new AMRMProxyService(this.context, this.dispatcher); - addService(this.amrmProxyService); - } else { - LOG.info("AMRMProxyService is disabled"); - } + createAMRMProxyService(conf); waitForContainersOnShutdownMillis = conf.getLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS, @@ -272,8 +260,20 @@ public class ContainerManagerImpl extends CompositeService implements recover(); } - public boolean isARMRMProxyEnabled() { - return amrmProxyEnabled; + protected void createAMRMProxyService(Configuration conf) { + this.amrmProxyEnabled = + conf.getBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, + YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED); + + if (amrmProxyEnabled) { + LOG.info("AMRMProxyService is enabled. " + + "All the AM->RM requests will be intercepted by the proxy"); + this.setAMRMProxyService( + new AMRMProxyService(this.context, this.dispatcher)); + addService(this.getAMRMProxyService()); + } else { + LOG.info("AMRMProxyService is disabled"); + } } @SuppressWarnings("unchecked") @@ -810,9 +810,9 @@ public class ContainerManagerImpl extends CompositeService implements // Initialize the AMRMProxy service instance only if the container is of // type AM and if the AMRMProxy service is enabled - if (isARMRMProxyEnabled() && containerTokenIdentifier - .getContainerType().equals(ContainerType.APPLICATION_MASTER)) { - this.amrmProxyService.processApplicationStartRequest(request); + if (amrmProxyEnabled && containerTokenIdentifier.getContainerType() + .equals(ContainerType.APPLICATION_MASTER)) { + this.getAMRMProxyService().processApplicationStartRequest(request); } startContainerInternal(nmTokenIdentifier, containerTokenIdentifier, @@ -1413,4 +1413,15 @@ public class ContainerManagerImpl extends CompositeService implements public Map getAuxServiceMetaData() { return this.auxiliaryServices.getMetaData(); } + + @Private + public AMRMProxyService getAMRMProxyService() { + return this.amrmProxyService; + } + + @Private + protected void setAMRMProxyService(AMRMProxyService amrmProxyService) { + this.amrmProxyService = amrmProxyService; + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java index 74b77323046..c933736b32e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java @@ -35,21 +35,23 @@ import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; import org.apache.hadoop.ha.HAServiceProtocol; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell.ShellCommandExecutor; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.HAUtil; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.api.ResourceTracker; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; @@ -61,24 +63,31 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer; import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore; import org.apache.hadoop.yarn.server.applicationhistoryservice.MemoryApplicationHistoryStore; +import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.DeletionService; +import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService; import org.apache.hadoop.yarn.server.nodemanager.NodeManager; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl; +import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyService; +import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor; +import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.RequestInterceptor; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; +import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService; -import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent; +import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore; import org.apache.hadoop.yarn.server.timeline.TimelineStore; import org.apache.hadoop.yarn.server.timeline.recovery.MemoryTimelineStateStore; import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore; import org.apache.hadoop.yarn.util.timeline.TimelineUtils; -import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.webapp.util.WebAppUtils; import com.google.common.annotations.VisibleForTesting; @@ -698,6 +707,15 @@ public class MiniYARNCluster extends CompositeService { protected void stopRMProxy() { } }; } + + @Override + protected ContainerManagerImpl createContainerManager(Context context, + ContainerExecutor exec, DeletionService del, + NodeStatusUpdater nodeStatusUpdater, ApplicationACLsManager aclsManager, + LocalDirsHandlerService dirsHandler) { + return new CustomContainerManagerImpl(context, exec, del, + nodeStatusUpdater, metrics, dirsHandler); + } } /** @@ -799,4 +817,55 @@ public class MiniYARNCluster extends CompositeService { public int getNumOfResourceManager() { return this.resourceManagers.length; } + + private class CustomContainerManagerImpl extends ContainerManagerImpl { + + public CustomContainerManagerImpl(Context context, ContainerExecutor exec, + DeletionService del, NodeStatusUpdater nodeStatusUpdater, + NodeManagerMetrics metrics, LocalDirsHandlerService dirsHandler) { + super(context, exec, del, nodeStatusUpdater, metrics, dirsHandler); + } + + @Override + protected void createAMRMProxyService(Configuration conf) { + this.amrmProxyEnabled = + conf.getBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, + YarnConfiguration.DEFAULT_AMRM_PROXY_ENABLED); + + if (this.amrmProxyEnabled) { + LOG.info("CustomAMRMProxyService is enabled. " + + "All the AM->RM requests will be intercepted by the proxy"); + AMRMProxyService amrmProxyService = + useRpc ? new AMRMProxyService(getContext(), dispatcher) + : new ShortCircuitedAMRMProxy(getContext(), dispatcher); + this.setAMRMProxyService(amrmProxyService); + addService(this.getAMRMProxyService()); + } else { + LOG.info("CustomAMRMProxyService is disabled"); + } + } + } + + private class ShortCircuitedAMRMProxy extends AMRMProxyService { + + public ShortCircuitedAMRMProxy(Context context, + AsyncDispatcher dispatcher) { + super(context, dispatcher); + } + + @Override + protected void initializePipeline(ApplicationAttemptId applicationAttemptId, + String user, Token amrmToken, + Token localToken) { + super.initializePipeline(applicationAttemptId, user, amrmToken, + localToken); + RequestInterceptor rt = getPipelines() + .get(applicationAttemptId.getApplicationId()).getRootInterceptor(); + if (rt instanceof DefaultRequestInterceptor) { + ((DefaultRequestInterceptor) rt) + .setRMClient(getResourceManager().getApplicationMasterService()); + } + } + + } } From 18c7e582839ea0b550463569b18b5827d23f8849 Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Mon, 28 Mar 2016 15:32:38 +0900 Subject: [PATCH 38/43] HDFS-9640. Remove hsftp from DistCp in trunk. Contributed by Wei-Chiu Chuang. --- .../java/org/apache/hadoop/tools/DistCp.java | 79 ------------------- .../apache/hadoop/tools/DistCpConstants.java | 16 ---- .../hadoop/tools/DistCpOptionSwitch.java | 9 --- .../apache/hadoop/tools/DistCpOptions.java | 22 ------ .../apache/hadoop/tools/OptionsParser.java | 5 -- .../hadoop/tools/mapred/CopyMapper.java | 68 ---------------- .../src/site/markdown/DistCp.md.vm | 39 --------- .../hadoop/tools/TestOptionsParser.java | 17 +--- 8 files changed, 1 insertion(+), 254 deletions(-) diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java index 9db2b1b1e0f..2342fa984ae 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java @@ -19,7 +19,6 @@ package org.apache.hadoop.tools; import java.io.IOException; -import java.net.URL; import java.util.Random; import org.apache.commons.logging.Log; @@ -254,88 +253,10 @@ public class DistCp extends Configured implements Tool { job.getConfiguration().set(JobContext.NUM_MAPS, String.valueOf(inputOptions.getMaxMaps())); - if (inputOptions.getSslConfigurationFile() != null) { - setupSSLConfig(job); - } - inputOptions.appendToConf(job.getConfiguration()); return job; } - /** - * Setup ssl configuration on the job configuration to enable hsftp access - * from map job. Also copy the ssl configuration file to Distributed cache - * - * @param job - Reference to job's handle - * @throws java.io.IOException - Exception if unable to locate ssl config file - */ - private void setupSSLConfig(Job job) throws IOException { - Configuration configuration = job.getConfiguration(); - URL sslFileUrl = configuration.getResource(inputOptions - .getSslConfigurationFile()); - if (sslFileUrl == null) { - throw new IOException( - "Given ssl configuration file doesn't exist in class path : " - + inputOptions.getSslConfigurationFile()); - } - Path sslConfigPath = new Path(sslFileUrl.toString()); - - addSSLFilesToDistCache(job, sslConfigPath); - configuration.set(DistCpConstants.CONF_LABEL_SSL_CONF, sslConfigPath.getName()); - configuration.set(DistCpConstants.CONF_LABEL_SSL_KEYSTORE, sslConfigPath.getName()); - } - - /** - * Add SSL files to distributed cache. Trust store, key store and ssl config xml - * - * @param job - Job handle - * @param sslConfigPath - ssl Configuration file specified through options - * @throws IOException - If any - */ - private void addSSLFilesToDistCache(Job job, - Path sslConfigPath) throws IOException { - Configuration configuration = job.getConfiguration(); - FileSystem localFS = FileSystem.getLocal(configuration); - - Configuration sslConf = new Configuration(false); - sslConf.addResource(sslConfigPath); - - Path localStorePath = getLocalStorePath(sslConf, - DistCpConstants.CONF_LABEL_SSL_TRUST_STORE_LOCATION); - job.addCacheFile(localStorePath.makeQualified(localFS.getUri(), - localFS.getWorkingDirectory()).toUri()); - configuration.set(DistCpConstants.CONF_LABEL_SSL_TRUST_STORE_LOCATION, - localStorePath.getName()); - - localStorePath = getLocalStorePath(sslConf, - DistCpConstants.CONF_LABEL_SSL_KEY_STORE_LOCATION); - job.addCacheFile(localStorePath.makeQualified(localFS.getUri(), - localFS.getWorkingDirectory()).toUri()); - configuration.set(DistCpConstants.CONF_LABEL_SSL_KEY_STORE_LOCATION, - localStorePath.getName()); - - job.addCacheFile(sslConfigPath.makeQualified(localFS.getUri(), - localFS.getWorkingDirectory()).toUri()); - - } - - /** - * Get Local Trust store/key store path - * - * @param sslConf - Config from SSL Client xml - * @param storeKey - Key for either trust store or key store - * @return - Path where the store is present - * @throws IOException -If any - */ - private Path getLocalStorePath(Configuration sslConf, String storeKey) throws IOException { - if (sslConf.get(storeKey) != null) { - return new Path(sslConf.get(storeKey)); - } else { - throw new IOException("Store for " + storeKey + " is not set in " + - inputOptions.getSslConfigurationFile()); - } - } - /** * Setup output format appropriately * diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java index 93d6a621cf2..95d26df3ae3 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java @@ -49,7 +49,6 @@ public class DistCpConstants { "distcp.preserve.rawxattrs"; public static final String CONF_LABEL_SYNC_FOLDERS = "distcp.sync.folders"; public static final String CONF_LABEL_DELETE_MISSING = "distcp.delete.missing.source"; - public static final String CONF_LABEL_SSL_CONF = "distcp.keystore.resource"; public static final String CONF_LABEL_LISTSTATUS_THREADS = "distcp.liststatus.threads"; public static final String CONF_LABEL_MAX_MAPS = "distcp.max.maps"; public static final String CONF_LABEL_SOURCE_LISTING = "distcp.source.listing"; @@ -76,9 +75,6 @@ public class DistCpConstants { /* Total number of paths to copy, includes directories. Unfiltered count */ public static final String CONF_LABEL_TOTAL_NUMBER_OF_RECORDS = "mapred.number.of.records"; - /* SSL keystore resource */ - public static final String CONF_LABEL_SSL_KEYSTORE = "dfs.https.client.keystore.resource"; - /* If input is based -f <>, file containing the src paths */ public static final String CONF_LABEL_LISTING_FILE_PATH = "distcp.listing.file.path"; @@ -106,18 +102,6 @@ public class DistCpConstants { /* DistCp CopyListing class override param */ public static final String CONF_LABEL_COPY_LISTING_CLASS = "distcp.copy.listing.class"; - /** - * Conf label for SSL Trust-store location. - */ - public static final String CONF_LABEL_SSL_TRUST_STORE_LOCATION - = "ssl.client.truststore.location"; - - /** - * Conf label for SSL Key-store location. - */ - public static final String CONF_LABEL_SSL_KEY_STORE_LOCATION - = "ssl.client.keystore.location"; - /** * Constants for DistCp return code to shell / consumer of ToolRunner's run */ diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java index 94855842b06..c1048966e90 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java @@ -74,15 +74,6 @@ public enum DistCpOptionSwitch { DELETE_MISSING(DistCpConstants.CONF_LABEL_DELETE_MISSING, new Option("delete", false, "Delete from target, " + "files missing in source")), - - /** - * Configuration file to use with hftps:// for securely copying - * files across clusters. Typically the configuration file contains - * truststore/keystore information such as location, password and type - */ - SSL_CONF(DistCpConstants.CONF_LABEL_SSL_CONF, - new Option("mapredSslConf", true, "Configuration for ssl config file" + - ", to use with hftps://. Must be in the classpath.")), /** * Number of threads for building source file listing (before map-reduce * phase, max one listStatus per thread at a time). diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java index cbd4ac077cb..0b1234b962b 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java @@ -49,8 +49,6 @@ public class DistCpOptions { private int maxMaps = DistCpConstants.DEFAULT_MAPS; private float mapBandwidth = DistCpConstants.DEFAULT_BANDWIDTH_MB; - private String sslConfigurationFile; - private String copyStrategy = DistCpConstants.UNIFORMSIZE; private EnumSet preserveStatus = EnumSet.noneOf(FileAttribute.class); @@ -134,7 +132,6 @@ public class DistCpOptions { this.numListstatusThreads = that.numListstatusThreads; this.maxMaps = that.maxMaps; this.mapBandwidth = that.mapBandwidth; - this.sslConfigurationFile = that.getSslConfigurationFile(); this.copyStrategy = that.copyStrategy; this.preserveStatus = that.preserveStatus; this.preserveRawXattrs = that.preserveRawXattrs; @@ -380,24 +377,6 @@ public class DistCpOptions { this.mapBandwidth = mapBandwidth; } - /** - * Get path where the ssl configuration file is present to use for hftps:// - * - * @return Path on local file system - */ - public String getSslConfigurationFile() { - return sslConfigurationFile; - } - - /** - * Set the SSL configuration file path to use with hftps:// (local path) - * - * @param sslConfigurationFile - Local ssl config file path - */ - public void setSslConfigurationFile(String sslConfigurationFile) { - this.sslConfigurationFile = sslConfigurationFile; - } - /** * Returns an iterator with the list of file attributes to preserve * @@ -670,7 +649,6 @@ public class DistCpOptions { ", numListstatusThreads=" + numListstatusThreads + ", maxMaps=" + maxMaps + ", mapBandwidth=" + mapBandwidth + - ", sslConfigurationFile='" + sslConfigurationFile + '\'' + ", copyStrategy='" + copyStrategy + '\'' + ", preserveStatus=" + preserveStatus + ", preserveRawXattrs=" + preserveRawXattrs + diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java index b41451346df..0f0ef2e39c5 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java @@ -137,11 +137,6 @@ public class OptionsParser { parseBandwidth(command, option); - if (command.hasOption(DistCpOptionSwitch.SSL_CONF.getSwitch())) { - option.setSslConfigurationFile(command. - getOptionValue(DistCpOptionSwitch.SSL_CONF.getSwitch())); - } - parseNumListStatusThreads(command, option); parseMaxMaps(command, option); diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java index f75fe76178c..09bceadbc59 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java @@ -19,10 +19,7 @@ package org.apache.hadoop.tools.mapred; import java.io.FileNotFoundException; -import java.io.FileOutputStream; import java.io.IOException; -import java.io.OutputStream; -import java.util.Arrays; import java.util.EnumSet; import org.apache.commons.logging.Log; @@ -33,7 +30,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.tools.CopyListingFileStatus; import org.apache.hadoop.tools.DistCpConstants; @@ -119,73 +115,9 @@ public class CopyMapper extends Mapper overWrite = true; // When target is an existing file, overwrite it. } - if (conf.get(DistCpConstants.CONF_LABEL_SSL_CONF) != null) { - initializeSSLConf(context); - } startEpoch = System.currentTimeMillis(); } - /** - * Initialize SSL Config if same is set in conf - * - * @throws IOException - If any - */ - private void initializeSSLConf(Context context) throws IOException { - LOG.info("Initializing SSL configuration"); - - String workDir = conf.get(JobContext.JOB_LOCAL_DIR) + "/work"; - Path[] cacheFiles = context.getLocalCacheFiles(); - - Configuration sslConfig = new Configuration(false); - String sslConfFileName = conf.get(DistCpConstants.CONF_LABEL_SSL_CONF); - Path sslClient = findCacheFile(cacheFiles, sslConfFileName); - if (sslClient == null) { - LOG.warn("SSL Client config file not found. Was looking for " + sslConfFileName + - " in " + Arrays.toString(cacheFiles)); - return; - } - sslConfig.addResource(sslClient); - - String trustStoreFile = conf.get("ssl.client.truststore.location"); - Path trustStorePath = findCacheFile(cacheFiles, trustStoreFile); - sslConfig.set("ssl.client.truststore.location", trustStorePath.toString()); - - String keyStoreFile = conf.get("ssl.client.keystore.location"); - Path keyStorePath = findCacheFile(cacheFiles, keyStoreFile); - sslConfig.set("ssl.client.keystore.location", keyStorePath.toString()); - - try { - OutputStream out = new FileOutputStream(workDir + "/" + sslConfFileName); - try { - sslConfig.writeXml(out); - } finally { - out.close(); - } - conf.set(DistCpConstants.CONF_LABEL_SSL_KEYSTORE, sslConfFileName); - } catch (IOException e) { - LOG.warn("Unable to write out the ssl configuration. " + - "Will fall back to default ssl-client.xml in class path, if there is one", e); - } - } - - /** - * Find entry from distributed cache - * - * @param cacheFiles - All localized cache files - * @param fileName - fileName to search - * @return Path of the filename if found, else null - */ - private Path findCacheFile(Path[] cacheFiles, String fileName) { - if (cacheFiles != null && cacheFiles.length > 0) { - for (Path file : cacheFiles) { - if (file.getName().equals(fileName)) { - return file; - } - } - } - return null; - } - /** * Implementation of the Mapper::map(). Does the copy. * @param relPath The target path. diff --git a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm index bac5eccafd1..4dd15832dd8 100644 --- a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm +++ b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm @@ -32,7 +32,6 @@ DistCp Guide - [Map sizing](#Map_sizing) - [Copying Between Versions of HDFS](#Copying_Between_Versions_of_HDFS) - [MapReduce and other side-effects](#MapReduce_and_other_side-effects) - - [SSL Configurations for HSFTP sources](#SSL_Configurations_for_HSFTP_sources) - [Frequently Asked Questions](#Frequently_Asked_Questions) --- @@ -232,7 +231,6 @@ Flag | Description | Notes `-strategy {dynamic|uniformsize}` | Choose the copy-strategy to be used in DistCp. | By default, uniformsize is used. (i.e. Maps are balanced on the total size of files copied by each map. Similar to legacy.) If "dynamic" is specified, `DynamicInputFormat` is used instead. (This is described in the Architecture section, under InputFormats.) `-bandwidth` | Specify bandwidth per map, in MB/second. | Each map will be restricted to consume only the specified bandwidth. This is not always exact. The map throttles back its bandwidth consumption during a copy, such that the **net** bandwidth used tends towards the specified value. `-atomic {-tmp }` | Specify atomic commit, with optional tmp directory. | `-atomic` instructs DistCp to copy the source data to a temporary target location, and then move the temporary target to the final-location atomically. Data will either be available at final target in a complete and consistent form, or not at all. Optionally, `-tmp` may be used to specify the location of the tmp-target. If not specified, a default is chosen. **Note:** tmp_dir must be on the final target cluster. -`-mapredSslConf ` | Specify SSL Config file, to be used with HSFTP source | When using the hsftp protocol with a source, the security- related properties may be specified in a config-file and passed to DistCp. \ needs to be in the classpath. `-async` | Run DistCp asynchronously. Quits as soon as the Hadoop Job is launched. | The Hadoop Job-id is logged, for tracking. `-diff` | Use snapshot diff report to identify the difference between source and target. | `-numListstatusThreads` | Number of threads to use for building file listing | At most 40 threads. @@ -432,43 +430,6 @@ $H3 MapReduce and other side-effects * If `mapreduce.map.speculative` is set set final and true, the result of the copy is undefined. -$H3 SSL Configurations for HSFTP sources - - To use an HSFTP source (i.e. using the hsftp protocol), a SSL configuration - file needs to be specified (via the `-mapredSslConf` option). This must - specify 3 parameters: - - * `ssl.client.truststore.location`: The local-filesystem location of the - trust-store file, containing the certificate for the NameNode. - * `ssl.client.truststore.type`: (Optional) The format of the trust-store - file. - * `ssl.client.truststore.password`: (Optional) Password for the trust-store - file. - - The following is an example SSL configuration file: - - - - ssl.client.truststore.location - /work/keystore.jks - Truststore to be used by clients like distcp. Must be specified. - - - - ssl.client.truststore.password - changeme - Optional. Default value is "". - - - - ssl.client.truststore.type - jks - Optional. Default value is "jks". - - - - The SSL configuration file must be in the class-path of the DistCp program. - Frequently Asked Questions -------------------------- diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java index 4c136a89ab2..6186f30b801 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java @@ -251,21 +251,6 @@ public class TestOptionsParser { } catch (IllegalArgumentException ignore) { } } - @Test - public void testParseSSLConf() { - DistCpOptions options = OptionsParser.parse(new String[] { - "hdfs://localhost:8020/source/first", - "hdfs://localhost:8020/target/"}); - Assert.assertNull(options.getSslConfigurationFile()); - - options = OptionsParser.parse(new String[] { - "-mapredSslConf", - "/tmp/ssl-client.xml", - "hdfs://localhost:8020/source/first", - "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getSslConfigurationFile(), "/tmp/ssl-client.xml"); - } - @Test public void testParseMaps() { DistCpOptions options = OptionsParser.parse(new String[] { @@ -402,7 +387,7 @@ public class TestOptionsParser { String val = "DistCpOptions{atomicCommit=false, syncFolder=false, " + "deleteMissing=false, ignoreFailures=false, overwrite=false, " + "skipCRC=false, blocking=true, numListstatusThreads=0, maxMaps=20, " - + "mapBandwidth=100.0, sslConfigurationFile='null', " + + "mapBandwidth=100.0, " + "copyStrategy='uniformsize', preserveStatus=[], " + "preserveRawXattrs=false, atomicWorkPath=null, logPath=null, " + "sourceFileListing=abc, sourcePaths=null, targetPath=xyz, " From 01cfee63815a1c9d63652edc21db63626df7e53c Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Mon, 28 Mar 2016 15:57:57 +0900 Subject: [PATCH 39/43] HADOOP-12873. Remove MRv1 terms from HttpAuthentication.md. Contributed by Brahma Reddy Battula. --- .../hadoop-common/src/site/markdown/HttpAuthentication.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/HttpAuthentication.md b/hadoop-common-project/hadoop-common/src/site/markdown/HttpAuthentication.md index a003fe5156b..d4eeb0e845e 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/HttpAuthentication.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/HttpAuthentication.md @@ -22,11 +22,11 @@ Introduction This document describes how to configure Hadoop HTTP web-consoles to require user authentication. -By default Hadoop HTTP web-consoles (JobTracker, NameNode, TaskTrackers and DataNodes) allow access without any form of authentication. +By default Hadoop HTTP web-consoles (ResourceManager, NameNode, NodeManagers and DataNodes) allow access without any form of authentication. Hadoop HTTP web-consoles can be configured to require Kerberos authentication using HTTP SPNEGO protocol (supported by browsers like Firefox and Internet Explorer). -In addition, Hadoop HTTP web-consoles support the equivalent of Hadoop's Pseudo/Simple authentication. If this option is enabled, the user name must be specified in the first browser interaction using the user.name query string parameter. e.g. `http://localhost:50030/jobtracker.jsp?user.name=babu`. +In addition, Hadoop HTTP web-consoles support the equivalent of Hadoop's Pseudo/Simple authentication. If this option is enabled, the user name must be specified in the first browser interaction using the user.name query string parameter. e.g. `http://localhost:8088/cluster?user.name=babu`. If a custom authentication mechanism is required for the HTTP web-consoles, it is possible to implement a plugin to support the alternate authentication mechanism (refer to Hadoop hadoop-auth for details on writing an `AuthenticatorHandler`). @@ -43,7 +43,7 @@ The following properties should be in the `core-site.xml` of all the nodes in th | `hadoop.http.authentication.type` | `simple` | Defines authentication used for the HTTP web-consoles. The supported values are: `simple` \| `kerberos` \| `#AUTHENTICATION_HANDLER_CLASSNAME#`. | | `hadoop.http.authentication.token.validity` | `36000` | Indicates how long (in seconds) an authentication token is valid before it has to be renewed. | | `hadoop.http.authentication.token.max-inactive-interval` | `-1` (disabled) | Specifies the time, in seconds, between client requests the server will invalidate the token. | -| `hadoop.http.authentication.signature.secret.file` | `$user.home/hadoop-http-auth-signature-secret` | The signature secret file for signing the authentication tokens. The same secret should be used for all nodes in the cluster, JobTracker, NameNode, DataNode and TastTracker. This file should be readable only by the Unix user running the daemons. | +| `hadoop.http.authentication.signature.secret.file` | `$user.home/hadoop-http-auth-signature-secret` | The signature secret file for signing the authentication tokens. The same secret should be used for all nodes in the cluster, ResourceManager, NameNode, DataNode and NodeManager. This file should be readable only by the Unix user running the daemons. | | `hadoop.http.authentication.cookie.domain` | | The domain to use for the HTTP cookie that stores the authentication token. For authentication to work correctly across all nodes in the cluster the domain must be correctly set. There is no default value, the HTTP cookie will not have a domain working only with the hostname issuing the HTTP cookie. | | `hadoop.http.authentication.cookie.persistent` | `false` (session cookie) | Specifies the persistence of the HTTP cookie. If the value is true, the cookie is a persistent one. Otherwise, it is a session cookie. *IMPORTANT*: when using IP addresses, browsers ignore cookies with domain settings. For this setting to work properly all nodes in the cluster must be configured to generate URLs with `hostname.domain` names on it. | | `hadoop.http.authentication.simple.anonymous.allowed` | `true` | Indicates whether anonymous requests are allowed when using 'simple' authentication. | From 115be193df07277e23818288ce9258609f272df3 Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Mon, 28 Mar 2016 16:05:35 +0900 Subject: [PATCH 40/43] Revert "MAPREDUCE-6543. Migrate MR client test cases part 2. Contributed by Dustin Cote." This reverts commit 2c268cc9365851f5b02d967d13c8c0cbca850a86. --- .../java/org/apache/hadoop/fs/DFSCIOTest.java | 8 +-- .../org/apache/hadoop/fs/TestFileSystem.java | 18 ++----- .../java/org/apache/hadoop/fs/TestJHLA.java | 6 +-- .../io/TestSequenceFileMergeProgress.java | 12 ++--- .../mapred/ClusterMapReduceTestCase.java | 14 ++--- .../apache/hadoop/mapred/TestAuditLogger.java | 9 ++-- .../apache/hadoop/mapred/TestBadRecords.java | 8 +-- .../mapred/TestClusterMapReduceTestCase.java | 10 ---- .../org/apache/hadoop/mapred/TestCollect.java | 10 ++-- .../mapred/TestCommandLineJobSubmission.java | 9 ++-- .../hadoop/mapred/TestFieldSelection.java | 6 +-- .../mapred/TestFileInputFormatPathFilter.java | 19 +++---- .../hadoop/mapred/TestGetSplitHosts.java | 7 ++- .../hadoop/mapred/TestIFileStreams.java | 13 +++-- .../apache/hadoop/mapred/TestInputPath.java | 7 ++- .../hadoop/mapred/TestJavaSerialization.java | 10 ++-- .../org/apache/hadoop/mapred/TestJobName.java | 6 --- .../hadoop/mapred/TestJobSysDirWithDFS.java | 10 ++-- .../mapred/TestKeyValueTextInputFormat.java | 15 +++--- .../apache/hadoop/mapred/TestLazyOutput.java | 7 ++- .../mapred/TestMRCJCFileInputFormat.java | 32 +++++------- .../mapred/TestMRCJCFileOutputCommitter.java | 28 ++++------ .../apache/hadoop/mapred/TestMapProgress.java | 9 ++-- .../org/apache/hadoop/mapred/TestMerge.java | 7 ++- .../hadoop/mapred/TestMiniMRBringup.java | 6 +-- .../hadoop/mapred/TestMiniMRDFSCaching.java | 14 ++--- .../mapred/TestMultiFileInputFormat.java | 19 ++++--- .../hadoop/mapred/TestMultiFileSplit.java | 10 ++-- .../mapred/TestMultipleLevelCaching.java | 12 ++--- .../mapred/TestMultipleTextOutputFormat.java | 23 ++++---- .../apache/hadoop/mapred/TestReduceFetch.java | 10 ++-- .../mapred/TestReduceFetchFromPartialMem.java | 46 +++++++++------- .../apache/hadoop/mapred/TestReduceTask.java | 18 +++---- .../TestSequenceFileAsBinaryInputFormat.java | 19 +++---- .../TestSequenceFileAsBinaryOutputFormat.java | 31 ++++------- .../TestSequenceFileAsTextInputFormat.java | 27 +++++----- .../mapred/TestSequenceFileInputFilter.java | 32 ++++++------ .../mapred/TestSequenceFileInputFormat.java | 26 +++++----- .../hadoop/mapred/TestSortedRanges.java | 19 +++---- .../TestSpecialCharactersInOutputPath.java | 21 ++++---- .../mapred/TestStatisticsCollector.java | 10 ++-- .../mapred/TestUserDefinedCounters.java | 24 ++++----- .../hadoop/mapred/TestWritableJobConf.java | 20 +++---- .../apache/hadoop/mapred/TestYARNRunner.java | 8 ++- .../hadoop/mapred/join/TestDatamerge.java | 42 +++++++-------- .../hadoop/mapred/join/TestTupleWritable.java | 24 +++------ .../TestWrappedRecordReaderClassloader.java | 7 ++- .../mapred/lib/TestDelegatingInputFormat.java | 9 ++-- .../mapred/lib/TestLineInputFormat.java | 7 ++- .../hadoop/mapred/lib/TestMultipleInputs.java | 2 + .../mapred/lib/aggregate/TestAggregates.java | 7 ++- .../mapred/lib/db/TestConstructQuery.java | 16 +++--- .../apache/hadoop/mapred/pipes/TestPipes.java | 9 ++-- .../hadoop/mapreduce/TestLocalRunner.java | 34 +++++------- .../hadoop/mapreduce/TestMRJobClient.java | 49 ++++++++--------- .../mapreduce/TestMapReduceLazyOutput.java | 9 ++-- .../hadoop/mapreduce/TestValueIterReset.java | 8 ++- .../TestYarnClientProtocolProvider.java | 5 +- .../aggregate/TestMapReduceAggregates.java | 23 ++++---- .../mapreduce/lib/db/TestDBOutputFormat.java | 17 +++--- .../mapreduce/lib/db/TestIntegerSplitter.java | 15 ++---- .../mapreduce/lib/db/TestTextSplitter.java | 18 ++----- .../lib/fieldsel/TestMRFieldSelection.java | 20 +++---- ...TestMRSequenceFileAsBinaryInputFormat.java | 21 +++----- .../TestMRSequenceFileAsTextInputFormat.java | 27 ++++------ .../input/TestMRSequenceFileInputFilter.java | 39 +++++++------- .../lib/input/TestNLineInputFormat.java | 34 +++++------- .../mapreduce/lib/join/TestJoinDatamerge.java | 52 ++++++++----------- .../lib/join/TestJoinProperties.java | 44 ++++++++-------- .../lib/join/TestJoinTupleWritable.java | 24 +++------ .../lib/join/TestWrappedRRClassloader.java | 17 ++---- ...estMRSequenceFileAsBinaryOutputFormat.java | 35 ++++--------- .../lib/partition/TestBinaryPartitioner.java | 16 ++---- .../lib/partition/TestKeyFieldHelper.java | 9 +--- .../TestMRKeyFieldBasedPartitioner.java | 6 +-- .../partition/TestTotalOrderPartitioner.java | 11 ++-- .../util/TestMRAsyncDiskService.java | 15 ++---- .../mapreduce/v2/TestMiniMRProxyUser.java | 30 +++++------ .../mapreduce/v2/TestNonExistentJob.java | 18 +++---- .../streaming/TestStreamingBadRecords.java | 9 +--- 80 files changed, 569 insertions(+), 834 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java index 12bec0869f6..1caa2cdae6c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/DFSCIOTest.java @@ -28,6 +28,8 @@ import java.io.PrintStream; import java.util.Date; import java.util.StringTokenizer; +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -37,9 +39,8 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.mapred.*; import org.junit.Ignore; -import org.junit.Test; - /** +/** * Distributed i/o benchmark. *

* This test writes into or reads from a specified number of files. @@ -67,7 +68,7 @@ import org.junit.Test; * */ @Ignore -public class DFSCIOTest { +public class DFSCIOTest extends TestCase { // Constants private static final Log LOG = LogFactory.getLog(DFSCIOTest.class); private static final int TEST_TYPE_READ = 0; @@ -97,7 +98,6 @@ public class DFSCIOTest { * * @throws Exception */ - @Test public void testIOs() throws Exception { testIOs(10, 10); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java index 72840cdfd17..4146b139c50 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java @@ -34,6 +34,8 @@ import java.util.HashMap; import java.net.InetSocketAddress; import java.net.URI; +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; @@ -48,15 +50,8 @@ import org.apache.hadoop.mapred.*; import org.apache.hadoop.mapred.lib.LongSumReducer; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.StringUtils; -import org.junit.Test; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.fail; - -public class TestFileSystem { +public class TestFileSystem extends TestCase { private static final Log LOG = FileSystem.LOG; private static Configuration conf = new Configuration(); @@ -71,7 +66,6 @@ public class TestFileSystem { private static Path READ_DIR = new Path(ROOT, "fs_read"); private static Path DATA_DIR = new Path(ROOT, "fs_data"); - @Test public void testFs() throws Exception { testFs(10 * MEGA, 100, 0); } @@ -96,7 +90,6 @@ public class TestFileSystem { fs.delete(READ_DIR, true); } - @Test public static void testCommandFormat() throws Exception { // This should go to TestFsShell.java when it is added. CommandFormat cf; @@ -495,7 +488,6 @@ public class TestFileSystem { } } - @Test public void testFsCache() throws Exception { { long now = System.currentTimeMillis(); @@ -569,7 +561,6 @@ public class TestFileSystem { + StringUtils.toUpperCase(add.getHostName()) + ":" + add.getPort())); } - @Test public void testFsClose() throws Exception { { Configuration conf = new Configuration(); @@ -578,7 +569,6 @@ public class TestFileSystem { } } - @Test public void testFsShutdownHook() throws Exception { final Set closed = Collections.synchronizedSet(new HashSet()); Configuration conf = new Configuration(); @@ -610,7 +600,7 @@ public class TestFileSystem { assertTrue(closed.contains(fsWithoutAuto)); } - @Test + public void testCacheKeysAreCaseInsensitive() throws Exception { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java index 31950fd6104..f2bc4edc46d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestJHLA.java @@ -23,18 +23,19 @@ import java.io.FileOutputStream; import java.io.OutputStreamWriter; import java.io.File; +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.junit.After; import org.junit.Before; -import org.junit.Test; /** * Test Job History Log Analyzer. * * @see JHLogAnalyzer */ -public class TestJHLA { +public class TestJHLA extends TestCase { private static final Log LOG = LogFactory.getLog(JHLogAnalyzer.class); private String historyLog = System.getProperty("test.build.data", "build/test/data") + "/history/test.log"; @@ -132,7 +133,6 @@ public class TestJHLA { /** * Run log analyzer in test mode for file test.log. */ - @Test public void testJHLA() { String[] args = {"-test", historyLog, "-jobDelimiter", ".!!FILE=.*!!"}; JHLogAnalyzer.main(args); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java index 97dfa26acf4..1d7b98a6719 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java @@ -32,25 +32,21 @@ import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.mapred.*; +import junit.framework.TestCase; import org.apache.commons.logging.*; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -public class TestSequenceFileMergeProgress { +public class TestSequenceFileMergeProgress extends TestCase { private static final Log LOG = FileInputFormat.LOG; private static final int RECORDS = 10000; - - @Test + public void testMergeProgressWithNoCompression() throws IOException { runTest(SequenceFile.CompressionType.NONE); } - @Test public void testMergeProgressWithRecordCompression() throws IOException { runTest(SequenceFile.CompressionType.RECORD); } - @Test public void testMergeProgressWithBlockCompression() throws IOException { runTest(SequenceFile.CompressionType.BLOCK); } @@ -96,7 +92,7 @@ public class TestSequenceFileMergeProgress { count++; } assertEquals(RECORDS, count); - assertEquals(1.0f, rIter.getProgress().get(), 0.0000); + assertEquals(1.0f, rIter.getProgress().get()); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java index 8d33b1580a8..5bf4ff11b89 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ClusterMapReduceTestCase.java @@ -17,11 +17,10 @@ */ package org.apache.hadoop.mapred; +import junit.framework.TestCase; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.junit.After; -import org.junit.Before; import java.io.IOException; import java.util.Map; @@ -42,7 +41,7 @@ import java.util.Properties; *

* The DFS filesystem is formated before the testcase starts and after it ends. */ -public abstract class ClusterMapReduceTestCase { +public abstract class ClusterMapReduceTestCase extends TestCase { private MiniDFSCluster dfsCluster = null; private MiniMRCluster mrCluster = null; @@ -51,8 +50,9 @@ public abstract class ClusterMapReduceTestCase { * * @throws Exception */ - @Before - public void setUp() throws Exception { + protected void setUp() throws Exception { + super.setUp(); + startCluster(true, null); } @@ -139,9 +139,9 @@ public abstract class ClusterMapReduceTestCase { * * @throws Exception */ - @After - public void tearDown() throws Exception { + protected void tearDown() throws Exception { stopCluster(); + super.tearDown(); } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java index bc85703bc84..353185b59e3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java @@ -28,13 +28,13 @@ import org.apache.hadoop.ipc.TestRPC.TestImpl; import org.apache.hadoop.ipc.TestRPC.TestProtocol; import org.apache.hadoop.mapred.AuditLogger.Keys; import org.apache.hadoop.net.NetUtils; -import org.junit.Test; -import static org.junit.Assert.assertEquals; + +import junit.framework.TestCase; /** * Tests {@link AuditLogger}. */ -public class TestAuditLogger { +public class TestAuditLogger extends TestCase { private static final String USER = "test"; private static final String OPERATION = "oper"; private static final String TARGET = "tgt"; @@ -44,7 +44,6 @@ public class TestAuditLogger { /** * Test the AuditLog format with key-val pair. */ - @Test public void testKeyValLogFormat() { StringBuilder actLog = new StringBuilder(); StringBuilder expLog = new StringBuilder(); @@ -115,7 +114,6 @@ public class TestAuditLogger { /** * Test {@link AuditLogger} without IP set. */ - @Test public void testAuditLoggerWithoutIP() throws Exception { // test without ip testSuccessLogFormat(false); @@ -139,7 +137,6 @@ public class TestAuditLogger { /** * Test {@link AuditLogger} with IP set. */ - @Test public void testAuditLoggerWithIP() throws Exception { Configuration conf = new Configuration(); // start the IPC server diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java index c2d6257823e..ea9f3d3f989 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestBadRecords.java @@ -40,11 +40,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.TaskCounter; import org.apache.hadoop.util.ReflectionUtils; import org.junit.Ignore; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertNotNull; @Ignore public class TestBadRecords extends ClusterMapReduceTestCase { @@ -211,8 +206,7 @@ public class TestBadRecords extends ClusterMapReduceTestCase { } return processed; } - - @Test + public void testBadMapRed() throws Exception { JobConf conf = createJobConf(); conf.setMapperClass(BadMapper.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java index f04fbd7a29a..ada2d0c634b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java @@ -29,12 +29,6 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; -import org.junit.Test; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertFalse; public class TestClusterMapReduceTestCase extends ClusterMapReduceTestCase { public void _testMapReduce(boolean restart) throws Exception { OutputStream os = getFileSystem().create(new Path(getInputDir(), "text.txt")); @@ -91,17 +85,14 @@ public class TestClusterMapReduceTestCase extends ClusterMapReduceTestCase { } - @Test public void testMapReduce() throws Exception { _testMapReduce(false); } - @Test public void testMapReduceRestarting() throws Exception { _testMapReduce(true); } - @Test public void testDFSRestart() throws Exception { Path file = new Path(getInputDir(), "text.txt"); OutputStream os = getFileSystem().create(file); @@ -118,7 +109,6 @@ public class TestClusterMapReduceTestCase extends ClusterMapReduceTestCase { } - @Test public void testMRConfig() throws Exception { JobConf conf = createJobConf(); assertNull(conf.get("xyz")); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCollect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCollect.java index 595d09cc2a0..4bd20d54ad5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCollect.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCollect.java @@ -21,15 +21,15 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.io.*; import org.apache.hadoop.mapred.UtilsForTests.RandomInputFormat; import org.apache.hadoop.mapreduce.MRConfig; -import org.junit.Test; +import junit.framework.TestCase; import java.io.*; import java.util.*; /** * TestCollect checks if the collect can handle simultaneous invocations. */ -public class TestCollect +public class TestCollect extends TestCase { final static Path OUTPUT_DIR = new Path("build/test/test.collect.output"); static final int NUM_FEEDERS = 10; @@ -127,7 +127,7 @@ public class TestCollect conf.setNumMapTasks(1); conf.setNumReduceTasks(1); } - @Test + public void testCollect() throws IOException { JobConf conf = new JobConf(); configure(conf); @@ -144,5 +144,9 @@ public class TestCollect fs.delete(OUTPUT_DIR, true); } } + + public static void main(String[] args) throws IOException { + new TestCollect().testCollect(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java index 7cf5e71e1a5..69353871cf4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java @@ -21,29 +21,28 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.Ignore; -import org.junit.Test; -import static org.junit.Assert.assertTrue; /** * check for the job submission options of * -libjars -files -archives */ @Ignore -public class TestCommandLineJobSubmission { - // Input output paths for this.. +public class TestCommandLineJobSubmission extends TestCase { + // Input output paths for this.. // these are all dummy and does not test // much in map reduce except for the command line // params static final Path input = new Path("/test/input/"); static final Path output = new Path("/test/output"); File buildDir = new File(System.getProperty("test.build.data", "/tmp")); - @Test public void testJobShell() throws Exception { MiniDFSCluster dfs = null; MiniMRCluster mr = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFieldSelection.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFieldSelection.java index 7d7a7b0330f..239c239230e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFieldSelection.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFieldSelection.java @@ -23,12 +23,11 @@ import org.apache.hadoop.mapred.lib.*; import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.lib.fieldsel.FieldSelectionHelper; import org.apache.hadoop.mapreduce.lib.fieldsel.TestMRFieldSelection; -import org.junit.Test; -import static org.junit.Assert.assertEquals; +import junit.framework.TestCase; import java.text.NumberFormat; -public class TestFieldSelection { +public class TestFieldSelection extends TestCase { private static NumberFormat idFormat = NumberFormat.getInstance(); static { @@ -36,7 +35,6 @@ private static NumberFormat idFormat = NumberFormat.getInstance(); idFormat.setGroupingUsed(false); } - @Test public void testFieldSelection() throws Exception { launch(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileInputFormatPathFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileInputFormatPathFilter.java index d87f6fd91a9..1c8be66d084 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileInputFormatPathFilter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileInputFormatPathFilter.java @@ -17,14 +17,12 @@ */ package org.apache.hadoop.mapred; +import junit.framework.TestCase; + import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import static org.junit.Assert.assertEquals; import java.io.IOException; import java.io.Writer; @@ -32,7 +30,7 @@ import java.io.OutputStreamWriter; import java.util.Set; import java.util.HashSet; -public class TestFileInputFormatPathFilter { +public class TestFileInputFormatPathFilter extends TestCase { public static class DummyFileInputFormat extends FileInputFormat { @@ -57,12 +55,12 @@ public class TestFileInputFormatPathFilter { new Path(new Path(System.getProperty("test.build.data", "."), "data"), "TestFileInputFormatPathFilter"); - @Before + public void setUp() throws Exception { tearDown(); localFs.mkdirs(workDir); } - @After + public void tearDown() throws Exception { if (localFs.exists(workDir)) { localFs.delete(workDir, true); @@ -131,19 +129,18 @@ public class TestFileInputFormatPathFilter { assertEquals(createdFiles, computedFiles); } - @Test public void testWithoutPathFilterWithoutGlob() throws Exception { _testInputFiles(false, false); } - @Test + public void testWithoutPathFilterWithGlob() throws Exception { _testInputFiles(false, true); } - @Test + public void testWithPathFilterWithoutGlob() throws Exception { _testInputFiles(true, false); } - @Test + public void testWithPathFilterWithGlob() throws Exception { _testInputFiles(true, true); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestGetSplitHosts.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestGetSplitHosts.java index 3d1c2e71bff..7891bca7990 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestGetSplitHosts.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestGetSplitHosts.java @@ -20,11 +20,10 @@ package org.apache.hadoop.mapred; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.net.NetworkTopology; -import org.junit.Test; -import static org.junit.Assert.assertTrue; +import junit.framework.TestCase; + +public class TestGetSplitHosts extends TestCase { -public class TestGetSplitHosts { - @Test public void testGetSplitHosts() throws Exception { int numBlocks = 3; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java index 2b97d3b95ad..86431e5c135 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestIFileStreams.java @@ -21,12 +21,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; -import org.junit.Test; -import static org.junit.Assert.fail; -import static org.junit.Assert.assertEquals; -public class TestIFileStreams { - @Test +import junit.framework.TestCase; + +public class TestIFileStreams extends TestCase { + public void testIFileStream() throws Exception { final int DLEN = 100; DataOutputBuffer dob = new DataOutputBuffer(DLEN + 4); @@ -43,7 +42,7 @@ public class TestIFileStreams { } ifis.close(); } - @Test + public void testBadIFileStream() throws Exception { final int DLEN = 100; DataOutputBuffer dob = new DataOutputBuffer(DLEN + 4); @@ -74,7 +73,7 @@ public class TestIFileStreams { } fail("Did not detect bad data in checksum"); } - @Test + public void testBadLength() throws Exception { final int DLEN = 100; DataOutputBuffer dob = new DataOutputBuffer(DLEN + 4); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestInputPath.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestInputPath.java index 0c20c335d89..1398f9e5aaa 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestInputPath.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestInputPath.java @@ -17,15 +17,14 @@ */ package org.apache.hadoop.mapred; +import junit.framework.TestCase; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.util.StringUtils; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -public class TestInputPath { - @Test +public class TestInputPath extends TestCase { public void testInputPath() throws Exception { JobConf jobConf = new JobConf(); Path workingDir = jobConf.getWorkingDirectory(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJavaSerialization.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJavaSerialization.java index a787e68c124..265118a70f6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJavaSerialization.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJavaSerialization.java @@ -26,6 +26,8 @@ import java.io.Writer; import java.util.Iterator; import java.util.StringTokenizer; +import junit.framework.TestCase; + import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; @@ -34,11 +36,8 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.serializer.JavaSerializationComparator; import org.apache.hadoop.mapreduce.MRConfig; -import org.junit.Test; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; -public class TestJavaSerialization { +public class TestJavaSerialization extends TestCase { private static String TEST_ROOT_DIR = new File(System.getProperty("test.build.data", "/tmp")).toURI() @@ -91,7 +90,7 @@ public class TestJavaSerialization { wr.write("b a\n"); wr.close(); } - @Test + public void testMapReduceJob() throws Exception { JobConf conf = new JobConf(TestJavaSerialization.class); @@ -150,7 +149,6 @@ public class TestJavaSerialization { * coupled to Writable types, if so, the job will fail. * */ - @Test public void testWriteToSequencefile() throws Exception { JobConf conf = new JobConf(TestJavaSerialization.class); conf.setJobName("JavaSerialization"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobName.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobName.java index 2659a14a70b..4b62b4a1d8e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobName.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobName.java @@ -29,13 +29,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.lib.IdentityMapper; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; - public class TestJobName extends ClusterMapReduceTestCase { - @Test public void testComplexName() throws Exception { OutputStream os = getFileSystem().create(new Path(getInputDir(), "text.txt")); @@ -70,7 +65,6 @@ public class TestJobName extends ClusterMapReduceTestCase { reader.close(); } - @Test public void testComplexNameWithRegex() throws Exception { OutputStream os = getFileSystem().create(new Path(getInputDir(), "text.txt")); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java index 3dbc5777bd5..109c781c2b0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java @@ -21,6 +21,8 @@ package org.apache.hadoop.mapred; import java.io.DataOutputStream; import java.io.IOException; +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -30,15 +32,11 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; /** * A JUnit test to test Job System Directory with Mini-DFS. */ -public class TestJobSysDirWithDFS { +public class TestJobSysDirWithDFS extends TestCase { private static final Log LOG = LogFactory.getLog(TestJobSysDirWithDFS.class.getName()); @@ -117,7 +115,7 @@ public class TestJobSysDirWithDFS { // between Job Client & Job Tracker assertTrue(result.job.isSuccessful()); } - @Test + public void testWithDFS() throws IOException { MiniDFSCluster dfs = null; MiniMRCluster mr = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java index bacc196008e..27070783e14 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java @@ -20,6 +20,7 @@ package org.apache.hadoop.mapred; import java.io.*; import java.util.*; +import junit.framework.TestCase; import org.apache.commons.logging.*; import org.apache.hadoop.fs.*; @@ -27,11 +28,8 @@ import org.apache.hadoop.io.*; import org.apache.hadoop.io.compress.*; import org.apache.hadoop.util.LineReader; import org.apache.hadoop.util.ReflectionUtils; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -public class TestKeyValueTextInputFormat { +public class TestKeyValueTextInputFormat extends TestCase { private static final Log LOG = LogFactory.getLog(TestKeyValueTextInputFormat.class.getName()); @@ -49,7 +47,7 @@ public class TestKeyValueTextInputFormat { private static Path workDir = new Path(new Path(System.getProperty("test.build.data", "."), "data"), "TestKeyValueTextInputFormat"); - @Test + public void testFormat() throws Exception { JobConf job = new JobConf(); Path file = new Path(workDir, "test.txt"); @@ -136,7 +134,7 @@ public class TestKeyValueTextInputFormat { (str.getBytes("UTF-8")), defaultConf); } - @Test + public void testUTF8() throws Exception { LineReader in = null; @@ -155,7 +153,7 @@ public class TestKeyValueTextInputFormat { } } } - @Test + public void testNewLines() throws Exception { LineReader in = null; try { @@ -221,8 +219,7 @@ public class TestKeyValueTextInputFormat { /** * Test using the gzip codec for reading */ - @Test - public void testGzip() throws IOException { + public static void testGzip() throws IOException { JobConf job = new JobConf(); CompressionCodec gzip = new GzipCodec(); ReflectionUtils.setConf(gzip, job); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java index dde9310607f..7412832d5c2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java @@ -35,15 +35,14 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.lib.LazyOutputFormat; -import org.junit.Test; -import static org.junit.Assert.assertTrue; +import junit.framework.TestCase; /** * A JUnit test to test the Map-Reduce framework's feature to create part * files only if there is an explicit output.collect. This helps in preventing * 0 byte files */ -public class TestLazyOutput { +public class TestLazyOutput extends TestCase { private static final int NUM_HADOOP_SLAVES = 3; private static final int NUM_MAPS_PER_NODE = 2; private static final Path INPUT = new Path("/testlazy/input"); @@ -133,7 +132,7 @@ public class TestLazyOutput { } } - @Test + public void testLazyOutput() throws Exception { MiniDFSCluster dfs = null; MiniMRCluster mr = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java index 20d0173cc81..fb9e8fcce3a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java @@ -17,6 +17,16 @@ */ package org.apache.hadoop.mapred; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.concurrent.TimeoutException; + +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataOutputStream; @@ -26,21 +36,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.io.Text; -import org.junit.After; -import org.junit.Test; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.util.concurrent.TimeoutException; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; @SuppressWarnings("deprecation") -public class TestMRCJCFileInputFormat { +public class TestMRCJCFileInputFormat extends TestCase { Configuration conf = new Configuration(); MiniDFSCluster dfs = null; @@ -52,7 +50,6 @@ public class TestMRCJCFileInputFormat { .build(); } - @Test public void testLocality() throws Exception { JobConf job = new JobConf(conf); dfs = newDFSCluster(job); @@ -112,7 +109,6 @@ public class TestMRCJCFileInputFormat { DFSTestUtil.waitReplication(fs, path, replication); } - @Test public void testNumInputs() throws Exception { JobConf job = new JobConf(conf); dfs = newDFSCluster(job); @@ -161,7 +157,6 @@ public class TestMRCJCFileInputFormat { } } - @Test public void testMultiLevelInput() throws Exception { JobConf job = new JobConf(conf); @@ -200,7 +195,6 @@ public class TestMRCJCFileInputFormat { } @SuppressWarnings("rawtypes") - @Test public void testLastInputSplitAtSplitBoundary() throws Exception { FileInputFormat fif = new FileInputFormatForTest(1024l * 1024 * 1024, 128l * 1024 * 1024); @@ -214,7 +208,6 @@ public class TestMRCJCFileInputFormat { } @SuppressWarnings("rawtypes") - @Test public void testLastInputSplitExceedingSplitBoundary() throws Exception { FileInputFormat fif = new FileInputFormatForTest(1027l * 1024 * 1024, 128l * 1024 * 1024); @@ -228,7 +221,6 @@ public class TestMRCJCFileInputFormat { } @SuppressWarnings("rawtypes") - @Test public void testLastInputSplitSingleSplit() throws Exception { FileInputFormat fif = new FileInputFormatForTest(100l * 1024 * 1024, 128l * 1024 * 1024); @@ -313,7 +305,7 @@ public class TestMRCJCFileInputFormat { DFSTestUtil.waitReplication(fileSys, name, replication); } - @After + @Override public void tearDown() throws Exception { if (dfs != null) { dfs.shutdown(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileOutputCommitter.java index 74b6d77f6a0..3b86f81cc23 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileOutputCommitter.java @@ -18,25 +18,18 @@ package org.apache.hadoop.mapred; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RawLocalFileSystem; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.JobStatus; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; +import java.io.*; import java.net.URI; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import junit.framework.TestCase; -public class TestMRCJCFileOutputCommitter { +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; +import org.apache.hadoop.mapred.JobContextImpl; +import org.apache.hadoop.mapred.TaskAttemptContextImpl; +import org.apache.hadoop.mapreduce.JobStatus; + +public class TestMRCJCFileOutputCommitter extends TestCase { private static Path outDir = new Path( System.getProperty("test.build.data", "/tmp"), "output"); @@ -74,7 +67,6 @@ public class TestMRCJCFileOutputCommitter { } @SuppressWarnings("unchecked") - @Test public void testCommitter() throws Exception { JobConf job = new JobConf(); setConfForFileOutputCommitter(job); @@ -116,7 +108,6 @@ public class TestMRCJCFileOutputCommitter { FileUtil.fullyDelete(new File(outDir.toString())); } - @Test public void testAbort() throws IOException { JobConf job = new JobConf(); setConfForFileOutputCommitter(job); @@ -170,7 +161,6 @@ public class TestMRCJCFileOutputCommitter { } } - @Test public void testFailAbort() throws IOException { JobConf job = new JobConf(); job.set(FileSystem.FS_DEFAULT_NAME_KEY, "faildel:///"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java index b8ff016d6af..db6348ba440 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapProgress.java @@ -22,6 +22,8 @@ import java.io.File; import java.io.IOException; import java.util.List; +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; @@ -38,8 +40,6 @@ import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo; import org.apache.hadoop.mapreduce.split.JobSplitWriter; import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader; import org.apache.hadoop.util.ReflectionUtils; -import org.junit.Test; -import static org.junit.Assert.assertTrue; /** * Validates map phase progress. @@ -55,7 +55,7 @@ import static org.junit.Assert.assertTrue; * once mapTask.run() is finished. Sort phase progress in map task is not * validated here. */ -public class TestMapProgress { +public class TestMapProgress extends TestCase { public static final Log LOG = LogFactory.getLog(TestMapProgress.class); private static String TEST_ROOT_DIR; static { @@ -234,8 +234,7 @@ public class TestMapProgress { /** * Validates map phase progress after each record is processed by map task * using custom task reporter. - */ - @Test + */ public void testMapProgress() throws Exception { JobConf job = new JobConf(); fs = FileSystem.getLocal(job); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java index a9e7f64c0b8..e19ff589fa4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java @@ -44,8 +44,8 @@ import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.mapred.Task.TaskReporter; -import org.junit.Test; -import static org.junit.Assert.assertEquals; + +import junit.framework.TestCase; @SuppressWarnings(value={"unchecked", "deprecation"}) /** @@ -56,7 +56,7 @@ import static org.junit.Assert.assertEquals; * framework's merge on the reduce side will merge the partitions created to * generate the final output which is sorted on the key. */ -public class TestMerge { +public class TestMerge extends TestCase { private static final int NUM_HADOOP_DATA_NODES = 2; // Number of input files is same as the number of mappers. private static final int NUM_MAPPERS = 10; @@ -69,7 +69,6 @@ public class TestMerge { // Where output goes. private static final Path OUTPUT = new Path("/testplugin/output"); - @Test public void testMerge() throws Exception { MiniDFSCluster dfsCluster = null; MiniMRClientCluster mrCluster = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRBringup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRBringup.java index b608d756a49..8b7b8f51b96 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRBringup.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRBringup.java @@ -18,16 +18,14 @@ package org.apache.hadoop.mapred; -import org.junit.Test; - import java.io.IOException; +import junit.framework.TestCase; /** * A Unit-test to test bringup and shutdown of Mini Map-Reduce Cluster. */ -public class TestMiniMRBringup { +public class TestMiniMRBringup extends TestCase { - @Test public void testBringUp() throws IOException { MiniMRCluster mr = null; try { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java index 3f64f7a35b9..45879aff623 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java @@ -18,23 +18,20 @@ package org.apache.hadoop.mapred; +import java.io.*; +import junit.framework.TestCase; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.mapred.MRCaching.TestResult; import org.junit.Ignore; -import org.junit.Test; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; /** * A JUnit test to test caching with DFS * */ @Ignore -public class TestMiniMRDFSCaching { +public class TestMiniMRDFSCaching extends TestCase { - @Test public void testWithDFS() throws IOException { MiniMRCluster mr = null; MiniDFSCluster dfs = null; @@ -73,4 +70,9 @@ public class TestMiniMRDFSCaching { } } } + + public static void main(String[] argv) throws Exception { + TestMiniMRDFSCaching td = new TestMiniMRDFSCaching(); + td.testWithDFS(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java index 1bd29542fcd..49825e99f57 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java @@ -21,17 +21,17 @@ import java.io.IOException; import java.util.BitSet; import java.util.HashMap; import java.util.Random; + +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -public class TestMultiFileInputFormat { +public class TestMultiFileInputFormat extends TestCase{ private static JobConf job = new JobConf(); @@ -79,8 +79,7 @@ public class TestMultiFileInputFormat { FileInputFormat.setInputPaths(job, multiFileDir); return multiFileDir; } - - @Test + public void testFormat() throws IOException { LOG.info("Test started"); LOG.info("Max split count = " + MAX_SPLIT_COUNT); @@ -123,8 +122,7 @@ public class TestMultiFileInputFormat { } LOG.info("Test Finished"); } - - @Test + public void testFormatWithLessPathsThanSplits() throws Exception { MultiFileInputFormat format = new DummyMultiFileInputFormat(); FileSystem fs = FileSystem.getLocal(job); @@ -137,4 +135,9 @@ public class TestMultiFileInputFormat { initFiles(fs, 2, 500); assertEquals(2, format.getSplits(job, 4).length); } + + public static void main(String[] args) throws Exception{ + TestMultiFileInputFormat test = new TestMultiFileInputFormat(); + test.testFormat(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileSplit.java index 5bb336e4e81..16ff6af9271 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileSplit.java @@ -27,19 +27,16 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Arrays; +import junit.framework.TestCase; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * * test MultiFileSplit class */ -public class TestMultiFileSplit { +public class TestMultiFileSplit extends TestCase{ - @Test public void testReadWrite() throws Exception { MultiFileSplit split = new MultiFileSplit(new JobConf(), new Path[] {new Path("/test/path/1"), new Path("/test/path/2")}, new long[] {100,200}); @@ -73,7 +70,6 @@ public class TestMultiFileSplit { * test method getLocations * @throws IOException */ - @Test public void testgetLocations() throws IOException{ JobConf job= new JobConf(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java index 7e8dfef03f1..294723a9c87 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.mapred; +import java.io.IOException; + +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -28,17 +32,12 @@ import org.apache.hadoop.mapred.lib.IdentityReducer; import org.apache.hadoop.mapreduce.JobCounter; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; import org.junit.Ignore; -import org.junit.Test; - -import java.io.IOException; - -import static org.junit.Assert.assertEquals; /** * This test checks whether the task caches are created and used properly. */ @Ignore -public class TestMultipleLevelCaching { +public class TestMultipleLevelCaching extends TestCase { private static final int MAX_LEVEL = 5; final Path inDir = new Path("/cachetesting"); final Path outputPath = new Path("/output"); @@ -72,7 +71,6 @@ public class TestMultipleLevelCaching { return rack.toString(); } - @Test public void testMultiLevelCaching() throws Exception { for (int i = 1 ; i <= MAX_LEVEL; ++i) { testCachingAtLevel(i); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java index b5047fc8331..14c097d77e1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java @@ -18,19 +18,15 @@ package org.apache.hadoop.mapred; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.lib.MultipleTextOutputFormat; -import org.junit.Test; +import java.io.*; +import junit.framework.TestCase; -import java.io.File; -import java.io.IOException; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import org.apache.hadoop.mapred.lib.*; -public class TestMultipleTextOutputFormat { +public class TestMultipleTextOutputFormat extends TestCase { private static JobConf defaultConf = new JobConf(); private static FileSystem localFs = null; @@ -87,8 +83,7 @@ public class TestMultipleTextOutputFormat { writeData(rw); rw.close(null); } - - @Test + public void testFormat() throws Exception { JobConf job = new JobConf(); job.set(JobContext.TASK_ATTEMPT_ID, attempt); @@ -150,4 +145,8 @@ public class TestMultipleTextOutputFormat { //System.out.printf("File_2 output: %s\n", output); assertEquals(output, expectedOutput.toString()); } + + public static void main(String[] args) throws Exception { + new TestMultipleTextOutputFormat().testFormat(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetch.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetch.java index 767459f88b4..586df38dcfc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetch.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetch.java @@ -19,18 +19,17 @@ package org.apache.hadoop.mapred; import org.apache.hadoop.mapreduce.TaskCounter; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; public class TestReduceFetch extends TestReduceFetchFromPartialMem { + static { + setSuite(TestReduceFetch.class); + } + /** * Verify that all segments are read from disk * @throws Exception might be thrown */ - @Test public void testReduceFromDisk() throws Exception { final int MAP_TASKS = 8; JobConf job = mrCluster.createJobConf(); @@ -54,7 +53,6 @@ public class TestReduceFetch extends TestReduceFetchFromPartialMem { * Verify that no segment hits disk. * @throws Exception might be thrown */ - @Test public void testReduceFromMem() throws Exception { final int MAP_TASKS = 3; JobConf job = mrCluster.createJobConf(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java index 9b04f64ac60..3a1a275ab91 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java @@ -18,6 +18,10 @@ package org.apache.hadoop.mapred; +import junit.extensions.TestSetup; +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -26,9 +30,7 @@ import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.mapreduce.TaskCounter; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.apache.hadoop.mapreduce.MRConfig; import java.io.DataInput; import java.io.DataOutput; @@ -37,27 +39,34 @@ import java.util.Arrays; import java.util.Formatter; import java.util.Iterator; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class TestReduceFetchFromPartialMem { +public class TestReduceFetchFromPartialMem extends TestCase { protected static MiniMRCluster mrCluster = null; protected static MiniDFSCluster dfsCluster = null; + protected static TestSuite mySuite; - @Before - public void setUp() throws Exception { - Configuration conf = new Configuration(); - dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); - mrCluster = new MiniMRCluster(2, - dfsCluster.getFileSystem().getUri().toString(), 1); + protected static void setSuite(Class klass) { + mySuite = new TestSuite(klass); } - @After - public void tearDown() throws Exception { - if (dfsCluster != null) { dfsCluster.shutdown(); } - if (mrCluster != null) { mrCluster.shutdown(); } + static { + setSuite(TestReduceFetchFromPartialMem.class); + } + + public static Test suite() { + TestSetup setup = new TestSetup(mySuite) { + protected void setUp() throws Exception { + Configuration conf = new Configuration(); + dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + mrCluster = new MiniMRCluster(2, + dfsCluster.getFileSystem().getUri().toString(), 1); + } + protected void tearDown() throws Exception { + if (dfsCluster != null) { dfsCluster.shutdown(); } + if (mrCluster != null) { mrCluster.shutdown(); } + } + }; + return setup; } private static final String tagfmt = "%04d"; @@ -69,7 +78,6 @@ public class TestReduceFetchFromPartialMem { } /** Verify that at least one segment does not hit disk */ - @Test public void testReduceFromPartialMem() throws Exception { final int MAP_TASKS = 7; JobConf job = mrCluster.createJobConf(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceTask.java index 69546a6cba2..43fd94871a2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceTask.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceTask.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.mapred; +import java.io.IOException; + +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; @@ -26,17 +30,11 @@ import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.util.Progressable; -import org.junit.Test; - -import java.io.IOException; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; /** * This test exercises the ValueIterator. */ -public class TestReduceTask { +public class TestReduceTask extends TestCase { static class NullProgress implements Progressable { public void progress() { } @@ -121,10 +119,9 @@ public class TestReduceTask { } assertEquals(vals.length, i); // make sure we have progress equal to 1.0 - assertEquals(1.0f, rawItr.getProgress().get(),0.0000); + assertEquals(1.0f, rawItr.getProgress().get()); } - @Test public void testValueIterator() throws Exception { Path tmpDir = new Path("build/test/test.reduce.task"); Configuration conf = new Configuration(); @@ -132,8 +129,7 @@ public class TestReduceTask { runValueIterator(tmpDir, testCase, conf, null); } } - - @Test + public void testValueIteratorWithCompression() throws Exception { Path tmpDir = new Path("build/test/test.reduce.task.compression"); Configuration conf = new Configuration(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java index 64b0983a5d6..b8be7400070 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java @@ -18,26 +18,19 @@ package org.apache.hadoop.mapred; -import org.apache.commons.logging.Log; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.DataInputBuffer; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; -import org.junit.Test; - import java.io.IOException; import java.util.Random; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; -public class TestSequenceFileAsBinaryInputFormat { +import junit.framework.TestCase; +import org.apache.commons.logging.*; + +public class TestSequenceFileAsBinaryInputFormat extends TestCase { private static final Log LOG = FileInputFormat.LOG; private static final int RECORDS = 10000; - @Test public void testBinary() throws IOException { JobConf job = new JobConf(); FileSystem fs = FileSystem.getLocal(job); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java index 03dc6a69003..abe21f223ef 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsBinaryOutputFormat.java @@ -18,35 +18,24 @@ package org.apache.hadoop.mapred; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BooleanWritable; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.DataInputBuffer; -import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.io.DoubleWritable; -import org.apache.hadoop.io.FloatWritable; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.SequenceFile.CompressionType; -import org.junit.Test; - import java.io.IOException; import java.util.Random; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; +import org.apache.hadoop.io.SequenceFile.CompressionType; -public class TestSequenceFileAsBinaryOutputFormat { +import junit.framework.TestCase; +import org.apache.commons.logging.*; + +public class TestSequenceFileAsBinaryOutputFormat extends TestCase { private static final Log LOG = LogFactory.getLog(TestSequenceFileAsBinaryOutputFormat.class.getName()); + private static final int RECORDS = 10000; // A random task attempt id for testing. private static final String attempt = "attempt_200707121733_0001_m_000000_0"; - @Test public void testBinary() throws IOException { JobConf job = new JobConf(); FileSystem fs = FileSystem.getLocal(job); @@ -140,8 +129,7 @@ public class TestSequenceFileAsBinaryOutputFormat { assertEquals("Some records not found", RECORDS, count); } - @Test - public void testSequenceOutputClassDefaultsToMapRedOutputClass() + public void testSequenceOutputClassDefaultsToMapRedOutputClass() throws IOException { JobConf job = new JobConf(); FileSystem fs = FileSystem.getLocal(job); @@ -175,7 +163,6 @@ public class TestSequenceFileAsBinaryOutputFormat { job)); } - @Test public void testcheckOutputSpecsForbidRecordCompression() throws IOException { JobConf job = new JobConf(); FileSystem fs = FileSystem.getLocal(job); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java index d4e5e17e11f..4cfd59af745 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java @@ -18,29 +18,22 @@ package org.apache.hadoop.mapred; -import org.apache.commons.logging.Log; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; -import org.junit.Test; +import java.io.*; +import java.util.*; +import junit.framework.TestCase; -import java.util.BitSet; -import java.util.Random; +import org.apache.commons.logging.*; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; +import org.apache.hadoop.conf.*; -public class TestSequenceFileAsTextInputFormat { +public class TestSequenceFileAsTextInputFormat extends TestCase { private static final Log LOG = FileInputFormat.LOG; private static int MAX_LENGTH = 10000; private static Configuration conf = new Configuration(); - @Test public void testFormat() throws Exception { JobConf job = new JobConf(conf); FileSystem fs = FileSystem.getLocal(conf); @@ -119,4 +112,8 @@ public class TestSequenceFileAsTextInputFormat { } } + + public static void main(String[] args) throws Exception { + new TestSequenceFileAsTextInputFormat().testFormat(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java index 93f21ce9e49..e50c396a434 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java @@ -18,21 +18,17 @@ package org.apache.hadoop.mapred; -import org.apache.commons.logging.Log; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; -import org.junit.Test; +import java.io.*; +import java.util.*; +import junit.framework.TestCase; -import java.io.IOException; -import java.util.Random; +import org.apache.commons.logging.*; -import static org.junit.Assert.assertEquals; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; +import org.apache.hadoop.conf.*; -public class TestSequenceFileInputFilter { +public class TestSequenceFileInputFilter extends TestCase { private static final Log LOG = FileInputFormat.LOG; private static final int MAX_LENGTH = 15000; @@ -101,8 +97,7 @@ public class TestSequenceFileInputFilter { } return count; } - - @Test + public void testRegexFilter() throws Exception { // set the filter class LOG.info("Testing Regex Filter with patter: \\A10*"); @@ -126,7 +121,6 @@ public class TestSequenceFileInputFilter { fs.delete(inDir, true); } - @Test public void testPercentFilter() throws Exception { LOG.info("Testing Percent Filter with frequency: 1000"); // set the filter class @@ -153,8 +147,7 @@ public class TestSequenceFileInputFilter { // clean up fs.delete(inDir, true); } - - @Test + public void testMD5Filter() throws Exception { // set the filter class LOG.info("Testing MD5 Filter with frequency: 1000"); @@ -175,4 +168,9 @@ public class TestSequenceFileInputFilter { // clean up fs.delete(inDir, true); } + + public static void main(String[] args) throws Exception { + TestSequenceFileInputFilter filter = new TestSequenceFileInputFilter(); + filter.testRegexFilter(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java index 338e91d4d35..575ed532545 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java @@ -18,28 +18,22 @@ package org.apache.hadoop.mapred; -import org.apache.commons.logging.Log; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.SequenceFile; -import org.junit.Test; +import java.io.*; +import java.util.*; +import junit.framework.TestCase; -import java.util.BitSet; -import java.util.Random; +import org.apache.commons.logging.*; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; +import org.apache.hadoop.conf.*; -public class TestSequenceFileInputFormat { +public class TestSequenceFileInputFormat extends TestCase { private static final Log LOG = FileInputFormat.LOG; private static int MAX_LENGTH = 10000; private static Configuration conf = new Configuration(); - @Test public void testFormat() throws Exception { JobConf job = new JobConf(conf); FileSystem fs = FileSystem.getLocal(conf); @@ -116,4 +110,8 @@ public class TestSequenceFileInputFormat { } } + + public static void main(String[] args) throws Exception { + new TestSequenceFileInputFormat().testFormat(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java index 82d1d2d09a1..ad4d4ce17a9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSortedRanges.java @@ -17,20 +17,18 @@ */ package org.apache.hadoop.mapred; +import java.util.Iterator; + +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.mapred.SortedRanges.Range; -import org.junit.Test; -import java.util.Iterator; - -import static org.junit.Assert.assertEquals; - -public class TestSortedRanges { - private static final Log LOG = +public class TestSortedRanges extends TestCase { + private static final Log LOG = LogFactory.getLog(TestSortedRanges.class); - - @Test + public void testAdd() { SortedRanges sr = new SortedRanges(); sr.add(new Range(2,9)); @@ -68,8 +66,7 @@ public class TestSortedRanges { assertEquals(77, it.next().longValue()); } - - @Test + public void testRemove() { SortedRanges sr = new SortedRanges(); sr.add(new Range(2,19)); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java index b9e32759fa4..426686f9bb5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java @@ -18,6 +18,12 @@ package org.apache.hadoop.mapred; +import java.io.DataOutputStream; +import java.io.IOException; +import java.net.URI; + +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -28,20 +34,14 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.IdentityReducer; +import org.apache.hadoop.mapreduce.MRConfig; +import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; import org.apache.hadoop.util.Progressable; -import org.junit.Test; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.net.URI; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * A JUnit test to test that jobs' output filenames are not HTML-encoded (cf HADOOP-1795). */ -public class TestSpecialCharactersInOutputPath { +public class TestSpecialCharactersInOutputPath extends TestCase { private static final Log LOG = LogFactory.getLog(TestSpecialCharactersInOutputPath.class.getName()); @@ -96,8 +96,7 @@ public class TestSpecialCharactersInOutputPath { LOG.info("job is complete: " + runningJob.isSuccessful()); return (runningJob.isSuccessful()); } - - @Test + public void testJobWithDFS() throws IOException { String namenode = null; MiniDFSCluster dfs = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestStatisticsCollector.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestStatisticsCollector.java index 8a83e8153e3..12568d09175 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestStatisticsCollector.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestStatisticsCollector.java @@ -19,18 +19,14 @@ package org.apache.hadoop.mapred; import java.util.Map; +import junit.framework.TestCase; + import org.apache.hadoop.mapred.StatisticsCollector.TimeWindow; import org.apache.hadoop.mapred.StatisticsCollector.Stat; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; - -public class TestStatisticsCollector { +public class TestStatisticsCollector extends TestCase{ @SuppressWarnings("rawtypes") - @Test public void testMovingWindow() throws Exception { StatisticsCollector collector = new StatisticsCollector(1); TimeWindow window = new TimeWindow("test", 6, 2); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestUserDefinedCounters.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestUserDefinedCounters.java index 2d67edc581a..3c2cf215fb3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestUserDefinedCounters.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestUserDefinedCounters.java @@ -17,15 +17,6 @@ */ package org.apache.hadoop.mapred; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.lib.IdentityMapper; -import org.apache.hadoop.mapred.lib.IdentityReducer; -import org.junit.Test; - import java.io.BufferedReader; import java.io.File; import java.io.IOException; @@ -35,10 +26,18 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.Writer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import junit.framework.TestCase; -public class TestUserDefinedCounters { +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.lib.IdentityMapper; +import org.apache.hadoop.mapred.lib.IdentityReducer; + +public class TestUserDefinedCounters extends TestCase { + private static String TEST_ROOT_DIR = new File(System.getProperty("test.build.data", "/tmp")).toURI() .toString().replace(' ', '+') @@ -76,7 +75,6 @@ public class TestUserDefinedCounters { wr.close(); } - @Test public void testMapReduceJob() throws Exception { JobConf conf = new JobConf(TestUserDefinedCounters.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestWritableJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestWritableJobConf.java index 82c68db30c5..2c0cedcbb30 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestWritableJobConf.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestWritableJobConf.java @@ -18,6 +18,12 @@ package org.apache.hadoop.mapred; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; @@ -25,15 +31,8 @@ import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.util.GenericsUtil; -import org.junit.Test; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -import static org.junit.Assert.assertTrue; - -public class TestWritableJobConf { +public class TestWritableJobConf extends TestCase { private static final Configuration CONF = new Configuration(); @@ -79,17 +78,15 @@ public class TestWritableJobConf { } } - assertTrue(map1.equals(map2)); + assertEquals(map1, map2); } - @Test public void testEmptyConfiguration() throws Exception { JobConf conf = new JobConf(); Configuration deser = serDeser(conf); assertEquals(conf, deser); } - @Test public void testNonEmptyConfiguration() throws Exception { JobConf conf = new JobConf(); conf.set("a", "A"); @@ -98,7 +95,6 @@ public class TestWritableJobConf { assertEquals(conf, deser); } - @Test public void testConfigurationWithDefaults() throws Exception { JobConf conf = new JobConf(false); conf.set("a", "A"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java index abf2e72e0d1..0e340428214 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java @@ -18,10 +18,6 @@ package org.apache.hadoop.mapred; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -42,6 +38,8 @@ import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Map; +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -115,7 +113,7 @@ import org.mockito.stubbing.Answer; * Test YarnRunner and make sure the client side plugin works * fine */ -public class TestYARNRunner { +public class TestYARNRunner extends TestCase { private static final Log LOG = LogFactory.getLog(TestYARNRunner.class); private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java index a3066765ec0..15cea69dab2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java @@ -22,6 +22,11 @@ import java.io.DataOutput; import java.io.IOException; import java.util.Iterator; +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; +import junit.extensions.TestSetup; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -49,27 +54,23 @@ import org.apache.hadoop.mapred.Utils; import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.IdentityReducer; import org.apache.hadoop.util.ReflectionUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; -public class TestDatamerge { +public class TestDatamerge extends TestCase { private static MiniDFSCluster cluster = null; - - @Before - public void setUp() throws Exception { - Configuration conf = new Configuration(); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); - } - @After - public void tearDown() throws Exception { - if (cluster != null) { - cluster.shutdown(); - } + public static Test suite() { + TestSetup setup = new TestSetup(new TestSuite(TestDatamerge.class)) { + protected void setUp() throws Exception { + Configuration conf = new Configuration(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + } + protected void tearDown() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + }; + return setup; } private static SequenceFile.Writer[] createWriters(Path testdir, @@ -245,22 +246,18 @@ public class TestDatamerge { base.getFileSystem(job).delete(base, true); } - @Test public void testSimpleInnerJoin() throws Exception { joinAs("inner", InnerJoinChecker.class); } - @Test public void testSimpleOuterJoin() throws Exception { joinAs("outer", OuterJoinChecker.class); } - @Test public void testSimpleOverride() throws Exception { joinAs("override", OverrideChecker.class); } - @Test public void testNestedJoin() throws Exception { // outer(inner(S1,...,Sn),outer(S1,...Sn)) final int SOURCES = 3; @@ -353,7 +350,6 @@ public class TestDatamerge { } - @Test public void testEmptyJoin() throws Exception { JobConf job = new JobConf(); Path base = cluster.getFileSystem().makeQualified(new Path("/empty")); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestTupleWritable.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestTupleWritable.java index 56871550dc9..e421ede9827 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestTupleWritable.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestTupleWritable.java @@ -26,6 +26,8 @@ import java.io.IOException; import java.util.Arrays; import java.util.Random; +import junit.framework.TestCase; + import org.apache.hadoop.io.BooleanWritable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.FloatWritable; @@ -34,12 +36,8 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; -import org.junit.Test; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -public class TestTupleWritable { +public class TestTupleWritable extends TestCase { private TupleWritable makeTuple(Writable[] writs) { Writable[] sub1 = { writs[1], writs[2] }; @@ -102,7 +100,6 @@ public class TestTupleWritable { return i; } - @Test public void testIterable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -124,7 +121,6 @@ public class TestTupleWritable { verifIter(writs, t, 0); } - @Test public void testNestedIterable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -143,7 +139,6 @@ public class TestTupleWritable { assertTrue("Bad count", writs.length == verifIter(writs, sTuple, 0)); } - @Test public void testWritable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -167,7 +162,6 @@ public class TestTupleWritable { assertTrue("Failed to write/read tuple", sTuple.equals(dTuple)); } - @Test public void testWideWritable() throws Exception { Writable[] manyWrits = makeRandomWritables(131); @@ -186,8 +180,7 @@ public class TestTupleWritable { assertTrue("Failed to write/read tuple", sTuple.equals(dTuple)); assertEquals("All tuple data has not been read from the stream",-1,in.read()); } - - @Test + public void testWideWritable2() throws Exception { Writable[] manyWrits = makeRandomWritables(71); @@ -209,7 +202,6 @@ public class TestTupleWritable { * Tests a tuple writable with more than 64 values and the values set written * spread far apart. */ - @Test public void testSparseWideWritable() throws Exception { Writable[] manyWrits = makeRandomWritables(131); @@ -228,7 +220,7 @@ public class TestTupleWritable { assertTrue("Failed to write/read tuple", sTuple.equals(dTuple)); assertEquals("All tuple data has not been read from the stream",-1,in.read()); } - @Test + public void testWideTuple() throws Exception { Text emptyText = new Text("Should be empty"); Writable[] values = new Writable[64]; @@ -248,7 +240,7 @@ public class TestTupleWritable { } } } - @Test + public void testWideTuple2() throws Exception { Text emptyText = new Text("Should be empty"); Writable[] values = new Writable[64]; @@ -272,7 +264,6 @@ public class TestTupleWritable { /** * Tests that we can write more than 64 values. */ - @Test public void testWideTupleBoundary() throws Exception { Text emptyText = new Text("Should not be set written"); Writable[] values = new Writable[65]; @@ -296,7 +287,6 @@ public class TestTupleWritable { /** * Tests compatibility with pre-0.21 versions of TupleWritable */ - @Test public void testPreVersion21Compatibility() throws Exception { Writable[] manyWrits = makeRandomWritables(64); PreVersion21TupleWritable oldTuple = new PreVersion21TupleWritable(manyWrits); @@ -314,7 +304,7 @@ public class TestTupleWritable { assertTrue("Tuple writable is unable to read pre-0.21 versions of TupleWritable", oldTuple.isCompatible(dTuple)); assertEquals("All tuple data has not been read from the stream",-1,in.read()); } - @Test + public void testPreVersion21CompatibilityEmptyTuple() throws Exception { Writable[] manyWrits = new Writable[0]; PreVersion21TupleWritable oldTuple = new PreVersion21TupleWritable(manyWrits); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java index ae5572f5dcd..3ca175a5049 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java @@ -21,6 +21,8 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import junit.framework.TestCase; + import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; @@ -33,16 +35,13 @@ import org.apache.hadoop.mapred.JobConfigurable; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.util.ReflectionUtils; -import org.junit.Test; -import static org.junit.Assert.assertTrue; -public class TestWrappedRecordReaderClassloader { +public class TestWrappedRecordReaderClassloader extends TestCase { /** * Tests the class loader set by {@link JobConf#setClassLoader(ClassLoader)} * is inherited by any {@link WrappedRecordReader}s created by * {@link CompositeRecordReader} */ - @Test public void testClassLoader() throws Exception { JobConf job = new JobConf(); Fake_ClassLoader classLoader = new Fake_ClassLoader(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java index b916026272e..8bd855433ea 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java @@ -20,6 +20,8 @@ package org.apache.hadoop.mapred.lib; import java.io.DataOutputStream; import java.io.IOException; +import junit.framework.TestCase; + import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -30,12 +32,9 @@ import org.apache.hadoop.mapred.Mapper; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.TextInputFormat; -import org.junit.Test; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; -public class TestDelegatingInputFormat { - @Test +public class TestDelegatingInputFormat extends TestCase { + public void testSplitting() throws Exception { JobConf conf = new JobConf(); MiniDFSCluster dfs = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestLineInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestLineInputFormat.java index 388de0fb88d..db9c219e9c1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestLineInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestLineInputFormat.java @@ -20,14 +20,13 @@ package org.apache.hadoop.mapred.lib; import java.io.*; import java.util.*; +import junit.framework.TestCase; import org.apache.hadoop.fs.*; import org.apache.hadoop.io.*; import org.apache.hadoop.mapred.*; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -public class TestLineInputFormat { +public class TestLineInputFormat extends TestCase { private static int MAX_LENGTH = 200; private static JobConf defaultConf = new JobConf(); @@ -44,7 +43,7 @@ public class TestLineInputFormat { private static Path workDir = new Path(new Path(System.getProperty("test.build.data", "."), "data"), "TestLineInputFormat"); - @Test + public void testFormat() throws Exception { JobConf job = new JobConf(); Path file = new Path(workDir, "test.txt"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java index 115a6f70d08..3a9cb9ec337 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java @@ -36,6 +36,7 @@ import static org.junit.Assert.assertEquals; * @see TestDelegatingInputFormat */ public class TestMultipleInputs { + @Test public void testAddInputPathWithFormat() { final JobConf conf = new JobConf(); @@ -48,6 +49,7 @@ public class TestMultipleInputs { assertEquals(KeyValueTextInputFormat.class, inputs.get(new Path("/bar")) .getClass()); } + @Test public void testAddInputPathWithMapper() { final JobConf conf = new JobConf(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java index f33f83cb6c1..6da96ce22bd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java @@ -22,14 +22,13 @@ import org.apache.hadoop.io.*; import org.apache.hadoop.mapred.*; import org.apache.hadoop.mapred.lib.*; import org.apache.hadoop.mapreduce.MapReduceTestUtil; -import org.junit.Test; -import static org.junit.Assert.assertEquals; +import junit.framework.TestCase; import java.io.*; import java.util.*; import java.text.NumberFormat; -public class TestAggregates { +public class TestAggregates extends TestCase { private static NumberFormat idFormat = NumberFormat.getInstance(); static { @@ -37,7 +36,7 @@ public class TestAggregates { idFormat.setGroupingUsed(false); } - @Test + public void testAggregates() throws Exception { launch(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/db/TestConstructQuery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/db/TestConstructQuery.java index 203da4e0b7c..968bb066565 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/db/TestConstructQuery.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/db/TestConstructQuery.java @@ -19,13 +19,13 @@ package org.apache.hadoop.mapred.lib.db; import java.io.IOException; +import junit.framework.TestCase; + import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapred.JobConf; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -public class TestConstructQuery { +public class TestConstructQuery extends TestCase { + private String[] fieldNames = new String[] { "id", "name", "value" }; private String[] nullFieldNames = new String[] { null, null, null }; private String expected = "INSERT INTO hadoop_output (id,name,value) VALUES (?,?,?);"; @@ -33,15 +33,15 @@ public class TestConstructQuery { private DBOutputFormat format = new DBOutputFormat(); - @Test - public void testConstructQuery() { + + public void testConstructQuery() { String actual = format.constructQuery("hadoop_output", fieldNames); assertEquals(expected, actual); - + actual = format.constructQuery("hadoop_output", nullFieldNames); assertEquals(nullExpected, actual); } - @Test + public void testSetOutput() throws IOException { JobConf job = new JobConf(); DBOutputFormat.setOutput(job, "hadoop_output", fieldNames); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java index 34b1d75dfed..dd7817d65b5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java @@ -44,13 +44,10 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.ToolRunner; import org.junit.Ignore; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; +import junit.framework.TestCase; @Ignore -public class TestPipes { +public class TestPipes extends TestCase { private static final Log LOG = LogFactory.getLog(TestPipes.class.getName()); @@ -69,7 +66,7 @@ public class TestPipes { fs.delete(p, true); assertFalse("output not cleaned up", fs.exists(p)); } - @Test + public void testPipes() throws IOException { if (System.getProperty("compile.c++") == null) { LOG.info("compile.c++ is not defined, so skipping TestPipes"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java index 8177ecd405b..29640c8854b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java @@ -17,42 +17,36 @@ */ package org.apache.hadoop.mapreduce; +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.fs.*; import org.apache.hadoop.mapred.LocalJobRunner; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.FileSplit; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.ReflectionUtils; + import org.junit.Test; - -import java.io.BufferedReader; -import java.io.BufferedWriter; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.util.ArrayList; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import junit.framework.TestCase; /** * Stress tests for the LocalJobRunner */ -public class TestLocalRunner { +public class TestLocalRunner extends TestCase { private static final Log LOG = LogFactory.getLog(TestLocalRunner.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java index 8fe9078e9e0..6f45b5f5dc2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java @@ -17,23 +17,6 @@ */ package org.apache.hadoop.mapreduce; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.mapred.ClusterMapReduceTestCase; -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import org.apache.hadoop.mapreduce.tools.CLI; -import org.apache.hadoop.util.ExitUtil; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; -import org.junit.Test; - import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -48,11 +31,23 @@ import java.io.PipedOutputStream; import java.io.PrintStream; import java.util.Arrays; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.RemoteIterator; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.junit.Assert; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.ClusterMapReduceTestCase; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hadoop.mapreduce.tools.CLI; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; /** test CLI class. CLI class implemented the Tool interface. @@ -108,7 +103,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { throw new IOException(); } } - @Test + public void testJobSubmissionSpecsAndFiles() throws Exception { Configuration conf = createJobConf(); Job job = MapReduceTestUtil.createJob(conf, getInputDir(), getOutputDir(), @@ -132,7 +127,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { /** * main test method */ - @Test + public void testJobClient() throws Exception { Configuration conf = createJobConf(); Job job = runJob(conf); @@ -185,7 +180,8 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { runTool(conf, jc, new String[] { "-fail-task", taid.toString() }, out); String answer = new String(out.toByteArray(), "UTF-8"); - assertTrue(answer.contains("Killed task " + taid + " by failing it")); + Assert + .assertTrue(answer.contains("Killed task " + taid + " by failing it")); } /** @@ -203,7 +199,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { runTool(conf, jc, new String[] { "-kill-task", taid.toString() }, out); String answer = new String(out.toByteArray(), "UTF-8"); - assertTrue(answer.contains("Killed task " + taid)); + Assert.assertTrue(answer.contains("Killed task " + taid)); } /** @@ -690,7 +686,6 @@ public class TestMRJobClient extends ClusterMapReduceTestCase { * Test -list option displays job name. * The name is capped to 20 characters for display. */ - @Test public void testJobName() throws Exception { Configuration conf = createJobConf(); CLI jc = createJobClient(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java index a69e06eacd9..1e4f4de9f93 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java @@ -25,6 +25,8 @@ import java.io.Writer; import java.util.Arrays; import java.util.List; +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; @@ -40,16 +42,13 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import org.junit.Test; - -import static org.junit.Assert.assertTrue; /** * A JUnit test to test the Map-Reduce framework's feature to create part * files only if there is an explicit output.collect. This helps in preventing * 0 byte files */ -public class TestMapReduceLazyOutput { +public class TestMapReduceLazyOutput extends TestCase { private static final int NUM_HADOOP_SLAVES = 3; private static final int NUM_MAPS_PER_NODE = 2; private static final Path INPUT = new Path("/testlazy/input"); @@ -123,7 +122,7 @@ public class TestMapReduceLazyOutput { } } - @Test + public void testLazyOutput() throws Exception { MiniDFSCluster dfs = null; MiniMRCluster mr = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java index b757fb2c34f..5cf08991869 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java @@ -27,6 +27,8 @@ import java.io.Writer; import java.util.ArrayList; import java.util.StringTokenizer; +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -41,15 +43,12 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import org.junit.Test; - -import static org.junit.Assert.assertTrue; /** * A JUnit test to test the Map-Reduce framework's support for the * "mark-reset" functionality in Reduce Values Iterator */ -public class TestValueIterReset { +public class TestValueIterReset extends TestCase { private static final int NUM_MAPS = 1; private static final int NUM_TESTS = 4; private static final int NUM_VALUES = 40; @@ -519,7 +518,6 @@ public class TestValueIterReset { } } - @Test public void testValueIterReset() { try { Configuration conf = new Configuration(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java index 308b7775a67..4d84fa9e108 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java @@ -18,7 +18,6 @@ package org.apache.hadoop.mapreduce; -import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -27,6 +26,7 @@ import static org.mockito.Mockito.doNothing; import java.io.IOException; import java.nio.ByteBuffer; +import junit.framework.TestCase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -44,7 +44,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.junit.Test; -public class TestYarnClientProtocolProvider { +public class TestYarnClientProtocolProvider extends TestCase { + private static final RecordFactory recordFactory = RecordFactoryProvider. getRecordFactory(null); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java index 789ed98193e..f24dffe2655 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java @@ -18,24 +18,22 @@ package org.apache.hadoop.mapreduce.lib.aggregate; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.*; +import org.apache.hadoop.mapred.Utils; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import org.junit.Test; +import junit.framework.TestCase; +import java.io.*; import java.text.NumberFormat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class TestMapReduceAggregates { +public class TestMapReduceAggregates extends TestCase { private static NumberFormat idFormat = NumberFormat.getInstance(); static { @@ -43,7 +41,7 @@ public class TestMapReduceAggregates { idFormat.setGroupingUsed(false); } - @Test + public void testAggregates() throws Exception { launch(); } @@ -124,4 +122,11 @@ public class TestMapReduceAggregates { fs.delete(OUTPUT_DIR, true); fs.delete(INPUT_DIR, true); } + + /** + * Launches all the tasks in order. + */ + public static void main(String[] argv) throws Exception { + launch(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java index 014855f7d6a..bff25d20038 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java @@ -19,15 +19,14 @@ package org.apache.hadoop.mapreduce.lib.db; import java.io.IOException; +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.Job; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; - -public class TestDBOutputFormat { +public class TestDBOutputFormat extends TestCase { + private String[] fieldNames = new String[] { "id", "name", "value" }; private String[] nullFieldNames = new String[] { null, null, null }; private String expected = "INSERT INTO hadoop_output " + @@ -36,17 +35,15 @@ public class TestDBOutputFormat { private DBOutputFormat format = new DBOutputFormat(); - - @Test - public void testConstructQuery() { + + public void testConstructQuery() { String actual = format.constructQuery("hadoop_output", fieldNames); assertEquals(expected, actual); actual = format.constructQuery("hadoop_output", nullFieldNames); assertEquals(nullExpected, actual); } - - @Test + public void testSetOutput() throws IOException { Job job = Job.getInstance(new Configuration()); DBOutputFormat.setOutput(job, "hadoop_output", fieldNames); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java index 8b5d907dcdc..e50aba4f462 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java @@ -17,15 +17,15 @@ */ package org.apache.hadoop.mapreduce.lib.db; -import org.junit.Test; - +import java.io.IOException; +import java.math.BigDecimal; import java.sql.SQLException; +import java.util.ArrayList; import java.util.List; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import junit.framework.TestCase; -public class TestIntegerSplitter { +public class TestIntegerSplitter extends TestCase { private long [] toLongArray(List in) { long [] out = new long[in.size()]; for (int i = 0; i < in.size(); i++) { @@ -70,14 +70,12 @@ public class TestIntegerSplitter { } } - @Test public void testEvenSplits() throws SQLException { List splits = new IntegerSplitter().split(10, 0, 100); long [] expected = { 0, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100 }; assertLongArrayEquals(expected, toLongArray(splits)); } - @Test public void testOddSplits() throws SQLException { List splits = new IntegerSplitter().split(10, 0, 95); long [] expected = { 0, 9, 18, 27, 36, 45, 54, 63, 72, 81, 90, 95 }; @@ -85,14 +83,12 @@ public class TestIntegerSplitter { } - @Test public void testSingletonSplit() throws SQLException { List splits = new IntegerSplitter().split(1, 5, 5); long [] expected = { 5, 5 }; assertLongArrayEquals(expected, toLongArray(splits)); } - @Test public void testSingletonSplit2() throws SQLException { // Same test, but overly-high numSplits List splits = new IntegerSplitter().split(5, 5, 5); @@ -100,7 +96,6 @@ public class TestIntegerSplitter { assertLongArrayEquals(expected, toLongArray(splits)); } - @Test public void testTooManySplits() throws SQLException { List splits = new IntegerSplitter().split(5, 3, 5); long [] expected = { 3, 4, 5 }; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java index e16f4234877..045e3a1b1f6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java @@ -17,16 +17,15 @@ */ package org.apache.hadoop.mapreduce.lib.db; -import org.junit.Test; - +import java.io.IOException; import java.math.BigDecimal; import java.sql.SQLException; +import java.util.ArrayList; import java.util.List; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import junit.framework.TestCase; -public class TestTextSplitter { +public class TestTextSplitter extends TestCase { public String formatArray(Object [] ar) { StringBuilder sb = new StringBuilder(); @@ -63,56 +62,48 @@ public class TestTextSplitter { } } - @Test public void testStringConvertEmpty() { TextSplitter splitter = new TextSplitter(); BigDecimal emptyBigDec = splitter.stringToBigDecimal(""); assertEquals(BigDecimal.ZERO, emptyBigDec); } - @Test public void testBigDecConvertEmpty() { TextSplitter splitter = new TextSplitter(); String emptyStr = splitter.bigDecimalToString(BigDecimal.ZERO); assertEquals("", emptyStr); } - @Test public void testConvertA() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("A")); assertEquals("A", out); } - @Test public void testConvertZ() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("Z")); assertEquals("Z", out); } - @Test public void testConvertThreeChars() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("abc")); assertEquals("abc", out); } - @Test public void testConvertStr() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("big str")); assertEquals("big str", out); } - @Test public void testConvertChomped() { TextSplitter splitter = new TextSplitter(); String out = splitter.bigDecimalToString(splitter.stringToBigDecimal("AVeryLongStringIndeed")); assertEquals("AVeryLon", out); } - @Test public void testAlphabetSplit() throws SQLException { // This should give us 25 splits, one per letter. TextSplitter splitter = new TextSplitter(); @@ -122,7 +113,6 @@ public class TestTextSplitter { assertArrayEquals(expected, splits.toArray(new String [0])); } - @Test public void testCommonPrefix() throws SQLException { // Splits between 'Hand' and 'Hardy' TextSplitter splitter = new TextSplitter(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java index 6f9183ab21b..91070f89c42 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java @@ -18,19 +18,15 @@ package org.apache.hadoop.mapreduce.lib.fieldsel; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.MapReduceTestUtil; -import org.junit.Test; +import junit.framework.TestCase; import java.text.NumberFormat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class TestMRFieldSelection { +public class TestMRFieldSelection extends TestCase { private static NumberFormat idFormat = NumberFormat.getInstance(); static { @@ -38,7 +34,6 @@ private static NumberFormat idFormat = NumberFormat.getInstance(); idFormat.setGroupingUsed(false); } - @Test public void testFieldSelection() throws Exception { launch(); } @@ -119,4 +114,11 @@ private static NumberFormat idFormat = NumberFormat.getInstance(); System.out.println("ExpectedData:"); System.out.println(expectedOutput.toString()); } + + /** + * Launches all the tasks in order. + */ + public static void main(String[] argv) throws Exception { + launch(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java index cbf9d183ef2..f0b3d57486c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java @@ -18,12 +18,11 @@ package org.apache.hadoop.mapreduce.lib.input; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.DataInputBuffer; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; +import java.io.IOException; +import java.util.Random; + +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; @@ -32,18 +31,12 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.junit.Test; -import java.io.IOException; -import java.util.Random; +import junit.framework.TestCase; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class TestMRSequenceFileAsBinaryInputFormat { +public class TestMRSequenceFileAsBinaryInputFormat extends TestCase { private static final int RECORDS = 10000; - @Test public void testBinary() throws IOException, InterruptedException { Job job = Job.getInstance(); FileSystem fs = FileSystem.getLocal(job.getConfiguration()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java index 335ce050d82..2d03c2dd96a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java @@ -18,13 +18,11 @@ package org.apache.hadoop.mapreduce.lib.input; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; +import java.util.*; +import junit.framework.TestCase; + +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; @@ -33,19 +31,12 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.junit.Test; +import org.apache.hadoop.conf.*; -import java.util.BitSet; -import java.util.Random; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; - -public class TestMRSequenceFileAsTextInputFormat { +public class TestMRSequenceFileAsTextInputFormat extends TestCase { private static int MAX_LENGTH = 10000; private static Configuration conf = new Configuration(); - @Test public void testFormat() throws Exception { Job job = Job.getInstance(conf); FileSystem fs = FileSystem.getLocal(conf); @@ -121,4 +112,8 @@ public class TestMRSequenceFileAsTextInputFormat { } } + + public static void main(String[] args) throws Exception { + new TestMRSequenceFileAsTextInputFormat().testFormat(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java index 89aa7b23057..edf7e1ad10d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java @@ -18,14 +18,14 @@ package org.apache.hadoop.mapreduce.lib.input; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; +import java.io.*; +import java.util.*; +import junit.framework.TestCase; + +import org.apache.commons.logging.*; + +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; @@ -34,15 +34,10 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.junit.Test; +import org.apache.hadoop.conf.*; -import java.io.IOException; -import java.util.Random; - -import static org.junit.Assert.assertEquals; - -public class TestMRSequenceFileInputFilter { - private static final Log LOG = +public class TestMRSequenceFileInputFilter extends TestCase { + private static final Log LOG = LogFactory.getLog(TestMRSequenceFileInputFilter.class.getName()); private static final int MAX_LENGTH = 15000; @@ -118,8 +113,7 @@ public class TestMRSequenceFileInputFilter { } return count; } - - @Test + public void testRegexFilter() throws Exception { // set the filter class LOG.info("Testing Regex Filter with patter: \\A10*"); @@ -144,7 +138,6 @@ public class TestMRSequenceFileInputFilter { fs.delete(inDir, true); } - @Test public void testPercentFilter() throws Exception { LOG.info("Testing Percent Filter with frequency: 1000"); // set the filter class @@ -172,8 +165,7 @@ public class TestMRSequenceFileInputFilter { // clean up fs.delete(inDir, true); } - - @Test + public void testMD5Filter() throws Exception { // set the filter class LOG.info("Testing MD5 Filter with frequency: 1000"); @@ -195,4 +187,9 @@ public class TestMRSequenceFileInputFilter { // clean up fs.delete(inDir, true); } + + public static void main(String[] args) throws Exception { + TestMRSequenceFileInputFilter filter = new TestMRSequenceFileInputFilter(); + filter.testRegexFilter(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java index 477866f4e35..7b3878d9475 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java @@ -18,28 +18,17 @@ package org.apache.hadoop.mapreduce.lib.input; +import java.io.*; +import java.util.*; +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.MapContext; -import org.apache.hadoop.mapreduce.MapReduceTestUtil; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; +import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.junit.Test; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.io.Writer; -import java.util.List; - -import static org.junit.Assert.assertEquals; - -public class TestNLineInputFormat { +public class TestNLineInputFormat extends TestCase { private static int MAX_LENGTH = 200; private static Configuration conf = new Configuration(); @@ -56,8 +45,7 @@ public class TestNLineInputFormat { private static Path workDir = new Path(new Path(System.getProperty("test.build.data", "."), "data"), "TestNLineInputFormat"); - - @Test + public void testFormat() throws Exception { Job job = Job.getInstance(conf); Path file = new Path(workDir, "test.txt"); @@ -128,4 +116,8 @@ public class TestNLineInputFormat { } } } + + public static void main(String[] args) throws Exception { + new TestNLineInputFormat().testFormat(); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java index 1173ea4fa47..d245bfd6cde 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java @@ -19,6 +19,11 @@ package org.apache.hadoop.mapreduce.lib.join; import java.io.IOException; +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; +import junit.extensions.TestSetup; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -32,31 +37,23 @@ import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class TestJoinDatamerge { +public class TestJoinDatamerge extends TestCase { private static MiniDFSCluster cluster = null; - - @BeforeClass - public static void setUp() throws Exception { - Configuration conf = new Configuration(); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); - } - - @AfterClass - public static void tearDown() throws Exception { - if (cluster != null) { - cluster.shutdown(); - } + public static Test suite() { + TestSetup setup = new TestSetup(new TestSuite(TestJoinDatamerge.class)) { + protected void setUp() throws Exception { + Configuration conf = new Configuration(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + } + protected void tearDown() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + }; + return setup; } private static SequenceFile.Writer[] createWriters(Path testdir, @@ -114,7 +111,7 @@ public class TestJoinDatamerge { extends Mapper{ protected final static IntWritable one = new IntWritable(1); int srcs; - + public void setup(Context context) { srcs = context.getConfiguration().getInt("testdatamerge.sources", 0); assertTrue("Invalid src count: " + srcs, srcs > 0); @@ -126,7 +123,7 @@ public class TestJoinDatamerge { protected final static IntWritable one = new IntWritable(1); int srcs; - + public void setup(Context context) { srcs = context.getConfiguration().getInt("testdatamerge.sources", 0); assertTrue("Invalid src count: " + srcs, srcs > 0); @@ -275,12 +272,10 @@ public class TestJoinDatamerge { base.getFileSystem(conf).delete(base, true); } - @Test public void testSimpleInnerJoin() throws Exception { joinAs("inner", InnerJoinMapChecker.class, InnerJoinReduceChecker.class); } - @Test public void testSimpleOuterJoin() throws Exception { joinAs("outer", OuterJoinMapChecker.class, OuterJoinReduceChecker.class); } @@ -327,13 +322,11 @@ public class TestJoinDatamerge { } return product; } - - @Test + public void testSimpleOverride() throws Exception { joinAs("override", OverrideMapChecker.class, OverrideReduceChecker.class); } - @Test public void testNestedJoin() throws Exception { // outer(inner(S1,...,Sn),outer(S1,...Sn)) final int SOURCES = 3; @@ -429,7 +422,6 @@ public class TestJoinDatamerge { } - @Test public void testEmptyJoin() throws Exception { Configuration conf = new Configuration(); Path base = cluster.getFileSystem().makeQualified(new Path("/empty")); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java index b6e76069d95..151bc875ad3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java @@ -20,6 +20,11 @@ package org.apache.hadoop.mapreduce.lib.join; import java.io.IOException; import java.util.List; +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; +import junit.extensions.TestSetup; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -31,14 +36,8 @@ import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class TestJoinProperties { +public class TestJoinProperties extends TestCase { private static MiniDFSCluster cluster = null; final static int SOURCES = 3; @@ -47,19 +46,21 @@ public class TestJoinProperties { static Path[] src; static Path base; - @BeforeClass - public static void setUp() throws Exception { - Configuration conf = new Configuration(); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); - base = cluster.getFileSystem().makeQualified(new Path("/nested")); - src = generateSources(conf); - } - - @AfterClass - public static void tearDown() throws Exception { - if (cluster != null) { - cluster.shutdown(); - } + public static Test suite() { + TestSetup setup = new TestSetup(new TestSuite(TestJoinProperties.class)) { + protected void setUp() throws Exception { + Configuration conf = new Configuration(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + base = cluster.getFileSystem().makeQualified(new Path("/nested")); + src = generateSources(conf); + } + protected void tearDown() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + }; + return setup; } // Sources from 0 to srcs-2 have IntWritable key and IntWritable value @@ -232,7 +233,6 @@ public class TestJoinProperties { } // outer(outer(A, B), C) == outer(A,outer(B, C)) == outer(A, B, C) - @Test public void testOuterAssociativity() throws Exception { Configuration conf = new Configuration(); testExpr1(conf, "outer", TestType.OUTER_ASSOCIATIVITY, 33); @@ -241,7 +241,6 @@ public class TestJoinProperties { } // inner(inner(A, B), C) == inner(A,inner(B, C)) == inner(A, B, C) - @Test public void testInnerAssociativity() throws Exception { Configuration conf = new Configuration(); testExpr1(conf, "inner", TestType.INNER_ASSOCIATIVITY, 2); @@ -250,7 +249,6 @@ public class TestJoinProperties { } // override(inner(A, B), A) == A - @Test public void testIdentity() throws Exception { Configuration conf = new Configuration(); testExpr4(conf); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java index 093da266b95..d35941fc884 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java @@ -24,6 +24,8 @@ import java.io.DataOutputStream; import java.util.Arrays; import java.util.Random; +import junit.framework.TestCase; + import org.apache.hadoop.io.BooleanWritable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.FloatWritable; @@ -31,13 +33,8 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class TestJoinTupleWritable { +public class TestJoinTupleWritable extends TestCase { private TupleWritable makeTuple(Writable[] writs) { Writable[] sub1 = { writs[1], writs[2] }; @@ -100,7 +97,6 @@ public class TestJoinTupleWritable { return i; } - @Test public void testIterable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -122,7 +118,6 @@ public class TestJoinTupleWritable { verifIter(writs, t, 0); } - @Test public void testNestedIterable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -141,7 +136,6 @@ public class TestJoinTupleWritable { assertTrue("Bad count", writs.length == verifIter(writs, sTuple, 0)); } - @Test public void testWritable() throws Exception { Random r = new Random(); Writable[] writs = { @@ -165,7 +159,6 @@ public class TestJoinTupleWritable { assertTrue("Failed to write/read tuple", sTuple.equals(dTuple)); } - @Test public void testWideWritable() throws Exception { Writable[] manyWrits = makeRandomWritables(131); @@ -185,8 +178,7 @@ public class TestJoinTupleWritable { assertEquals("All tuple data has not been read from the stream", -1, in.read()); } - - @Test + public void testWideWritable2() throws Exception { Writable[] manyWrits = makeRandomWritables(71); @@ -209,7 +201,6 @@ public class TestJoinTupleWritable { * Tests a tuple writable with more than 64 values and the values set written * spread far apart. */ - @Test public void testSparseWideWritable() throws Exception { Writable[] manyWrits = makeRandomWritables(131); @@ -229,8 +220,7 @@ public class TestJoinTupleWritable { assertEquals("All tuple data has not been read from the stream", -1, in.read()); } - - @Test + public void testWideTuple() throws Exception { Text emptyText = new Text("Should be empty"); Writable[] values = new Writable[64]; @@ -251,8 +241,7 @@ public class TestJoinTupleWritable { } } } - - @Test + public void testWideTuple2() throws Exception { Text emptyText = new Text("Should be empty"); Writable[] values = new Writable[64]; @@ -277,7 +266,6 @@ public class TestJoinTupleWritable { /** * Tests that we can write more than 64 values. */ - @Test public void testWideTupleBoundary() throws Exception { Text emptyText = new Text("Should not be set written"); Writable[] values = new Writable[65]; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java index 680e246b4e3..36cf1872ad4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java @@ -17,32 +17,23 @@ */ package org.apache.hadoop.mapreduce.lib.join; +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.MapReduceTestUtil; +import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.MapReduceTestUtil.Fake_RR; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; -import org.junit.Test; -import static org.junit.Assert.assertTrue; - -public class TestWrappedRRClassloader { +public class TestWrappedRRClassloader extends TestCase { /** * Tests the class loader set by * {@link Configuration#setClassLoader(ClassLoader)} * is inherited by any {@link WrappedRecordReader}s created by * {@link CompositeRecordReader} */ - @Test public void testClassLoader() throws Exception { Configuration conf = new Configuration(); Fake_ClassLoader classLoader = new Fake_ClassLoader(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java index 5a8aeda83be..2e40f72fdd2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java @@ -18,17 +18,12 @@ package org.apache.hadoop.mapreduce.lib.output; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import java.io.IOException; +import java.util.Random; + import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BooleanWritable; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.io.DoubleWritable; -import org.apache.hadoop.io.FloatWritable; -import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.io.*; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.mapred.InvalidJobConfException; import org.apache.hadoop.mapreduce.InputFormat; @@ -43,22 +38,16 @@ import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.task.MapContextImpl; -import org.junit.Test; -import java.io.IOException; -import java.util.Random; +import junit.framework.TestCase; +import org.apache.commons.logging.*; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class TestMRSequenceFileAsBinaryOutputFormat { +public class TestMRSequenceFileAsBinaryOutputFormat extends TestCase { private static final Log LOG = LogFactory.getLog(TestMRSequenceFileAsBinaryOutputFormat.class.getName()); private static final int RECORDS = 10000; - - @Test + public void testBinary() throws IOException, InterruptedException { Configuration conf = new Configuration(); Job job = Job.getInstance(conf); @@ -155,8 +144,7 @@ public class TestMRSequenceFileAsBinaryOutputFormat { assertEquals("Some records not found", RECORDS, count); } - @Test - public void testSequenceOutputClassDefaultsToMapRedOutputClass() + public void testSequenceOutputClassDefaultsToMapRedOutputClass() throws IOException { Job job = Job.getInstance(); // Setting Random class to test getSequenceFileOutput{Key,Value}Class @@ -184,8 +172,7 @@ public class TestMRSequenceFileAsBinaryOutputFormat { SequenceFileAsBinaryOutputFormat.getSequenceFileOutputValueClass(job)); } - @Test - public void testcheckOutputSpecsForbidRecordCompression() + public void testcheckOutputSpecsForbidRecordCompression() throws IOException { Job job = Job.getInstance(); FileSystem fs = FileSystem.getLocal(job.getConfiguration()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java index f83bc11a216..7be538ecf41 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java @@ -22,14 +22,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BinaryComparable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.util.ReflectionUtils; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import junit.framework.TestCase; -public class TestBinaryPartitioner { +public class TestBinaryPartitioner extends TestCase { - @Test public void testDefaultOffsets() { Configuration conf = new Configuration(); BinaryPartitioner partitioner = @@ -53,8 +50,7 @@ public class TestBinaryPartitioner { partition2 = partitioner.getPartition(key2, null, 10); assertTrue(partition1 != partition2); } - - @Test + public void testCustomOffsets() { Configuration conf = new Configuration(); BinaryComparable key1 = new BytesWritable(new byte[] { 1, 2, 3, 4, 5 }); @@ -79,8 +75,7 @@ public class TestBinaryPartitioner { partition2 = partitioner.getPartition(key2, null, 10); assertEquals(partition1, partition2); } - - @Test + public void testLowerBound() { Configuration conf = new Configuration(); BinaryPartitioner.setLeftOffset(conf, 0); @@ -92,8 +87,7 @@ public class TestBinaryPartitioner { int partition2 = partitioner.getPartition(key2, null, 10); assertTrue(partition1 != partition2); } - - @Test + public void testUpperBound() { Configuration conf = new Configuration(); BinaryPartitioner.setRightOffset(conf, 4); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java index 4d05d13d445..6bad846f6d3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java @@ -19,17 +19,14 @@ package org.apache.hadoop.mapreduce.lib.partition; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import junit.framework.TestCase; -public class TestKeyFieldHelper { +public class TestKeyFieldHelper extends TestCase { private static final Log LOG = LogFactory.getLog(TestKeyFieldHelper.class); /** * Test is key-field-helper's parse option. */ - @Test public void testparseOption() throws Exception { KeyFieldHelper helper = new KeyFieldHelper(); helper.setKeyFieldSeparator("\t"); @@ -215,7 +212,6 @@ public class TestKeyFieldHelper { /** * Test is key-field-helper's getWordLengths. */ - @Test public void testGetWordLengths() throws Exception { KeyFieldHelper helper = new KeyFieldHelper(); helper.setKeyFieldSeparator("\t"); @@ -274,7 +270,6 @@ public class TestKeyFieldHelper { /** * Test is key-field-helper's getStartOffset/getEndOffset. */ - @Test public void testgetStartEndOffset() throws Exception { KeyFieldHelper helper = new KeyFieldHelper(); helper.setKeyFieldSeparator("\t"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java index 00b415f32cb..9c2fb48d9bf 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java @@ -19,16 +19,14 @@ package org.apache.hadoop.mapreduce.lib.partition; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; -import org.junit.Test; -import static org.junit.Assert.assertEquals; +import junit.framework.TestCase; -public class TestMRKeyFieldBasedPartitioner { +public class TestMRKeyFieldBasedPartitioner extends TestCase { /** * Test is key-field-based partitioned works with empty key. */ - @Test public void testEmptyKey() throws Exception { int numReducers = 10; KeyFieldBasedPartitioner kfbp = diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java index bdb4ff4794e..a844737e09d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java @@ -23,6 +23,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import junit.framework.TestCase; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; @@ -39,11 +41,8 @@ import org.apache.hadoop.io.serializer.JavaSerializationComparator; import org.apache.hadoop.io.serializer.Serialization; import org.apache.hadoop.io.serializer.WritableSerialization; import org.apache.hadoop.mapreduce.MRJobConfig; -import org.junit.Test; -import static org.junit.Assert.assertEquals; - -public class TestTotalOrderPartitioner { +public class TestTotalOrderPartitioner extends TestCase { private static final Text[] splitStrings = new Text[] { // -inf // 0 @@ -141,7 +140,6 @@ public class TestTotalOrderPartitioner { return p; } - @Test public void testTotalOrderWithCustomSerialization() throws Exception { TotalOrderPartitioner partitioner = new TotalOrderPartitioner(); @@ -167,7 +165,6 @@ public class TestTotalOrderPartitioner { } } - @Test public void testTotalOrderMemCmp() throws Exception { TotalOrderPartitioner partitioner = new TotalOrderPartitioner(); @@ -187,7 +184,6 @@ public class TestTotalOrderPartitioner { } } - @Test public void testTotalOrderBinarySearch() throws Exception { TotalOrderPartitioner partitioner = new TotalOrderPartitioner(); @@ -220,7 +216,6 @@ public class TestTotalOrderPartitioner { } } - @Test public void testTotalOrderCustomComparator() throws Exception { TotalOrderPartitioner partitioner = new TotalOrderPartitioner(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java index 07b5d8b9f50..e1849a3ce9c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java @@ -20,6 +20,8 @@ package org.apache.hadoop.mapreduce.util; import java.io.File; import java.io.IOException; +import junit.framework.TestCase; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -28,27 +30,20 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.mapreduce.util.MRAsyncDiskService; -import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - /** * A test for MRAsyncDiskService. */ -public class TestMRAsyncDiskService { +public class TestMRAsyncDiskService extends TestCase { public static final Log LOG = LogFactory.getLog(TestMRAsyncDiskService.class); private static String TEST_ROOT_DIR = new Path(System.getProperty( "test.build.data", "/tmp")).toString(); - @Before - public void setUp() { + @Override + protected void setUp() { FileUtil.fullyDelete(new File(TEST_ROOT_DIR)); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java index f68cc8310a6..aa769f85974 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.mapreduce.v2; +import junit.framework.TestCase; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; @@ -28,25 +29,22 @@ import org.apache.hadoop.mapred.MiniMRCluster; import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.ProxyUsers; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import java.net.InetAddress; +import java.io.File; +import java.io.FileOutputStream; +import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.Writer; -import java.net.InetAddress; import java.security.PrivilegedExceptionAction; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class TestMiniMRProxyUser { +public class TestMiniMRProxyUser extends TestCase { private MiniDFSCluster dfsCluster = null; private MiniMRCluster mrCluster = null; - - @Before - public void setUp() throws Exception { + + protected void setUp() throws Exception { + super.setUp(); if (System.getProperty("hadoop.log.dir") == null) { System.setProperty("hadoop.log.dir", "/tmp"); } @@ -93,14 +91,15 @@ public class TestMiniMRProxyUser { return mrCluster.createJobConf(); } - @After - public void tearDown() throws Exception { + @Override + protected void tearDown() throws Exception { if (mrCluster != null) { mrCluster.shutdown(); } if (dfsCluster != null) { dfsCluster.shutdown(); } + super.tearDown(); } private void mrRun() throws Exception { @@ -126,13 +125,11 @@ public class TestMiniMRProxyUser { assertTrue(runJob.isComplete()); assertTrue(runJob.isSuccessful()); } - - @Test + public void __testCurrentUser() throws Exception { mrRun(); } - @Test public void testValidProxyUser() throws Exception { UserGroupInformation ugi = UserGroupInformation.createProxyUser("u1", UserGroupInformation.getLoginUser()); ugi.doAs(new PrivilegedExceptionAction() { @@ -145,7 +142,6 @@ public class TestMiniMRProxyUser { }); } - @Test public void ___testInvalidProxyUser() throws Exception { UserGroupInformation ugi = UserGroupInformation.createProxyUser("u2", UserGroupInformation.getLoginUser()); ugi.doAs(new PrivilegedExceptionAction() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java index e90c509d7a8..b6947f3fc48 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.mapreduce.v2; +import junit.framework.TestCase; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; @@ -27,22 +28,17 @@ import org.apache.hadoop.mapred.JobID; import org.apache.hadoop.mapred.MiniMRCluster; import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.security.authorize.ProxyUsers; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; import java.io.IOException; import java.net.InetAddress; -import static org.junit.Assert.assertNull; - -public class TestNonExistentJob { +public class TestNonExistentJob extends TestCase { private MiniDFSCluster dfsCluster = null; private MiniMRCluster mrCluster = null; - @Before - public void setUp() throws Exception { + protected void setUp() throws Exception { + super.setUp(); if (System.getProperty("hadoop.log.dir") == null) { System.setProperty("hadoop.log.dir", "/tmp"); } @@ -82,17 +78,17 @@ public class TestNonExistentJob { return mrCluster.createJobConf(); } - @After - public void tearDown() throws Exception { + @Override + protected void tearDown() throws Exception { if (mrCluster != null) { mrCluster.shutdown(); } if (dfsCluster != null) { dfsCluster.shutdown(); } + super.tearDown(); } - @Test public void testGetInvalidJob() throws Exception { RunningJob runJob = new JobClient(getJobConf()).getJob(JobID.forName("job_0_0")); assertNull(runJob); diff --git a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingBadRecords.java b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingBadRecords.java index 860fb89cfcf..7b7901faad1 100644 --- a/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingBadRecords.java +++ b/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingBadRecords.java @@ -42,11 +42,6 @@ import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.SkipBadRecords; import org.apache.hadoop.mapred.Utils; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; -import org.junit.Before; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; public class TestStreamingBadRecords extends ClusterMapReduceTestCase { @@ -73,8 +68,7 @@ public class TestStreamingBadRecords extends ClusterMapReduceTestCase utilTest.redirectIfAntJunit(); } - @Before - public void setUp() throws Exception { + protected void setUp() throws Exception { Properties props = new Properties(); props.setProperty(JTConfig.JT_RETIREJOBS, "false"); props.setProperty(JTConfig.JT_PERSIST_JOBSTATUS, "false"); @@ -248,7 +242,6 @@ public class TestStreamingBadRecords extends ClusterMapReduceTestCase } */ - @Test public void testNoOp() { // Added to avoid warnings when running this disabled test } From d8383c687c95dbb37effa307ab2d41497da1cfc2 Mon Sep 17 00:00:00 2001 From: Walter Su Date: Mon, 28 Mar 2016 15:44:25 +0800 Subject: [PATCH 41/43] HDFS-10182. Hedged read might overwrite user's buf. Contributed by zhouyingchao. --- .../org/apache/hadoop/hdfs/DFSInputStream.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 7661e82b8a9..26898414448 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -1279,7 +1279,7 @@ public class DFSInputStream extends FSInputStream // chooseDataNode is a commitment. If no node, we go to // the NN to reget block locations. Only go here on first read. chosenNode = chooseDataNode(block, ignored); - bb = ByteBuffer.wrap(buf, offset, len); + bb = ByteBuffer.allocate(len); Callable getFromDataNodeCallable = getFromOneDataNode( chosenNode, block, start, end, bb, corruptedBlocks, hedgedReadId++); @@ -1290,7 +1290,9 @@ public class DFSInputStream extends FSInputStream Future future = hedgedService.poll( conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS); if (future != null) { - future.get(); + ByteBuffer result = future.get(); + System.arraycopy(result.array(), result.position(), buf, offset, + len); return; } DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged " @@ -1328,13 +1330,9 @@ public class DFSInputStream extends FSInputStream ByteBuffer result = getFirstToComplete(hedgedService, futures); // cancel the rest. cancelAll(futures); - if (result.array() != buf) { // compare the array pointers - dfsClient.getHedgedReadMetrics().incHedgedReadWins(); - System.arraycopy(result.array(), result.position(), buf, offset, - len); - } else { - dfsClient.getHedgedReadMetrics().incHedgedReadOps(); - } + dfsClient.getHedgedReadMetrics().incHedgedReadWins(); + System.arraycopy(result.array(), result.position(), buf, offset, + len); return; } catch (InterruptedException ie) { // Ignore and retry From 90fcb16c0fc7cded2593948f1cc08f3d1ae88bda Mon Sep 17 00:00:00 2001 From: Junping Du Date: Mon, 28 Mar 2016 05:30:42 -0700 Subject: [PATCH 42/43] MAPREDUCE-6655. Fix a typo (STRICT_IE6) in Encrypted Shuffle. Contributed by Wei-Chiu Chuang. --- .../hadoop-common/src/main/resources/core-default.xml | 2 +- .../src/site/markdown/EncryptedShuffle.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 187f92376d9..d6e2bcd8ca0 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1577,7 +1577,7 @@ DEFAULT The hostname verifier to provide for HttpsURLConnections. - Valid values are: DEFAULT, STRICT, STRICT_I6, DEFAULT_AND_LOCALHOST and + Valid values are: DEFAULT, STRICT, STRICT_IE6, DEFAULT_AND_LOCALHOST and ALLOW_ALL diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/EncryptedShuffle.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/EncryptedShuffle.md index f67978111a2..6aa466986b3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/EncryptedShuffle.md +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/EncryptedShuffle.md @@ -40,7 +40,7 @@ To enable encrypted shuffle, set the following properties in core-site.xml of al | **Property** | **Default Value** | **Explanation** | |:---- |:---- |:---- | | `hadoop.ssl.require.client.cert` | `false` | Whether client certificates are required | -| `hadoop.ssl.hostname.verifier` | `DEFAULT` | The hostname verifier to provide for HttpsURLConnections. Valid values are: **DEFAULT**, **STRICT**, **STRICT\_I6**, **DEFAULT\_AND\_LOCALHOST** and **ALLOW\_ALL** | +| `hadoop.ssl.hostname.verifier` | `DEFAULT` | The hostname verifier to provide for HttpsURLConnections. Valid values are: **DEFAULT**, **STRICT**, **STRICT\_IE6**, **DEFAULT\_AND\_LOCALHOST** and **ALLOW\_ALL** | | `hadoop.ssl.keystores.factory.class` | `org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory` | The KeyStoresFactory implementation to use | | `hadoop.ssl.server.conf` | `ssl-server.xml` | Resource file from which ssl server keystore information will be extracted. This file is looked up in the classpath, typically it should be in Hadoop conf/ directory | | `hadoop.ssl.client.conf` | `ssl-client.xml` | Resource file from which ssl server keystore information will be extracted. This file is looked up in the classpath, typically it should be in Hadoop conf/ directory | From 8831b18c65c8a61e400346dc308e068381835357 Mon Sep 17 00:00:00 2001 From: Jian He Date: Mon, 28 Mar 2016 09:23:06 -0700 Subject: [PATCH 43/43] Missing file for YARN-4117. --- .../yarn/client/api/impl/TestAMRMProxy.java | 413 ++++++++++++++++++ 1 file changed, 413 insertions(+) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java new file mode 100644 index 00000000000..b92538acdd0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java @@ -0,0 +1,413 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.hadoop.yarn.client.api.impl; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; +import org.apache.hadoop.yarn.api.records.ApplicationAccessType; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.NodeState; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.ClientRMProxy; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; +import org.apache.hadoop.yarn.server.MiniYARNCluster; +import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyTokenSecretManager; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import org.apache.hadoop.yarn.util.Records; +import org.junit.Assert; +import org.junit.Test; + +public class TestAMRMProxy { + + private static final Log LOG = LogFactory.getLog(TestAMRMProxy.class); + + /* + * This test validates register, allocate and finish of an application through + * the AMRMPRoxy. + */ + @Test(timeout = 60000) + public void testAMRMProxyE2E() throws Exception { + MiniYARNCluster cluster = new MiniYARNCluster("testAMRMProxyE2E", 1, 1, 1); + YarnClient rmClient = null; + ApplicationMasterProtocol client; + + try { + Configuration conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true); + cluster.init(conf); + cluster.start(); + final Configuration yarnConf = cluster.getConfig(); + + // the client has to connect to AMRMProxy + + yarnConf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_AMRM_PROXY_ADDRESS); + rmClient = YarnClient.createYarnClient(); + rmClient.init(yarnConf); + rmClient.start(); + + // Submit application + + ApplicationId appId = createApp(rmClient, cluster); + + client = createAMRMProtocol(rmClient, appId, cluster, yarnConf); + + LOG.info("testAMRMProxyE2E - Register Application Master"); + + RegisterApplicationMasterResponse responseRegister = + client.registerApplicationMaster(RegisterApplicationMasterRequest + .newInstance(NetUtils.getHostname(), 1024, "")); + + Assert.assertNotNull(responseRegister); + Assert.assertNotNull(responseRegister.getQueue()); + Assert.assertNotNull(responseRegister.getApplicationACLs()); + Assert.assertNotNull(responseRegister.getClientToAMTokenMasterKey()); + Assert + .assertNotNull(responseRegister.getContainersFromPreviousAttempts()); + Assert.assertNotNull(responseRegister.getSchedulerResourceTypes()); + Assert.assertNotNull(responseRegister.getMaximumResourceCapability()); + + RMApp rmApp = + cluster.getResourceManager().getRMContext().getRMApps().get(appId); + Assert.assertEquals(RMAppState.RUNNING, rmApp.getState()); + + LOG.info("testAMRMProxyE2E - Allocate Resources Application Master"); + + AllocateRequest request = + createAllocateRequest(rmClient.getNodeReports(NodeState.RUNNING)); + + AllocateResponse allocResponse = client.allocate(request); + Assert.assertNotNull(allocResponse); + Assert.assertEquals(0, allocResponse.getAllocatedContainers().size()); + + request.setAskList(new ArrayList()); + request.setResponseId(request.getResponseId() + 1); + + Thread.sleep(1000); + + // RM should allocate container within 2 calls to allocate() + allocResponse = client.allocate(request); + Assert.assertNotNull(allocResponse); + Assert.assertEquals(2, allocResponse.getAllocatedContainers().size()); + + LOG.info("testAMRMPRoxy - Finish Application Master"); + + FinishApplicationMasterResponse responseFinish = + client.finishApplicationMaster(FinishApplicationMasterRequest + .newInstance(FinalApplicationStatus.SUCCEEDED, "success", null)); + + Assert.assertNotNull(responseFinish); + + Thread.sleep(500); + Assert.assertNotEquals(RMAppState.FINISHED, rmApp.getState()); + + } finally { + if (rmClient != null) { + rmClient.stop(); + } + cluster.stop(); + } + } + + /* + * This test validates the token renewal from the AMRMPRoxy. The test verifies + * that the received token it is different from the previous one within 5 + * requests. + */ + @Test(timeout = 60000) + public void testE2ETokenRenewal() throws Exception { + MiniYARNCluster cluster = + new MiniYARNCluster("testE2ETokenRenewal", 1, 1, 1); + YarnClient rmClient = null; + ApplicationMasterProtocol client; + + try { + Configuration conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true); + conf.setInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 1500); + conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 1500); + conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 1500); + // RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS should be at least + // RM_AM_EXPIRY_INTERVAL_MS * 1.5 *3 + conf.setInt( + YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS, 6); + cluster.init(conf); + cluster.start(); + final Configuration yarnConf = cluster.getConfig(); + yarnConf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_AMRM_PROXY_ADDRESS); + rmClient = YarnClient.createYarnClient(); + rmClient.init(yarnConf); + rmClient.start(); + + // Submit + + ApplicationId appId = createApp(rmClient, cluster); + + client = createAMRMProtocol(rmClient, appId, cluster, yarnConf); + + client.registerApplicationMaster(RegisterApplicationMasterRequest + .newInstance(NetUtils.getHostname(), 1024, "")); + + LOG.info("testAMRMPRoxy - Allocate Resources Application Master"); + + AllocateRequest request = + createAllocateRequest(rmClient.getNodeReports(NodeState.RUNNING)); + + Token lastToken = null; + AllocateResponse response = null; + + for (int i = 0; i < 5; i++) { + + response = client.allocate(request); + request.setResponseId(request.getResponseId() + 1); + + if (response.getAMRMToken() != null + && !response.getAMRMToken().equals(lastToken)) { + break; + } + + lastToken = response.getAMRMToken(); + + // Time slot to be sure the RM renew the token + Thread.sleep(1500); + + } + + Assert.assertFalse(response.getAMRMToken().equals(lastToken)); + + LOG.info("testAMRMPRoxy - Finish Application Master"); + + client.finishApplicationMaster(FinishApplicationMasterRequest + .newInstance(FinalApplicationStatus.SUCCEEDED, "success", null)); + + } finally { + if (rmClient != null) { + rmClient.stop(); + } + cluster.stop(); + } + } + + /* + * This test validates that an AM cannot register directly to the RM, with the + * token provided by the AMRMProxy. + */ + @Test(timeout = 60000) + public void testE2ETokenSwap() throws Exception { + MiniYARNCluster cluster = new MiniYARNCluster("testE2ETokenSwap", 1, 1, 1); + YarnClient rmClient = null; + ApplicationMasterProtocol client; + + try { + Configuration conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true); + cluster.init(conf); + cluster.start(); + + // the client will connect to the RM with the token provided by AMRMProxy + final Configuration yarnConf = cluster.getConfig(); + rmClient = YarnClient.createYarnClient(); + rmClient.init(yarnConf); + rmClient.start(); + + ApplicationId appId = createApp(rmClient, cluster); + + client = createAMRMProtocol(rmClient, appId, cluster, yarnConf); + + try { + client.registerApplicationMaster(RegisterApplicationMasterRequest + .newInstance(NetUtils.getHostname(), 1024, "")); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue( + e.getMessage().startsWith("Invalid AMRMToken from appattempt_")); + } + + } finally { + if (rmClient != null) { + rmClient.stop(); + } + cluster.stop(); + } + } + + private ApplicationMasterProtocol createAMRMProtocol(YarnClient rmClient, + ApplicationId appId, MiniYARNCluster cluster, + final Configuration yarnConf) + throws IOException, InterruptedException, YarnException { + + UserGroupInformation user = null; + + // Get the AMRMToken from AMRMProxy + + ApplicationReport report = rmClient.getApplicationReport(appId); + + user = UserGroupInformation.createProxyUser( + report.getCurrentApplicationAttemptId().toString(), + UserGroupInformation.getCurrentUser()); + + ContainerManagerImpl containerManager = (ContainerManagerImpl) cluster + .getNodeManager(0).getNMContext().getContainerManager(); + + AMRMProxyTokenSecretManager amrmTokenSecretManager = + containerManager.getAMRMProxyService().getSecretManager(); + org.apache.hadoop.security.token.Token token = + amrmTokenSecretManager + .createAndGetAMRMToken(report.getCurrentApplicationAttemptId()); + + SecurityUtil.setTokenService(token, + containerManager.getAMRMProxyService().getBindAddress()); + user.addToken(token); + + // Start Application Master + + return user + .doAs(new PrivilegedExceptionAction() { + @Override + public ApplicationMasterProtocol run() throws Exception { + return ClientRMProxy.createRMProxy(yarnConf, + ApplicationMasterProtocol.class); + } + }); + } + + private AllocateRequest createAllocateRequest(List listNode) { + // The test needs AMRMClient to create a real allocate request + AMRMClientImpl amClient = + new AMRMClientImpl(); + + Resource capability = Resource.newInstance(1024, 2); + Priority priority = Priority.newInstance(1); + List nodeReports = listNode; + String node = nodeReports.get(0).getNodeId().getHost(); + String[] nodes = new String[] { node }; + + ContainerRequest storedContainer1 = + new ContainerRequest(capability, nodes, null, priority); + amClient.addContainerRequest(storedContainer1); + amClient.addContainerRequest(storedContainer1); + + List resourceAsk = new ArrayList(); + for (ResourceRequest rr : amClient.ask) { + resourceAsk.add(rr); + } + + ResourceBlacklistRequest resourceBlacklistRequest = ResourceBlacklistRequest + .newInstance(new ArrayList(), new ArrayList()); + + int responseId = 1; + + return AllocateRequest.newInstance(responseId, 0, resourceAsk, + new ArrayList(), resourceBlacklistRequest); + } + + private ApplicationId createApp(YarnClient yarnClient, + MiniYARNCluster yarnCluster) throws Exception { + + ApplicationSubmissionContext appContext = + yarnClient.createApplication().getApplicationSubmissionContext(); + ApplicationId appId = appContext.getApplicationId(); + + appContext.setApplicationName("Test"); + + Priority pri = Records.newRecord(Priority.class); + pri.setPriority(0); + appContext.setPriority(pri); + + appContext.setQueue("default"); + + ContainerLaunchContext amContainer = BuilderUtils.newContainerLaunchContext( + Collections. emptyMap(), + new HashMap(), Arrays.asList("sleep", "10000"), + new HashMap(), null, + new HashMap()); + appContext.setAMContainerSpec(amContainer); + appContext.setResource(Resource.newInstance(1024, 1)); + + SubmitApplicationRequest appRequest = + Records.newRecord(SubmitApplicationRequest.class); + appRequest.setApplicationSubmissionContext(appContext); + + yarnClient.submitApplication(appContext); + + RMAppAttempt appAttempt = null; + while (true) { + ApplicationReport appReport = yarnClient.getApplicationReport(appId); + if (appReport + .getYarnApplicationState() == YarnApplicationState.ACCEPTED) { + ApplicationAttemptId attemptId = + appReport.getCurrentApplicationAttemptId(); + appAttempt = yarnCluster.getResourceManager().getRMContext().getRMApps() + .get(attemptId.getApplicationId()).getCurrentAppAttempt(); + while (true) { + if (appAttempt.getAppAttemptState() == RMAppAttemptState.LAUNCHED) { + break; + } + } + break; + } + } + Thread.sleep(1000); + return appId; + } +}