diff --git a/dev-support/smart-apply-patch.sh b/dev-support/smart-apply-patch.sh index 4ef1e8ae37..3e7b16d974 100755 --- a/dev-support/smart-apply-patch.sh +++ b/dev-support/smart-apply-patch.sh @@ -39,40 +39,68 @@ fi # Come up with a list of changed files into $TMP TMP=/tmp/tmp.paths.$$ TOCLEAN="$TOCLEAN $TMP" -grep '^+++\|^---' $PATCH_FILE | cut -c '5-' | grep -v /dev/null | sort | uniq > $TMP -# Assume p0 to start -PLEVEL=0 +if $PATCH -p0 -E --dry-run < $PATCH_FILE 2>&1 > $TMP; then + PLEVEL=0 + #if the patch applied at P0 there is the possability that all we are doing + # is adding new files and they would apply anywhere. So try to guess the + # correct place to put those files. -# if all of the lines start with a/ or b/, then this is a git patch that -# was generated without --no-prefix -if ! grep -qv '^a/\|^b/' $TMP ; then - echo Looks like this is a git patch. Stripping a/ and b/ prefixes - echo and incrementing PLEVEL - PLEVEL=$[$PLEVEL + 1] - sed -i -e 's,^[ab]/,,' $TMP -fi + TMP2=/tmp/tmp.paths.2.$$ + TOCLEAN="$TOCLEAN $TMP2" -PREFIX_DIRS=$(cut -d '/' -f 1 $TMP | sort | uniq) + grep '^patching file ' $TMP | awk '{print $3}' | grep -v /dev/null | sort | uniq > $TMP2 -# if we are at the project root then nothing more to do -if [[ -d hadoop-common-project ]]; then - echo Looks like this is being run at project root + #first off check that all of the files do not exist + FOUND_ANY=0 + for CHECK_FILE in $(cat $TMP2) + do + if [[ -f $CHECK_FILE ]]; then + FOUND_ANY=1 + fi + done -# if all of the lines start with hadoop-common/, hadoop-hdfs/, or hadoop-mapreduce/, this is -# relative to the hadoop root instead of the subproject root, so we need -# to chop off another layer -elif [[ "$PREFIX_DIRS" =~ ^(hadoop-common-project|hadoop-hdfs-project|hadoop-mapreduce-project)$ ]]; then + if [[ "$FOUND_ANY" = "0" ]]; then + #all of the files are new files so we have to guess where the correct place to put it is. - echo Looks like this is relative to project root. Increasing PLEVEL - PLEVEL=$[$PLEVEL + 1] + # if all of the lines start with a/ or b/, then this is a git patch that + # was generated without --no-prefix + if ! grep -qv '^a/\|^b/' $TMP2 ; then + echo Looks like this is a git patch. Stripping a/ and b/ prefixes + echo and incrementing PLEVEL + PLEVEL=$[$PLEVEL + 1] + sed -i -e 's,^[ab]/,,' $TMP2 + fi -elif ! echo "$PREFIX_DIRS" | grep -vxq 'hadoop-common-project\|hadoop-hdfs-project\|hadoop-mapreduce-project' ; then - echo Looks like this is a cross-subproject patch. Try applying from the project root - exit 1 + PREFIX_DIRS_AND_FILES=$(cut -d '/' -f 1 | sort | uniq) + + # if we are at the project root then nothing more to do + if [[ -d hadoop-common-project ]]; then + echo Looks like this is being run at project root + + # if all of the lines start with hadoop-common/, hadoop-hdfs/, or hadoop-mapreduce/, this is + # relative to the hadoop root instead of the subproject root, so we need + # to chop off another layer + elif [[ "$PREFIX_DIRS_AND_FILES" =~ ^(hadoop-common-project|hadoop-hdfs-project|hadoop-mapreduce-project)$ ]]; then + + echo Looks like this is relative to project root. Increasing PLEVEL + PLEVEL=$[$PLEVEL + 1] + + elif ! echo "$PREFIX_DIRS_AND_FILES" | grep -vxq 'hadoop-common-project\|hadoop-hdfs-project\|hadoop-mapreduce-project' ; then + echo Looks like this is a cross-subproject patch. Try applying from the project root + cleanup 1 + fi + fi +elif $PATCH -p1 -E --dry-run < $PATCH_FILE 2>&1 > /dev/null; then + PLEVEL=1 +elif $PATCH -p2 -E --dry-run < $PATCH_FILE 2>&1 > /dev/null; then + PLEVEL=2 +else + echo "The patch does not appear to apply with p0 to p2"; + cleanup 1; fi echo Going to apply patch with: $PATCH -p$PLEVEL $PATCH -p$PLEVEL -E < $PATCH_FILE -cleanup 0 +cleanup $? diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh index 1be8d04850..76b020a01e 100755 --- a/dev-support/test-patch.sh +++ b/dev-support/test-patch.sh @@ -64,6 +64,7 @@ printUsage() { echo "--findbugs-home= Findbugs home directory (default FINDBUGS_HOME environment variable)" echo "--forrest-home= Forrest home directory (default FORREST_HOME environment variable)" echo "--dirty-workspace Allow the local SVN workspace to have uncommitted changes" + echo "--run-tests Run all tests below the base directory" echo echo "Jenkins-only options:" echo "--jenkins Run by Jenkins (runs tests and posts results to JIRA)" @@ -130,6 +131,9 @@ parseArgs() { --dirty-workspace) DIRTY_WORKSPACE=true ;; + --run-tests) + RUN_TESTS=true + ;; *) PATCH_OR_DEFECT=$i ;; @@ -249,6 +253,18 @@ setup () { echo "======================================================================" echo "" echo "" + if [[ ! -d hadoop-common-project ]]; then + cd $bindir/.. + echo "Compiling $(pwd)" + echo "$MVN clean test -DskipTests > $PATCH_DIR/trunkCompile.txt 2>&1" + $MVN clean test -DskipTests > $PATCH_DIR/trunkCompile.txt 2>&1 + if [[ $? != 0 ]] ; then + echo "Top-level trunk compilation is broken?" + cleanupAndExit 1 + fi + cd - + fi + echo "Compiling $(pwd)" echo "$MVN clean test -DskipTests -D${PROJECT_NAME}PatchProcess -Ptest-patch > $PATCH_DIR/trunkJavacWarnings.txt 2>&1" $MVN clean test -DskipTests -D${PROJECT_NAME}PatchProcess -Ptest-patch > $PATCH_DIR/trunkJavacWarnings.txt 2>&1 if [[ $? != 0 ]] ; then @@ -550,8 +566,10 @@ checkFindbugsWarnings () { $FINDBUGS_HOME/bin/convertXmlToText -html \ $PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.xml \ $PATCH_DIR/newPatchFindbugsWarnings${module_suffix}.html - JIRA_COMMENT_FOOTER="Findbugs warnings: $BUILD_URL/artifact/trunk/patchprocess/newPatchFindbugsWarnings${module_suffix}.html + if [[ $newFindbugsWarnings > 0 ]] ; then + JIRA_COMMENT_FOOTER="Findbugs warnings: $BUILD_URL/artifact/trunk/$(basename $BASEDIR)/patchprocess/newPatchFindbugsWarnings${module_suffix}.html $JIRA_COMMENT_FOOTER" + fi done ### if current warnings greater than OK_FINDBUGS_WARNINGS @@ -580,26 +598,12 @@ runTests () { echo "" echo "" - failed_tests="" - modules=$(findModules) - for module in $modules; - do - pushd $module - echo " Running tests in $module" - ### Kill any rogue build processes from the last attempt - $PS auxwww | $GREP ${PROJECT_NAME}PatchProcess | $AWK '{print $2}' | /usr/bin/xargs -t -I {} /bin/kill -9 {} > /dev/null - - echo "$MVN clean test -Pnative -D${PROJECT_NAME}PatchProcess" - $MVN clean test -Pnative -D${PROJECT_NAME}PatchProcess - if [[ $? != 0 ]] ; then - ### Find and format names of failed tests - module_failed_tests=`find . -name 'TEST*.xml' | xargs $GREP -l -E " $TMP - - # if all of the lines start with a/ or b/, then this is a git patch that - # was generated without --no-prefix - if ! $GREP -qv '^a/\|^b/' $TMP ; then - sed -i -e 's,^[ab]/,,' $TMP - fi - - PREFIX_DIRS=$(cut -d '/' -f 1 $TMP | sort | uniq) - - # if all of the lines start with hadoop-common-project/, hadoop-hdfs-project/, or hadoop-mapreduce-project/, this is - # relative to the hadoop root instead of the subproject root - if [[ "$PREFIX_DIRS" =~ ^(hadoop-common-project|hadoop-hdfs-project|hadoop-mapreduce-project)$ ]]; then - echo $PREFIX_DIRS - return 0 - elif ! echo "$PREFIX_DIRS" | grep -vxq 'hadoop-common-project\|hadoop-hdfs-project\|hadoop-mapreduce-project' ; then - echo $PREFIX_DIRS - return 0 - fi - - # No modules found. Running from current directory. - echo . -} - ############################################################################### ### Run the test-contrib target runContribTests () { @@ -820,8 +794,8 @@ checkFindbugsWarnings (( RESULT = RESULT + $? )) checkReleaseAuditWarnings (( RESULT = RESULT + $? )) -### Do not call these when run by a developer -if [[ $JENKINS == "true" ]] ; then +### Run tests for Jenkins or if explictly asked for by a developer +if [[ $JENKINS == "true" || $RUN_TESTS == "true" ]] ; then runTests (( RESULT = RESULT + $? )) runContribTests diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml index 4d4e537738..8d67b2a8a2 100644 --- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml +++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml @@ -78,6 +78,13 @@ *-site.xml + + ${basedir}/src/main/packages/templates/conf + /share/hadoop/${hadoop.component}/templates/conf + + * + + ${project.build.directory} /share/hadoop/${hadoop.component} diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index d62fa2db3a..98a78fe065 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -5,13 +5,19 @@ Trunk (unreleased changes) IMPROVEMENTS HADOOP-7595. Upgrade dependency to Avro 1.5.3. (Alejandro Abdelnur via atm) - HADOOP-7524 Change RPC to allow multiple protocols including multuple versions of the same protocol (sanjay Radia) + + HADOOP-7524. Change RPC to allow multiple protocols including multuple + versions of the same protocol (sanjay Radia) + + HADOOP-7607. Simplify the RPC proxy cleanup process. (atm) BUGS HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required by Jersey (Alejandro Abdelnur via atm) + HADOOP-7610. Fix for hadoop debian package (Eric Yang via gkesavan) + Release 0.23.0 - Unreleased INCOMPATIBLE CHANGES @@ -371,6 +377,12 @@ Release 0.23.0 - Unreleased HADOOP-7507. Allow ganglia metrics to include the metrics system tags in the gmetric names. (Alejandro Abdelnur via todd) + HADOOP-7612. Change test-patch to run tests for all nested modules. + (tomwhite) + + HADOOP-7599. Script improvements to setup a secure Hadoop cluster + (Eric Yang via ddas) + OPTIMIZATIONS HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole @@ -563,6 +575,14 @@ Release 0.23.0 - Unreleased HADOOP-7593. Fix AssertionError in TestHttpServer.testMaxThreads(). (Uma Maheswara Rao G via szetszwo) + HADOOP-7598. Fix smart-apply-patch.sh to handle patching from a sub + directory correctly. (Robert Evans via acmurthy) + + HADOOP-7328. When a serializer class is missing, return null, not throw + an NPE. (Harsh J Chouraria via todd) + + HADOOP-7626. Bugfix for a config generator (Eric Yang via ddas) + Release 0.22.0 - Unreleased INCOMPATIBLE CHANGES @@ -1075,6 +1095,9 @@ Release 0.22.0 - Unreleased HADOOP-7390. VersionInfo not generated properly in git after unsplit. (todd via atm) + HADOOP-7568. SequenceFile should not print into stdout. + (Plamen Jeliazkov via shv) + Release 0.21.1 - Unreleased IMPROVEMENTS diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java index 0da6630b20..a7d1ba63ec 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java @@ -1668,7 +1668,6 @@ private void initialize(Path filename, FSDataInputStream in, try { seek(start); this.end = this.in.getPos() + length; - System.out.println("Setting end to " + end); // if it wrapped around, use the max if (end < length) { end = Long.MAX_VALUE; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java index a48d114b18..52a0a253bb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java @@ -27,10 +27,10 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.io.serializer.avro.AvroReflectSerialization; import org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization; import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hadoop.util.StringUtils; /** *

@@ -50,14 +50,15 @@ public class SerializationFactory extends Configured { *

* Serializations are found by reading the io.serializations * property from conf, which is a comma-delimited list of - * classnames. + * classnames. *

*/ public SerializationFactory(Configuration conf) { super(conf); - for (String serializerName : conf.getStrings("io.serializations", - new String[]{WritableSerialization.class.getName(), - AvroSpecificSerialization.class.getName(), + for (String serializerName : conf.getStrings( + CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, + new String[]{WritableSerialization.class.getName(), + AvroSpecificSerialization.class.getName(), AvroReflectSerialization.class.getName()})) { add(conf, serializerName); } @@ -67,27 +68,35 @@ public SerializationFactory(Configuration conf) { private void add(Configuration conf, String serializationName) { try { Class serializionClass = - (Class) conf.getClassByName(serializationName); + (Class) conf.getClassByName(serializationName); serializations.add((Serialization) - ReflectionUtils.newInstance(serializionClass, getConf())); + ReflectionUtils.newInstance(serializionClass, getConf())); } catch (ClassNotFoundException e) { LOG.warn("Serialization class not found: ", e); } } public Serializer getSerializer(Class c) { - return getSerialization(c).getSerializer(c); + Serialization serializer = getSerialization(c); + if (serializer != null) { + return serializer.getSerializer(c); + } + return null; } public Deserializer getDeserializer(Class c) { - return getSerialization(c).getDeserializer(c); + Serialization serializer = getSerialization(c); + if (serializer != null) { + return serializer.getDeserializer(c); + } + return null; } @SuppressWarnings("unchecked") public Serialization getSerialization(Class c) { for (Serialization serialization : serializations) { if (serialization.accept(c)) { - return (Serialization) serialization; + return (Serialization) serialization; } } return null; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java index 5ab379c1da..1b73351bf1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java @@ -131,7 +131,7 @@ public void writeBuffers(List buffers) throws IOException { } public void close() throws IOException { - ENGINE.stopProxy(tunnel); + RPC.stopProxy(tunnel); } } @@ -152,15 +152,6 @@ public ProtocolProxy getProxy(Class protocol, long clientVersion, false); } - /** Stop this proxy. */ - public void stopProxy(Object proxy) { - try { - ((Invoker)Proxy.getInvocationHandler(proxy)).close(); - } catch (IOException e) { - LOG.warn("Error while stopping "+proxy, e); - } - } - private class Invoker implements InvocationHandler, Closeable { private final ClientTransceiver tx; private final SpecificRequestor requestor; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java index 453a5dd175..beb8cc1cfa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ipc; +import java.lang.reflect.InvocationHandler; import java.lang.reflect.Proxy; import java.lang.reflect.Method; @@ -26,6 +27,7 @@ import java.net.NoRouteToHostException; import java.net.SocketTimeoutException; import java.io.*; +import java.io.Closeable; import java.util.Map; import java.util.HashMap; @@ -80,12 +82,8 @@ static public String getProtocolName(Class protocol) { private RPC() {} // no public ctor // cache of RpcEngines by protocol - private static final Map PROTOCOL_ENGINES - = new HashMap(); - - // track what RpcEngine is used by a proxy class, for stopProxy() - private static final Map PROXY_ENGINES - = new HashMap(); + private static final Map,RpcEngine> PROTOCOL_ENGINES + = new HashMap,RpcEngine>(); private static final String ENGINE_PROP = "rpc.engine"; @@ -96,32 +94,23 @@ private RPC() {} // no public ctor * @param engine the RpcEngine impl */ public static void setProtocolEngine(Configuration conf, - Class protocol, Class engine) { + Class protocol, Class engine) { conf.setClass(ENGINE_PROP+"."+protocol.getName(), engine, RpcEngine.class); } // return the RpcEngine configured to handle a protocol - private static synchronized RpcEngine getProtocolEngine(Class protocol, + private static synchronized RpcEngine getProtocolEngine(Class protocol, Configuration conf) { RpcEngine engine = PROTOCOL_ENGINES.get(protocol); if (engine == null) { Class impl = conf.getClass(ENGINE_PROP+"."+protocol.getName(), WritableRpcEngine.class); engine = (RpcEngine)ReflectionUtils.newInstance(impl, conf); - if (protocol.isInterface()) - PROXY_ENGINES.put(Proxy.getProxyClass(protocol.getClassLoader(), - protocol), - engine); PROTOCOL_ENGINES.put(protocol, engine); } return engine; } - // return the RpcEngine that handles a proxy object - private static synchronized RpcEngine getProxyEngine(Object proxy) { - return PROXY_ENGINES.get(proxy.getClass()); - } - /** * A version mismatch for the RPC protocol. */ @@ -477,13 +466,30 @@ public static ProtocolProxy getProtocolProxy(Class protocol, } /** - * Stop this proxy and release its invoker's resource - * @param proxy the proxy to be stopped + * Stop this proxy and release its invoker's resource by getting the + * invocation handler for the given proxy object and calling + * {@link Closeable#close} if that invocation handler implements + * {@link Closeable}. + * + * @param proxy the RPC proxy object to be stopped */ public static void stopProxy(Object proxy) { - RpcEngine rpcEngine; - if (proxy!=null && (rpcEngine = getProxyEngine(proxy)) != null) { - rpcEngine.stopProxy(proxy); + InvocationHandler invocationHandler = null; + try { + invocationHandler = Proxy.getInvocationHandler(proxy); + } catch (IllegalArgumentException e) { + LOG.error("Tried to call RPC.stopProxy on an object that is not a proxy.", e); + } + if (proxy != null && invocationHandler != null && + invocationHandler instanceof Closeable) { + try { + ((Closeable)invocationHandler).close(); + } catch (IOException e) { + LOG.error("Stopping RPC invocation handler caused exception", e); + } + } else { + LOG.error("Could not get invocation handler " + invocationHandler + + " for proxy " + proxy + ", or invocation handler is not closeable."); } } @@ -532,7 +538,7 @@ public static Server getServer(final Object instance, final String bindAddress, } /** Construct a server for a protocol implementation instance. */ - public static Server getServer(Class protocol, + public static Server getServer(Class protocol, Object instance, String bindAddress, int port, Configuration conf) throws IOException { @@ -543,7 +549,7 @@ public static Server getServer(Class protocol, * @deprecated secretManager should be passed. */ @Deprecated - public static Server getServer(Class protocol, + public static Server getServer(Class protocol, Object instance, String bindAddress, int port, int numHandlers, boolean verbose, Configuration conf) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java index 500cd9537c..a9076e7d1e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java @@ -41,9 +41,6 @@ ProtocolProxy getProxy(Class protocol, UserGroupInformation ticket, Configuration conf, SocketFactory factory, int rpcTimeout) throws IOException; - /** Stop this proxy. */ - void stopProxy(Object proxy); - /** Expert: Make multiple, parallel calls to a set of servers. */ Object[] call(Method method, Object[][] params, InetSocketAddress[] addrs, UserGroupInformation ticket, Configuration conf) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java index b28949d99a..75b8d51f05 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.io.Closeable; import java.util.Map; import java.util.HashMap; @@ -219,7 +220,7 @@ public Configuration getConf() { private static ClientCache CLIENTS=new ClientCache(); - private static class Invoker implements InvocationHandler { + private static class Invoker implements InvocationHandler, Closeable { private Client.ConnectionId remoteId; private Client client; private boolean isClosed = false; @@ -250,7 +251,7 @@ public Object invoke(Object proxy, Method method, Object[] args) } /* close the IPC client that's responsible for this invoker's RPCs */ - synchronized private void close() { + synchronized public void close() { if (!isClosed) { isClosed = true; CLIENTS.stopClient(client); @@ -281,15 +282,6 @@ public ProtocolProxy getProxy(Class protocol, long clientVersion, factory, rpcTimeout)); return new ProtocolProxy(protocol, proxy, true); } - - /** - * Stop this proxy and release its invoker's resource - * @param proxy the proxy to be stopped - */ - public void stopProxy(Object proxy) { - ((Invoker)Proxy.getInvocationHandler(proxy)).close(); - } - /** Expert: Make multiple, parallel calls to a set of servers. */ public Object[] call(Method method, Object[][] params, diff --git a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-datanode b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-datanode index 77bc6a5f17..81795879cf 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-datanode +++ b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-datanode @@ -39,6 +39,14 @@ fi . /lib/lsb/init-functions +if [ -n "$HADOOP_SECURE_DN_USER" ]; then + DN_USER="root" + IDENT_USER=${HADOOP_SECURE_DN_USER} +else + DN_USER="hdfs" + IDENT_USER=${DN_USER} +fi + # Are we running from init? run_by_init() { ([ "$previous" ] && [ "$runlevel" ]) || [ "$runlevel" = S ] @@ -67,13 +75,14 @@ check_privsep_dir() { } export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" +export HADOOP_PREFIX="/usr" case "$1" in start) check_privsep_dir check_for_no_start log_daemon_msg "Starting Apache Hadoop Data Node server" "hadoop-datanode" - if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-datanode.pid -c hdfs -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start datanode; then + if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-${IDENT_USER}-datanode.pid -c ${DN_USER} -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start datanode; then log_end_msg 0 else log_end_msg 1 @@ -81,7 +90,7 @@ case "$1" in ;; stop) log_daemon_msg "Stopping Apache Hadoop Data Node server" "hadoop-datanode" - if start-stop-daemon --stop --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-datanode.pid; then + if start-stop-daemon --stop --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-${IDENT_USER}-datanode.pid; then log_end_msg 0 else log_end_msg 1 @@ -91,9 +100,9 @@ case "$1" in restart) check_privsep_dir log_daemon_msg "Restarting Apache Hadoop Data Node server" "hadoop-datanode" - start-stop-daemon --stop --quiet --oknodo --retry 30 --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-datanode.pid + start-stop-daemon --stop --quiet --oknodo --retry 30 --pidfile ${HADOOP_PID_DIR}/hadoop-${IDENT_USER}-datanode.pid check_for_no_start log_end_msg - if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-datanode.pid -c hdfs -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start datanode; then + if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-${IDENT_USER}-datanode.pid -c ${DN_USER} -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start datanode; then log_end_msg 0 else log_end_msg 1 @@ -104,14 +113,14 @@ case "$1" in check_privsep_dir log_daemon_msg "Restarting Apache Hadoop Data Node server" "hadoop-datanode" set +e - start-stop-daemon --stop --quiet --retry 30 --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-datanode.pid + start-stop-daemon --stop --quiet --retry 30 --pidfile ${HADOOP_PID_DIR}/hadoop-${IDENT_USER}-datanode.pid RET="$?" set -e case $RET in 0) # old daemon stopped check_for_no_start log_end_msg - if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-datanode.pid -c hdfs -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start datanode; then + if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-${IDENT_USER}-datanode.pid -c ${DN_USER} -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start datanode; then log_end_msg 0 else log_end_msg 1 @@ -131,7 +140,7 @@ case "$1" in ;; status) - status_of_proc -p ${HADOOP_PID_DIR}/hadoop-hdfs-datanode.pid ${JAVA_HOME}/bin/java hadoop-datanode && exit 0 || exit $? + status_of_proc -p ${HADOOP_PID_DIR}/hadoop-${IDENT_USER}-datanode.pid ${JAVA_HOME}/bin/java hadoop-datanode && exit 0 || exit $? ;; *) diff --git a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-jobtracker b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-jobtracker index 61e47af342..c483d76a17 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-jobtracker +++ b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-jobtracker @@ -67,6 +67,7 @@ check_privsep_dir() { } export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" +export HADOOP_PREFIX="/usr" case "$1" in start) diff --git a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-namenode b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-namenode index 520177facf..e2608769d4 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-namenode +++ b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-namenode @@ -67,10 +67,11 @@ check_privsep_dir() { } format() { - su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} namenode -format' hdfs + sudo -u hdfs ${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} namenode -format } export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" +export HADOOP_PREFIX="/usr" case "$1" in start) diff --git a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-tasktracker b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-tasktracker index 582875ce85..983030bbb6 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-tasktracker +++ b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-tasktracker @@ -67,6 +67,7 @@ check_privsep_dir() { } export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" +export HADOOP_PREFIX="/usr" case "$1" in start) diff --git a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-create-user.sh b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-create-user.sh index 2426d4dc8b..a5d4304af5 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-create-user.sh +++ b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-create-user.sh @@ -14,9 +14,10 @@ # 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. - -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` +this="${BASH_SOURCE-$0}" +bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P) +script="$(basename -- "$this")" +this="$bin/$script" if [ "$HADOOP_HOME" != "" ]; then echo "Warning: \$HADOOP_HOME is deprecated." @@ -29,30 +30,86 @@ usage() { echo " usage: $0 Require parameter: - -u Create user on HDFS + --config /etc/hadoop Location of Hadoop configuration file + -u Create user on HDFS Optional parameters: - -h Display this message + -h Display this message + --kerberos-realm=KERBEROS.EXAMPLE.COM Set Kerberos realm + --super-user=hdfs Set super user id + --super-user-keytab=/etc/security/keytabs/hdfs.keytab Set super user keytab location " exit 1 } -# Parse script parameters -if [ $# != 2 ] ; then +OPTS=$(getopt \ + -n $0 \ + -o '' \ + -l 'kerberos-realm:' \ + -l 'super-user:' \ + -l 'super-user-keytab:' \ + -o 'h' \ + -o 'u' \ + -- "$@") + +if [ $? != 0 ] ; then usage exit 1 fi -while getopts "hu:" OPTION -do - case $OPTION in - u) - SETUP_USER=$2; shift 2 - ;; - h) +create_user() { + if [ "${SETUP_USER}" = "" ]; then + break + fi + HADOOP_HDFS_USER=${HADOOP_HDFS_USER:-hdfs} + export HADOOP_PREFIX + export HADOOP_CONF_DIR + export JAVA_HOME + export SETUP_USER=${SETUP_USER} + export SETUP_PATH=/user/${SETUP_USER} + + if [ ! "${KERBEROS_REALM}" = "" ]; then + # locate kinit cmd + if [ -e /etc/lsb-release ]; then + KINIT_CMD="/usr/bin/kinit -kt ${HDFS_USER_KEYTAB} ${HADOOP_HDFS_USER}" + else + KINIT_CMD="/usr/kerberos/bin/kinit -kt ${HDFS_USER_KEYTAB} ${HADOOP_HDFS_USER}" + fi + su -c "${KINIT_CMD}" ${HADOOP_HDFS_USER} + fi + + su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -mkdir ${SETUP_PATH}" ${HADOOP_HDFS_USER} + su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chown ${SETUP_USER}:${SETUP_USER} ${SETUP_PATH}" ${HADOOP_HDFS_USER} + su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chmod 711 ${SETUP_PATH}" ${HADOOP_HDFS_USER} + + if [ "$?" == "0" ]; then + echo "User directory has been setup: ${SETUP_PATH}" + fi +} + +eval set -- "${OPTS}" +while true; do + case "$1" in + -u) + shift + ;; + --kerberos-realm) + KERBEROS_REALM=$2; shift 2 + ;; + --super-user) + HADOOP_HDFS_USER=$2; shift 2 + ;; + --super-user-keytab) + HDFS_USER_KEYTAB=$2; shift 2 + ;; + -h) usage ;; --) - shift ; break + while shift; do + SETUP_USER=$1 + create_user + done + break ;; *) echo "Unknown option: $1" @@ -62,15 +119,3 @@ do esac done -# Create user directory on HDFS -export SETUP_USER -export SETUP_PATH=/user/${SETUP_USER} -export HADOOP_PREFIX -export HADOOP_CONF_DIR - -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -mkdir ${SETUP_PATH}' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chown ${SETUP_USER}:${SETUP_USER} ${SETUP_PATH}' hdfs - -if [ "$?" == "0" ]; then - echo "User directory has been setup: ${SETUP_PATH}" -fi diff --git a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh index eb72b86b2e..64a8d57db8 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh +++ b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh @@ -18,35 +18,60 @@ bin=`dirname "$0"` bin=`cd "$bin"; pwd` -if [ "$HADOOP_HOME" != "" ]; then - echo "Warning: \$HADOOP_HOME is deprecated." - echo -fi - -. "$bin"/../libexec/hadoop-config.sh +this="${BASH_SOURCE-$0}" +export HADOOP_PREFIX=`dirname "$this"`/.. usage() { echo " usage: $0 Optional parameters: - --auto Setup automatically - --default Generate default config - --conf-dir=/etc/hadoop Set config directory + --auto Setup path and configuration automatically + --default Setup configuration as default + --conf-dir=/etc/hadoop Set configuration directory --datanode-dir=/var/lib/hadoop/hdfs/datanode Set datanode directory + --group=hadoop Set Hadoop group name -h Display this message - --jobtracker-url=hostname:9001 Set jobtracker url + --hdfs-user=hdfs Set HDFS user + --jobtracker-host=hostname Set jobtracker host + --namenode-host=hostname Set namenode host + --secondarynamenode-host=hostname Set secondary namenode host + --kerberos-realm=KERBEROS.EXAMPLE.COM Set Kerberos realm + --kinit-location=/usr/kerberos/bin/kinit Set kinit location + --keytab-dir=/etc/security/keytabs Set keytab directory --log-dir=/var/log/hadoop Set log directory - --hdfs-dir=/var/lib/hadoop/hdfs Set hdfs directory + --pid-dir=/var/run/hadoop Set pid directory + --hdfs-dir=/var/lib/hadoop/hdfs Set HDFS directory + --hdfs-user-keytab=/home/hdfs/hdfs.keytab Set HDFS user key tab --mapred-dir=/var/lib/hadoop/mapred Set mapreduce directory + --mapreduce-user=mr Set mapreduce user + --mapreduce-user-keytab=/home/mr/hdfs.keytab Set mapreduce user key tab --namenode-dir=/var/lib/hadoop/hdfs/namenode Set namenode directory - --namenode-url=hdfs://hostname:9000/ Set namenode url --replication=3 Set replication factor --taskscheduler=org.apache.hadoop.mapred.JobQueueTaskScheduler Set task scheduler + --datanodes=hostname1,hostname2,... SET the datanodes + --tasktrackers=hostname1,hostname2,... SET the tasktrackers " exit 1 } +check_permission() { + TARGET=$1 + OWNER="0" + RESULT=0 + while [ "$TARGET" != "/" ]; do + PARENT=`dirname $TARGET` + NAME=`basename $TARGET` + OWNER=`ls -ln $PARENT | grep $NAME| awk '{print $3}'` + if [ "$OWNER" != "0" ]; then + RESULT=1 + break + fi + TARGET=`dirname $TARGET` + done + return $RESULT +} + template_generator() { REGEX='(\$\{[a-zA-Z_][a-zA-Z_0-9]*\})' cat $1 | @@ -64,17 +89,30 @@ OPTS=$(getopt \ -n $0 \ -o '' \ -l 'auto' \ + -l 'java-home:' \ -l 'conf-dir:' \ -l 'default' \ + -l 'group:' \ -l 'hdfs-dir:' \ -l 'namenode-dir:' \ -l 'datanode-dir:' \ -l 'mapred-dir:' \ - -l 'namenode-url:' \ - -l 'jobtracker-url:' \ + -l 'namenode-host:' \ + -l 'secondarynamenode-host:' \ + -l 'jobtracker-host:' \ -l 'log-dir:' \ + -l 'pid-dir:' \ -l 'replication:' \ -l 'taskscheduler:' \ + -l 'hdfs-user:' \ + -l 'hdfs-user-keytab:' \ + -l 'mapreduce-user:' \ + -l 'mapreduce-user-keytab:' \ + -l 'keytab-dir:' \ + -l 'kerberos-realm:' \ + -l 'kinit-location:' \ + -l 'datanodes:' \ + -l 'tasktrackers:' \ -o 'h' \ -- "$@") @@ -84,8 +122,7 @@ fi # Make sure the HADOOP_LOG_DIR is not picked up from user environment. unset HADOOP_LOG_DIR - -# Parse script parameters + eval set -- "${OPTS}" while true ; do case "$1" in @@ -94,6 +131,10 @@ while true ; do AUTOMATED=1 shift ;; + --java-home) + JAVA_HOME=$2; shift 2 + AUTOMATED=1 + ;; --conf-dir) HADOOP_CONF_DIR=$2; shift 2 AUTOMATED=1 @@ -101,6 +142,10 @@ while true ; do --default) AUTOMATED=1; shift ;; + --group) + HADOOP_GROUP=$2; shift 2 + AUTOMATED=1 + ;; -h) usage ;; @@ -120,11 +165,15 @@ while true ; do HADOOP_MAPRED_DIR=$2; shift 2 AUTOMATED=1 ;; - --namenode-url) + --namenode-host) HADOOP_NN_HOST=$2; shift 2 AUTOMATED=1 ;; - --jobtracker-url) + --secondarynamenode-host) + HADOOP_SNN_HOST=$2; shift 2 + AUTOMATED=1 + ;; + --jobtracker-host) HADOOP_JT_HOST=$2; shift 2 AUTOMATED=1 ;; @@ -132,6 +181,10 @@ while true ; do HADOOP_LOG_DIR=$2; shift 2 AUTOMATED=1 ;; + --pid-dir) + HADOOP_PID_DIR=$2; shift 2 + AUTOMATED=1 + ;; --replication) HADOOP_REPLICATION=$2; shift 2 AUTOMATED=1 @@ -139,7 +192,46 @@ while true ; do --taskscheduler) HADOOP_TASK_SCHEDULER=$2; shift 2 AUTOMATED=1 - ;; + ;; + --hdfs-user) + HADOOP_HDFS_USER=$2; shift 2 + AUTOMATED=1 + ;; + --mapreduce-user) + HADOOP_MR_USER=$2; shift 2 + AUTOMATED=1 + ;; + --keytab-dir) + KEYTAB_DIR=$2; shift 2 + AUTOMATED=1 + ;; + --hdfs-user-keytab) + HDFS_KEYTAB=$2; shift 2 + AUTOMATED=1 + ;; + --mapreduce-user-keytab) + MR_KEYTAB=$2; shift 2 + AUTOMATED=1 + ;; + --kerberos-realm) + KERBEROS_REALM=$2; shift 2 + SECURITY_TYPE="kerberos" + AUTOMATED=1 + ;; + --kinit-location) + KINIT=$2; shift 2 + AUTOMATED=1 + ;; + --datanodes) + DATANODES=$2; shift 2 + AUTOMATED=1 + DATANODES=$(echo $DATANODES | tr ',' ' ') + ;; + --tasktrackers) + TASKTRACKERS=$2; shift 2 + AUTOMATED=1 + TASKTRACKERS=$(echo $TASKTRACKERS | tr ',' ' ') + ;; --) shift ; break ;; @@ -151,21 +243,40 @@ while true ; do esac done -# Fill in default values, if parameters have not been defined. AUTOSETUP=${AUTOSETUP:-1} JAVA_HOME=${JAVA_HOME:-/usr/java/default} -HADOOP_NN_HOST=${HADOOP_NN_HOST:-hdfs://`hostname`:9000/} +HADOOP_GROUP=${HADOOP_GROUP:-hadoop} +HADOOP_NN_HOST=${HADOOP_NN_HOST:-`hostname`} HADOOP_NN_DIR=${HADOOP_NN_DIR:-/var/lib/hadoop/hdfs/namenode} HADOOP_DN_DIR=${HADOOP_DN_DIR:-/var/lib/hadoop/hdfs/datanode} -HADOOP_JT_HOST=${HADOOP_JT_HOST:-`hostname`:9001} +HADOOP_JT_HOST=${HADOOP_JT_HOST:-`hostname`} HADOOP_HDFS_DIR=${HADOOP_HDFS_DIR:-/var/lib/hadoop/hdfs} HADOOP_MAPRED_DIR=${HADOOP_MAPRED_DIR:-/var/lib/hadoop/mapred} HADOOP_LOG_DIR=${HADOOP_LOG_DIR:-/var/log/hadoop} +HADOOP_PID_DIR=${HADOOP_PID_DIR:-/var/log/hadoop} HADOOP_CONF_DIR=${HADOOP_CONF_DIR:-/etc/hadoop} HADOOP_REPLICATION=${HADOOP_RELICATION:-3} HADOOP_TASK_SCHEDULER=${HADOOP_TASK_SCHEDULER:-org.apache.hadoop.mapred.JobQueueTaskScheduler} +HADOOP_HDFS_USER=${HADOOP_HDFS_USER:-hdfs} +HADOOP_MR_USER=${HADOOP_MR_USER:-mr} +KEYTAB_DIR=${KEYTAB_DIR:-/etc/security/keytabs} +HDFS_KEYTAB=${HDFS_KEYTAB:-/home/hdfs/hdfs.keytab} +MR_KEYTAB=${MR_KEYTAB:-/home/mr/mr.keytab} +KERBEROS_REALM=${KERBEROS_REALM:-KERBEROS.EXAMPLE.COM} +SECURITY_TYPE=${SECURITY_TYPE:-simple} +KINIT=${KINIT:-/usr/kerberos/bin/kinit} +if [ "${SECURITY_TYPE}" = "kerberos" ]; then + TASK_CONTROLLER="org.apache.hadoop.mapred.LinuxTaskController" + HADOOP_DN_ADDR="0.0.0.0:1019" + HADOOP_DN_HTTP_ADDR="0.0.0.0:1022" + SECURITY="true" +else + TASK_CONTROLLER="org.apache.hadoop.mapred.DefaultTaskController" + HADDOP_DN_ADDR="0.0.0.0:50010" + HADOOP_DN_HTTP_ADDR="0.0.0.0:50075" + SECURITY="false" +fi -# Interactive setup wizard if [ "${AUTOMATED}" != "1" ]; then echo "Setup Hadoop Configuration" echo @@ -173,18 +284,18 @@ if [ "${AUTOMATED}" != "1" ]; then read USER_HADOOP_CONF_DIR echo -n "Where would you like to put log directory? (${HADOOP_LOG_DIR}) " read USER_HADOOP_LOG_DIR - echo -n "What is the url of the namenode? (${HADOOP_NN_HOST}) " + echo -n "Where would you like to put pid directory? (${HADOOP_PID_DIR}) " + read USER_HADOOP_PID_DIR + echo -n "What is the host of the namenode? (${HADOOP_NN_HOST}) " read USER_HADOOP_NN_HOST echo -n "Where would you like to put namenode data directory? (${HADOOP_NN_DIR}) " read USER_HADOOP_NN_DIR echo -n "Where would you like to put datanode data directory? (${HADOOP_DN_DIR}) " read USER_HADOOP_DN_DIR - echo -n "What is the url of the jobtracker? (${HADOOP_JT_HOST}) " + echo -n "What is the host of the jobtracker? (${HADOOP_JT_HOST}) " read USER_HADOOP_JT_HOST echo -n "Where would you like to put jobtracker/tasktracker data directory? (${HADOOP_MAPRED_DIR}) " read USER_HADOOP_MAPRED_DIR - echo -n "Which taskscheduler would you like? (${HADOOP_TASK_SCHEDULER}) " - read USER_HADOOP_TASK_SCHEDULER echo -n "Where is JAVA_HOME directory? (${JAVA_HOME}) " read USER_JAVA_HOME echo -n "Would you like to create directories/copy conf files to localhost? (Y/n) " @@ -199,16 +310,18 @@ if [ "${AUTOMATED}" != "1" ]; then HADOOP_MAPRED_DIR=${USER_HADOOP_MAPRED_DIR:-$HADOOP_MAPRED_DIR} HADOOP_TASK_SCHEDULER=${HADOOP_TASK_SCHEDULER:-org.apache.hadoop.mapred.JobQueueTaskScheduler} HADOOP_LOG_DIR=${USER_HADOOP_LOG_DIR:-$HADOOP_LOG_DIR} + HADOOP_PID_DIR=${USER_HADOOP_PID_DIR:-$HADOOP_PID_DIR} HADOOP_CONF_DIR=${USER_HADOOP_CONF_DIR:-$HADOOP_CONF_DIR} AUTOSETUP=${USER_AUTOSETUP:-y} echo "Review your choices:" echo echo "Config directory : ${HADOOP_CONF_DIR}" echo "Log directory : ${HADOOP_LOG_DIR}" - echo "Namenode url : ${HADOOP_NN_HOST}" + echo "PID directory : ${HADOOP_PID_DIR}" + echo "Namenode host : ${HADOOP_NN_HOST}" echo "Namenode directory : ${HADOOP_NN_DIR}" echo "Datanode directory : ${HADOOP_DN_DIR}" - echo "Jobtracker url : ${HADOOP_JT_HOST}" + echo "Jobtracker host : ${HADOOP_JT_HOST}" echo "Mapreduce directory : ${HADOOP_MAPRED_DIR}" echo "Task scheduler : ${HADOOP_TASK_SCHEDULER}" echo "JAVA_HOME directory : ${JAVA_HOME}" @@ -222,61 +335,180 @@ if [ "${AUTOMATED}" != "1" ]; then fi fi -if [ "${AUTOSETUP}" == "1" ]; then - # If user wants to setup local system automatically, - # set config file generation location to HADOOP_CONF_DIR. - DEST=${HADOOP_CONF_DIR} -else - # If user is only interested to generate config file locally, - # place config files in the current working directory. - DEST=`pwd` -fi - -# remove existing config file, they are existed in current directory. -rm -f ${DEST}/core-site.xml >/dev/null -rm -f ${DEST}/hdfs-site.xml >/dev/null -rm -f ${DEST}/mapred-site.xml >/dev/null -rm -f ${DEST}/hadoop-env.sh >/dev/null - -# Generate config file with specified parameters. -template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/core-site.xml ${DEST}/core-site.xml -template_generator ${HADOOP_PREFIX}/share/hadoop/hdfs/templates/hdfs-site.xml ${DEST}/hdfs-site.xml -template_generator ${HADOOP_PREFIX}/share/hadoop/mapreduce/templates/mapred-site.xml ${DEST}/mapred-site.xml -template_generator ${HADOOP_CONF_DIR}/hadoop-env.sh.template ${DEST}/hadoop-env.sh - -chown root:hadoop ${DEST}/hadoop-env.sh -chmod 755 ${DEST}/hadoop-env.sh - -# Setup directory path and copy config files, if AUTOSETUP is chosen. if [ "${AUTOSETUP}" == "1" -o "${AUTOSETUP}" == "y" ]; then - mkdir -p ${HADOOP_HDFS_DIR} - mkdir -p ${HADOOP_NN_DIR} - mkdir -p ${HADOOP_DN_DIR} - mkdir -p ${HADOOP_MAPRED_DIR} + if [ -d ${KEYTAB_DIR} ]; then + chmod 700 ${KEYTAB_DIR}/* + chown ${HADOOP_MR_USER}:${HADOOP_GROUP} ${KEYTAB_DIR}/[jt]t.service.keytab + chown ${HADOOP_HDFS_USER}:${HADOOP_GROUP} ${KEYTAB_DIR}/[dns]n.service.keytab + fi + chmod 755 -R ${HADOOP_PREFIX}/sbin/*hadoop* + chmod 755 -R ${HADOOP_PREFIX}/bin/hadoop + chmod 755 -R ${HADOOP_PREFIX}/libexec/hadoop-config.sh + mkdir -p /home/${HADOOP_MR_USER} + chown ${HADOOP_MR_USER}:${HADOOP_GROUP} /home/${HADOOP_MR_USER} + HDFS_DIR=`echo ${HADOOP_HDFS_DIR} | sed -e 's/,/ /g'` + mkdir -p ${HDFS_DIR} + if [ -e ${HADOOP_NN_DIR} ]; then + rm -rf ${HADOOP_NN_DIR} + fi + DATANODE_DIR=`echo ${HADOOP_DN_DIR} | sed -e 's/,/ /g'` + mkdir -p ${DATANODE_DIR} + MAPRED_DIR=`echo ${HADOOP_MAPRED_DIR} | sed -e 's/,/ /g'` + mkdir -p ${MAPRED_DIR} mkdir -p ${HADOOP_CONF_DIR} + check_permission ${HADOOP_CONF_DIR} + if [ $? == 1 ]; then + echo "Full path to ${HADOOP_CONF_DIR} should be owned by root." + exit 1 + fi + mkdir -p ${HADOOP_LOG_DIR} - mkdir -p ${HADOOP_LOG_DIR}/hdfs - mkdir -p ${HADOOP_LOG_DIR}/mapred - chown hdfs:hadoop ${HADOOP_HDFS_DIR} - chown hdfs:hadoop ${HADOOP_NN_DIR} - chown hdfs:hadoop ${HADOOP_DN_DIR} - chown mapred:hadoop ${HADOOP_MAPRED_DIR} - chown root:hadoop ${HADOOP_LOG_DIR} + #create the log sub dir for diff users + mkdir -p ${HADOOP_LOG_DIR}/${HADOOP_HDFS_USER} + mkdir -p ${HADOOP_LOG_DIR}/${HADOOP_MR_USER} + + mkdir -p ${HADOOP_PID_DIR} + chown ${HADOOP_HDFS_USER}:${HADOOP_GROUP} ${HDFS_DIR} + chown ${HADOOP_HDFS_USER}:${HADOOP_GROUP} ${DATANODE_DIR} + chmod 700 -R ${DATANODE_DIR} + chown ${HADOOP_MR_USER}:${HADOOP_GROUP} ${MAPRED_DIR} + chown ${HADOOP_HDFS_USER}:${HADOOP_GROUP} ${HADOOP_LOG_DIR} chmod 775 ${HADOOP_LOG_DIR} - chown hdfs:hadoop ${HADOOP_LOG_DIR}/hdfs - chown mapred:hadoop ${HADOOP_LOG_DIR}/mapred + chmod 775 ${HADOOP_PID_DIR} + chown root:${HADOOP_GROUP} ${HADOOP_PID_DIR} + + #change the permission and the owner + chmod 755 ${HADOOP_LOG_DIR}/${HADOOP_HDFS_USER} + chown ${HADOOP_HDFS_USER}:${HADOOP_GROUP} ${HADOOP_LOG_DIR}/${HADOOP_HDFS_USER} + chmod 755 ${HADOOP_LOG_DIR}/${HADOOP_MR_USER} + chown ${HADOOP_MR_USER}:${HADOOP_GROUP} ${HADOOP_LOG_DIR}/${HADOOP_MR_USER} + + if [ -e ${HADOOP_CONF_DIR}/core-site.xml ]; then + mv -f ${HADOOP_CONF_DIR}/core-site.xml ${HADOOP_CONF_DIR}/core-site.xml.bak + fi + if [ -e ${HADOOP_CONF_DIR}/hdfs-site.xml ]; then + mv -f ${HADOOP_CONF_DIR}/hdfs-site.xml ${HADOOP_CONF_DIR}/hdfs-site.xml.bak + fi + if [ -e ${HADOOP_CONF_DIR}/mapred-site.xml ]; then + mv -f ${HADOOP_CONF_DIR}/mapred-site.xml ${HADOOP_CONF_DIR}/mapred-site.xml.bak + fi + if [ -e ${HADOOP_CONF_DIR}/hadoop-env.sh ]; then + mv -f ${HADOOP_CONF_DIR}/hadoop-env.sh ${HADOOP_CONF_DIR}/hadoop-env.sh.bak + fi + if [ -e ${HADOOP_CONF_DIR}/hadoop-policy.xml ]; then + mv -f ${HADOOP_CONF_DIR}/hadoop-policy.xml ${HADOOP_CONF_DIR}/hadoop-policy.xml.bak + fi + if [ -e ${HADOOP_CONF_DIR}/mapred-queue-acls.xml ]; then + mv -f ${HADOOP_CONF_DIR}/mapred-queue-acls.xml ${HADOOP_CONF_DIR}/mapred-queue-acls.xml.bak + fi + if [ -e ${HADOOP_CONF_DIR}/commons-logging.properties ]; then + mv -f ${HADOOP_CONF_DIR}/commons-logging.properties ${HADOOP_CONF_DIR}/commons-logging.properties.bak + fi + if [ -e ${HADOOP_CONF_DIR}/taskcontroller.cfg ]; then + mv -f ${HADOOP_CONF_DIR}/taskcontroller.cfg ${HADOOP_CONF_DIR}/taskcontroller.cfg.bak + fi + if [ -e ${HADOOP_CONF_DIR}/slaves ]; then + mv -f ${HADOOP_CONF_DIR}/slaves ${HADOOP_CONF_DIR}/slaves.bak + fi + if [ -e ${HADOOP_CONF_DIR}/dfs.include ]; then + mv -f ${HADOOP_CONF_DIR}/dfs.include ${HADOOP_CONF_DIR}/dfs.include.bak + fi + if [ -e ${HADOOP_CONF_DIR}/dfs.exclude ]; then + mv -f ${HADOOP_CONF_DIR}/dfs.exclude ${HADOOP_CONF_DIR}/dfs.exclude.bak + fi + if [ -e ${HADOOP_CONF_DIR}/mapred.include ]; then + mv -f ${HADOOP_CONF_DIR}/mapred.include ${HADOOP_CONF_DIR}/mapred.include.bak + fi + if [ -e ${HADOOP_CONF_DIR}/mapred.exclude ]; then + mv -f ${HADOOP_CONF_DIR}/mapred.exclude ${HADOOP_CONF_DIR}/mapred.exclude.bak + fi + + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/core-site.xml ${HADOOP_CONF_DIR}/core-site.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hdfs-site.xml ${HADOOP_CONF_DIR}/hdfs-site.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/mapred-site.xml ${HADOOP_CONF_DIR}/mapred-site.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-env.sh ${HADOOP_CONF_DIR}/hadoop-env.sh + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-policy.xml ${HADOOP_CONF_DIR}/hadoop-policy.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/commons-logging.properties ${HADOOP_CONF_DIR}/commons-logging.properties + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/mapred-queue-acls.xml ${HADOOP_CONF_DIR}/mapred-queue-acls.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/taskcontroller.cfg ${HADOOP_CONF_DIR}/taskcontroller.cfg + + #set the owner of the hadoop dir to root + chown root ${HADOOP_PREFIX} + chown root:${HADOOP_GROUP} ${HADOOP_CONF_DIR}/hadoop-env.sh + chmod 755 ${HADOOP_CONF_DIR}/hadoop-env.sh + + #set taskcontroller + chown root:${HADOOP_GROUP} ${HADOOP_CONF_DIR}/taskcontroller.cfg + chmod 400 ${HADOOP_CONF_DIR}/taskcontroller.cfg + chown root:${HADOOP_GROUP} ${HADOOP_PREFIX}/bin/task-controller + chmod 6050 ${HADOOP_PREFIX}/bin/task-controller + + + #generate the slaves file and include and exclude files for hdfs and mapred + echo '' > ${HADOOP_CONF_DIR}/slaves + echo '' > ${HADOOP_CONF_DIR}/dfs.include + echo '' > ${HADOOP_CONF_DIR}/dfs.exclude + echo '' > ${HADOOP_CONF_DIR}/mapred.include + echo '' > ${HADOOP_CONF_DIR}/mapred.exclude + for dn in $DATANODES + do + echo $dn >> ${HADOOP_CONF_DIR}/slaves + echo $dn >> ${HADOOP_CONF_DIR}/dfs.include + done + for tt in $TASKTRACKERS + do + echo $tt >> ${HADOOP_CONF_DIR}/mapred.include + done + echo "Configuration setup is completed." if [[ "$HADOOP_NN_HOST" =~ "`hostname`" ]]; then echo "Proceed to run hadoop-setup-hdfs.sh on namenode." fi else + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/core-site.xml ${HADOOP_CONF_DIR}/core-site.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hdfs-site.xml ${HADOOP_CONF_DIR}/hdfs-site.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/mapred-site.xml ${HADOOP_CONF_DIR}/mapred-site.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-env.sh ${HADOOP_CONF_DIR}/hadoop-env.sh + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-policy.xml ${HADOOP_CONF_DIR}/hadoop-policy.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/commons-logging.properties ${HADOOP_CONF_DIR}/commons-logging.properties + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/mapred-queue-acls.xml ${HADOOP_CONF_DIR}/mapred-queue-acls.xml + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/taskcontroller.cfg ${HADOOP_CONF_DIR}/taskcontroller.cfg + template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-metrics2.properties ${HADOOP_CONF_DIR}/hadoop-metrics2.properties + if [ ! -e ${HADOOP_CONF_DIR}/capacity-scheduler.xml ]; then + template_generator ${HADOOP_PREFIX}/share/hadoop/templates/conf/capacity-scheduler.xml ${HADOOP_CONF_DIR}/capacity-scheduler.xml + fi + + chown root:${HADOOP_GROUP} ${HADOOP_CONF_DIR}/hadoop-env.sh + chmod 755 ${HADOOP_CONF_DIR}/hadoop-env.sh + #set taskcontroller + chown root:${HADOOP_GROUP} ${HADOOP_CONF_DIR}/taskcontroller.cfg + chmod 400 ${HADOOP_CONF_DIR}/taskcontroller.cfg + chown root:${HADOOP_GROUP} ${HADOOP_PREFIX}/bin/task-controller + chmod 6050 ${HADOOP_PREFIX}/bin/task-controller + + #generate the slaves file and include and exclude files for hdfs and mapred + echo '' > ${HADOOP_CONF_DIR}/slaves + echo '' > ${HADOOP_CONF_DIR}/dfs.include + echo '' > ${HADOOP_CONF_DIR}/dfs.exclude + echo '' > ${HADOOP_CONF_DIR}/mapred.include + echo '' > ${HADOOP_CONF_DIR}/mapred.exclude + for dn in $DATANODES + do + echo $dn >> ${HADOOP_CONF_DIR}/slaves + echo $dn >> ${HADOOP_CONF_DIR}/dfs.include + done + for tt in $TASKTRACKERS + do + echo $tt >> ${HADOOP_CONF_DIR}/mapred.include + done + echo - echo "Configuration file has been generated, please copy:" + echo "Configuration file has been generated in:" echo - echo "core-site.xml" - echo "hdfs-site.xml" - echo "mapred-site.xml" - echo "hadoop-env.sh" + echo "${HADOOP_CONF_DIR}/core-site.xml" + echo "${HADOOP_CONF_DIR}/hdfs-site.xml" + echo "${HADOOP_CONF_DIR}/mapred-site.xml" + echo "${HADOOP_CONF_DIR}/hadoop-env.sh" echo echo " to ${HADOOP_CONF_DIR} on all nodes, and proceed to run hadoop-setup-hdfs.sh on namenode." fi diff --git a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-hdfs.sh b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-hdfs.sh index 8ac6f312c5..cd99463b37 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-hdfs.sh +++ b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-hdfs.sh @@ -18,37 +18,65 @@ bin=`dirname "$0"` bin=`cd "$bin"; pwd` -if [ "$HADOOP_HOME" != "" ]; then - echo "Warning: \$HADOOP_HOME is deprecated." - echo -fi - . "$bin"/../libexec/hadoop-config.sh usage() { echo " usage: $0 - Require parameter: - -c Set cluster identifier for HDFS + Optional parameters: - -h Display this message + --format Force namenode format + --group=hadoop Set Hadoop group + -h Display this message + --hdfs-user=hdfs Set HDFS user + --kerberos-realm=KERBEROS.EXAMPLE.COM Set Kerberos realm + --hdfs-user-keytab=/home/hdfs/hdfs.keytab Set HDFS user key tab + --mapreduce-user=mr Set mapreduce user " exit 1 } -if [ $# != 2 ] ; then +OPTS=$(getopt \ + -n $0 \ + -o '' \ + -l 'format' \ + -l 'hdfs-user:' \ + -l 'hdfs-user-keytab:' \ + -l 'mapreduce-user:' \ + -l 'kerberos-realm:' \ + -o 'h' \ + -- "$@") + +if [ $? != 0 ] ; then usage - exit 1 fi -while getopts "hc:" OPTION -do - case $OPTION in - c) - SETUP_CLUSTER=$2; shift 2 +eval set -- "${OPTS}" +while true ; do + case "$1" in + --format) + FORMAT_NAMENODE=1; shift + AUTOMATED=1 ;; - h) - usage + --group) + HADOOP_GROUP=$2; shift 2 + AUTOMATED=1 + ;; + --hdfs-user) + HADOOP_HDFS_USER=$2; shift 2 + AUTOMATED=1 + ;; + --mapreduce-user) + HADOOP_MR_USER=$2; shift 2 + AUTOMATED=1 + ;; + --hdfs-user-keytab) + HDFS_KEYTAB=$2; shift 2 + AUTOMATED=1 + ;; + --kerberos-realm) + KERBEROS_REALM=$2; shift 2 + AUTOMATED=1 ;; --) shift ; break @@ -61,30 +89,56 @@ do esac done -export HADOOP_PREFIX -export HADOOP_CONF_DIR -export SETUP_CLUSTER +HADOOP_GROUP=${HADOOP_GROUP:-hadoop} +HADOOP_HDFS_USER=${HADOOP_HDFS_USER:-hdfs} +HADOOP_MAPREDUCE_USER=${HADOOP_MR_USER:-mapred} + +if [ "${KERBEROS_REALM}" != "" ]; then + # Determine kerberos location base on Linux distro. + if [ -e /etc/lsb-release ]; then + KERBEROS_BIN=/usr/bin + else + KERBEROS_BIN=/usr/kerberos/bin + fi + kinit_cmd="${KERBEROS_BIN}/kinit -k -t ${HDFS_KEYTAB} ${HADOOP_HDFS_USER}" + su -c "${kinit_cmd}" ${HADOOP_HDFS_USER} +fi -# Start namenode and initialize file system structure echo "Setup Hadoop Distributed File System" echo -echo "Formatting namenode" -echo -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} namenode -format -clusterid ${SETUP_CLUSTER}' hdfs -echo + +# Format namenode +if [ "${FORMAT_NAMENODE}" == "1" ]; then + echo "Formatting namenode" + echo + su -c "echo Y | ${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} namenode -format" ${HADOOP_HDFS_USER} + echo +fi + +# Start namenode process echo "Starting namenode process" echo -/etc/init.d/hadoop-namenode start +if [ -e ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh ]; then + DAEMON_PATH=${HADOOP_PREFIX}/sbin +else + DAEMON_PATH=${HADOOP_PREFIX}/bin +fi +su -c "${DAEMON_PATH}/hadoop-daemon.sh --config ${HADOOP_CONF_DIR} start namenode" ${HADOOP_HDFS_USER} echo echo "Initialize HDFS file system: " echo -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -mkdir /jobtracker' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chown mapred:mapred /jobtracker' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -mkdir /user/mapred' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chown mapred:mapred /user/mapred' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -mkdir /tmp' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chmod 777 /tmp' hdfs +#create the /user dir +su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -mkdir /user" ${HADOOP_HDFS_USER} + +#create /tmp and give it 777 +su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -mkdir /tmp" ${HADOOP_HDFS_USER} +su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -chmod 777 /tmp" ${HADOOP_HDFS_USER} + +#create /mapred +su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -mkdir /mapred" ${HADOOP_HDFS_USER} +su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -chmod 700 /mapred" ${HADOOP_HDFS_USER} +su -c "${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -chown ${HADOOP_MAPREDUCE_USER}:system /mapred" ${HADOOP_HDFS_USER} if [ $? -eq 0 ]; then echo "Completed." diff --git a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-single-node.sh b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-single-node.sh index 1ab40b75fc..5b47e4431c 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-single-node.sh +++ b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-single-node.sh @@ -17,16 +17,16 @@ # Script for setup HDFS file system for single node deployment -bin=`which $0` -bin=`dirname ${bin}` +bin=`dirname "$0"` bin=`cd "$bin"; pwd` -export HADOOP_PREFIX=${bin}/.. - -if [ -e /etc/hadoop/hadoop-env.sh ]; then - . /etc/hadoop/hadoop-env.sh +if [ "$HADOOP_HOME" != "" ]; then + echo "Warning: \$HADOOP_HOME is deprecated." + echo fi +. "$bin"/../libexec/hadoop-config.sh + usage() { echo " usage: $0 @@ -38,7 +38,19 @@ usage: $0 exit 1 } -# Parse script parameters +template_generator() { + REGEX='(\$\{[a-zA-Z_][a-zA-Z_0-9]*\})' + cat $1 | + while read line ; do + while [[ "$line" =~ $REGEX ]] ; do + LHS=${BASH_REMATCH[1]} + RHS="$(eval echo "\"$LHS\"")" + line=${line//$LHS/$RHS} + done + echo $line >> $2 + done +} + OPTS=$(getopt \ -n $0 \ -o '' \ @@ -49,6 +61,10 @@ if [ $? != 0 ] ; then usage fi +if [ -e /etc/hadoop/hadoop-env.sh ]; then + . /etc/hadoop/hadoop-env.sh +fi + eval set -- "${OPTS}" while true ; do case "$1" in @@ -69,7 +85,6 @@ while true ; do esac done -# Interactive setup wizard if [ "${AUTOMATED}" != "1" ]; then echo "Welcome to Hadoop single node setup wizard" echo @@ -119,68 +134,59 @@ SET_REBOOT=${SET_REBOOT:-y} /etc/init.d/hadoop-jobtracker stop 2>/dev/null >/dev/null /etc/init.d/hadoop-tasktracker stop 2>/dev/null >/dev/null -# Default settings -JAVA_HOME=${JAVA_HOME:-/usr/java/default} -HADOOP_NN_HOST=${HADOOP_NN_HOST:-hdfs://localhost:9000/} -HADOOP_NN_DIR=${HADOOP_NN_DIR:-/var/lib/hadoop/hdfs/namenode} -HADOOP_DN_DIR=${HADOOP_DN_DIR:-/var/lib/hadoop/hdfs/datanode} -HADOOP_JT_HOST=${HADOOP_JT_HOST:-localhost:9001} -HADOOP_HDFS_DIR=${HADOOP_MAPRED_DIR:-/var/lib/hadoop/hdfs} -HADOOP_MAPRED_DIR=${HADOOP_MAPRED_DIR:-/var/lib/hadoop/mapred} -HADOOP_LOG_DIR="/var/log/hadoop" -HADOOP_CONF_DIR=${HADOOP_CONF_DIR:-/etc/hadoop} -HADOOP_REPLICATION=${HADOOP_RELICATION:-1} -HADOOP_TASK_SCHEDULER=${HADOOP_TASK_SCHEDULER:-org.apache.hadoop.mapred.JobQueueTaskScheduler} - -# Setup config files if [ "${SET_CONFIG}" == "y" ]; then + JAVA_HOME=${JAVA_HOME:-/usr/java/default} + HADOOP_NN_HOST=${HADOOP_NN_HOST:-localhost} + HADOOP_NN_DIR=${HADOOP_NN_DIR:-/var/lib/hadoop/hdfs/namenode} + HADOOP_DN_DIR=${HADOOP_DN_DIR:-/var/lib/hadoop/hdfs/datanode} + HADOOP_JT_HOST=${HADOOP_JT_HOST:-localhost} + HADOOP_HDFS_DIR=${HADOOP_MAPRED_DIR:-/var/lib/hadoop/hdfs} + HADOOP_MAPRED_DIR=${HADOOP_MAPRED_DIR:-/var/lib/hadoop/mapred} + HADOOP_PID_DIR=${HADOOP_PID_DIR:-/var/run/hadoop} + HADOOP_LOG_DIR="/var/log/hadoop" + HADOOP_CONF_DIR=${HADOOP_CONF_DIR:-/etc/hadoop} + HADOOP_REPLICATION=${HADOOP_RELICATION:-1} ${HADOOP_PREFIX}/sbin/hadoop-setup-conf.sh --auto \ + --hdfs-user=hdfs \ + --mapreduce-user=mapred \ --conf-dir=${HADOOP_CONF_DIR} \ --datanode-dir=${HADOOP_DN_DIR} \ --hdfs-dir=${HADOOP_HDFS_DIR} \ - --jobtracker-url=${HADOOP_JT_HOST} \ + --jobtracker-host=${HADOOP_JT_HOST} \ --log-dir=${HADOOP_LOG_DIR} \ + --pid-dir=${HADOOP_PID_DIR} \ --mapred-dir=${HADOOP_MAPRED_DIR} \ --namenode-dir=${HADOOP_NN_DIR} \ - --namenode-url=${HADOOP_NN_HOST} \ + --namenode-host=${HADOOP_NN_HOST} \ --replication=${HADOOP_REPLICATION} fi -export HADOOP_CONF_DIR - -# Format namenode if [ ! -e ${HADOOP_NN_DIR} ]; then rm -rf ${HADOOP_HDFS_DIR} 2>/dev/null >/dev/null mkdir -p ${HADOOP_HDFS_DIR} chmod 755 ${HADOOP_HDFS_DIR} chown hdfs:hadoop ${HADOOP_HDFS_DIR} - su -c '${HADOOP_PREFIX}/bin/hdfs --config ${HADOOP_CONF_DIR} namenode -format -clusterid hadoop' hdfs + /etc/init.d/hadoop-namenode format elif [ "${SET_FORMAT}" == "y" ]; then rm -rf ${HADOOP_HDFS_DIR} 2>/dev/null >/dev/null mkdir -p ${HADOOP_HDFS_DIR} chmod 755 ${HADOOP_HDFS_DIR} chown hdfs:hadoop ${HADOOP_HDFS_DIR} - rm -rf /var/lib/hadoop/hdfs/namenode - su -c '${HADOOP_PREFIX}/bin/hdfs --config ${HADOOP_CONF_DIR} namenode -format -clusterid hadoop' hdfs + rm -rf ${HADOOP_NN_DIR} + /etc/init.d/hadoop-namenode format fi -# Start hdfs service /etc/init.d/hadoop-namenode start /etc/init.d/hadoop-datanode start -# Initialize file system structure -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -mkdir /user/mapred' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chown mapred:mapred /user/mapred' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -mkdir /tmp' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chmod 777 /tmp' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -mkdir /jobtracker' hdfs -su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} fs -chown mapred:mapred /jobtracker' hdfs +su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -mkdir /user/mapred' hdfs +su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -chown mapred:mapred /user/mapred' hdfs +su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -mkdir /tmp' hdfs +su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -chmod 777 /tmp' hdfs -# Start mapreduce service /etc/init.d/hadoop-jobtracker start /etc/init.d/hadoop-tasktracker start -# Toggle service startup on reboot if [ "${SET_REBOOT}" == "y" ]; then if [ -e /etc/debian_version ]; then ln -sf ../init.d/hadoop-namenode /etc/rc2.d/S90hadoop-namenode @@ -203,7 +209,6 @@ if [ "${SET_REBOOT}" == "y" ]; then fi fi -# Shutdown service, if user choose to stop services after setup if [ "${STARTUP}" != "y" ]; then /etc/init.d/hadoop-namenode stop /etc/init.d/hadoop-datanode stop diff --git a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-datanode b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-datanode index 019051b150..c324773f36 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-datanode +++ b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-datanode @@ -27,10 +27,15 @@ source /etc/default/hadoop-env.sh RETVAL=0 PIDFILE="${HADOOP_PID_DIR}/hadoop-hdfs-datanode.pid" desc="Hadoop datanode daemon" +HADOOP_PREFIX="/usr" start() { echo -n $"Starting $desc (hadoop-datanode): " - daemon --user hdfs ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh --config "${HADOOP_CONF_DIR}" start datanode + if [ -n "$HADOOP_SECURE_DN_USER" ]; then + daemon ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh --config "${HADOOP_CONF_DIR}" start datanode + else + daemon --user hdfs ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh --config "${HADOOP_CONF_DIR}" start datanode + fi RETVAL=$? echo [ $RETVAL -eq 0 ] && touch /var/lock/subsys/hadoop-datanode @@ -39,7 +44,11 @@ start() { stop() { echo -n $"Stopping $desc (hadoop-datanode): " - daemon --user hdfs ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh --config "${HADOOP_CONF_DIR}" stop datanode + if [ -n "$HADOOP_SECURE_DN_USER" ]; then + daemon ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh --config "${HADOOP_CONF_DIR}" stop datanode + else + daemon --user hdfs ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh --config "${HADOOP_CONF_DIR}" stop datanode + fi RETVAL=$? sleep 5 echo diff --git a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-jobtracker b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-jobtracker index 7aa6e28c7b..5c2272b8fd 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-jobtracker +++ b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-jobtracker @@ -27,6 +27,7 @@ source /etc/default/hadoop-env.sh RETVAL=0 PIDFILE="${HADOOP_PID_DIR}/hadoop-mapred-jobtracker.pid" desc="Hadoop jobtracker daemon" +export HADOOP_PREFIX="/usr" start() { echo -n $"Starting $desc (hadoop-jobtracker): " diff --git a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-namenode b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-namenode index dd0c94bf0f..4e7f74b046 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-namenode +++ b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-namenode @@ -27,6 +27,7 @@ source /etc/default/hadoop-env.sh RETVAL=0 PIDFILE="${HADOOP_PID_DIR}/hadoop-hdfs-namenode.pid" desc="Hadoop namenode daemon" +export HADOOP_PREFIX="/usr" start() { echo -n $"Starting $desc (hadoop-namenode): " diff --git a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-tasktracker b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-tasktracker index 303ed869fd..2d06c5a9c9 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-tasktracker +++ b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-tasktracker @@ -27,6 +27,7 @@ source /etc/default/hadoop-env.sh RETVAL=0 PIDFILE="${HADOOP_PID_DIR}/hadoop-mapred-tasktracker.pid" desc="Hadoop tasktracker daemon" +export HADOOP_PREFIX="/usr" start() { echo -n $"Starting $desc (hadoop-tasktracker): " diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/capacity-scheduler.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/capacity-scheduler.xml new file mode 100644 index 0000000000..1bf8e8fb7c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/capacity-scheduler.xml @@ -0,0 +1,178 @@ + + + + + + + + + + + mapred.capacity-scheduler.maximum-system-jobs + 3000 + Maximum number of jobs in the system which can be initialized, + concurrently, by the CapacityScheduler. + + + + + mapred.capacity-scheduler.queue.default.capacity + 100 + Percentage of the number of slots in the cluster that are + to be available for jobs in this queue. + + + + + mapred.capacity-scheduler.queue.default.maximum-capacity + -1 + + maximum-capacity defines a limit beyond which a queue cannot use the capacity of the cluster. + This provides a means to limit how much excess capacity a queue can use. By default, there is no limit. + The maximum-capacity of a queue can only be greater than or equal to its minimum capacity. + Default value of -1 implies a queue can use complete capacity of the cluster. + + This property could be to curtail certain jobs which are long running in nature from occupying more than a + certain percentage of the cluster, which in the absence of pre-emption, could lead to capacity guarantees of + other queues being affected. + + One important thing to note is that maximum-capacity is a percentage , so based on the cluster's capacity + the max capacity would change. So if large no of nodes or racks get added to the cluster , max Capacity in + absolute terms would increase accordingly. + + + + + mapred.capacity-scheduler.queue.default.supports-priority + false + If true, priorities of jobs will be taken into + account in scheduling decisions. + + + + + mapred.capacity-scheduler.queue.default.minimum-user-limit-percent + 100 + Each queue enforces a limit on the percentage of resources + allocated to a user at any given time, if there is competition for them. + This user limit can vary between a minimum and maximum value. The former + depends on the number of users who have submitted jobs, and the latter is + set to this property value. For example, suppose the value of this + property is 25. If two users have submitted jobs to a queue, no single + user can use more than 50% of the queue resources. If a third user submits + a job, no single user can use more than 33% of the queue resources. With 4 + or more users, no user can use more than 25% of the queue's resources. A + value of 100 implies no user limits are imposed. + + + + + mapred.capacity-scheduler.queue.default.user-limit-factor + 1 + The multiple of the queue capacity which can be configured to + allow a single user to acquire more slots. + + + + + mapred.capacity-scheduler.queue.default.maximum-initialized-active-tasks + 200000 + The maximum number of tasks, across all jobs in the queue, + which can be initialized concurrently. Once the queue's jobs exceed this + limit they will be queued on disk. + + + + + mapred.capacity-scheduler.queue.default.maximum-initialized-active-tasks-per-user + 100000 + The maximum number of tasks per-user, across all the of the + user's jobs in the queue, which can be initialized concurrently. Once the + user's jobs exceed this limit they will be queued on disk. + + + + + mapred.capacity-scheduler.queue.default.init-accept-jobs-factor + 10 + The multipe of (maximum-system-jobs * queue-capacity) used to + determine the number of jobs which are accepted by the scheduler. + + + + + + + + mapred.capacity-scheduler.default-supports-priority + false + If true, priorities of jobs will be taken into + account in scheduling decisions by default in a job queue. + + + + + mapred.capacity-scheduler.default-minimum-user-limit-percent + 100 + The percentage of the resources limited to a particular user + for the job queue at any given point of time by default. + + + + + + mapred.capacity-scheduler.default-user-limit-factor + 1 + The default multiple of queue-capacity which is used to + determine the amount of slots a single user can consume concurrently. + + + + + mapred.capacity-scheduler.default-maximum-active-tasks-per-queue + 200000 + The default maximum number of tasks, across all jobs in the + queue, which can be initialized concurrently. Once the queue's jobs exceed + this limit they will be queued on disk. + + + + + mapred.capacity-scheduler.default-maximum-active-tasks-per-user + 100000 + The default maximum number of tasks per-user, across all the of + the user's jobs in the queue, which can be initialized concurrently. Once + the user's jobs exceed this limit they will be queued on disk. + + + + + mapred.capacity-scheduler.default-init-accept-jobs-factor + 10 + The default multipe of (maximum-system-jobs * queue-capacity) + used to determine the number of jobs which are accepted by the scheduler. + + + + + + mapred.capacity-scheduler.init-poll-interval + 5000 + The amount of time in miliseconds which is used to poll + the job queues for jobs to initialize. + + + + mapred.capacity-scheduler.init-worker-threads + 5 + Number of worker threads which would be used by + Initialization poller to initialize jobs in a set of queue. + If number mentioned in property is equal to number of job queues + then a single thread would initialize jobs in a queue. If lesser + then a thread would get a set of queues assigned. If the number + is greater then number of threads would be equal to number of + job queues. + + + + diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/commons-logging.properties b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/commons-logging.properties new file mode 100644 index 0000000000..4b27c824a9 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/commons-logging.properties @@ -0,0 +1,7 @@ +#Logging Implementation + +#Log4J +org.apache.commons.logging.Log=org.apache.commons.logging.impl.Log4JLogger + +#JDK Logger +#org.apache.commons.logging.Log=org.apache.commons.logging.impl.Jdk14Logger diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/core-site.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/core-site.xml index c99b86cb70..a9ba524819 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/core-site.xml +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/core-site.xml @@ -1,27 +1,78 @@ - + + + local.realm + ${KERBEROS_REALM} + + + + fs.default.name - ${HADOOP_NN_HOST} + hdfs://${HADOOP_NN_HOST}:8020 + The name of the default file system. Either the + literal string "local" or a host:port for NDFS. + + true + + + fs.trash.interval + 360 + Number of minutes between trash checkpoints. + If zero, the trash feature is disabled. + + + + + hadoop.security.auth_to_local + + RULE:[2:$1@$0]([jt]t@.*${KERBEROS_REALM})s/.*/${HADOOP_MR_USER}/ + RULE:[2:$1@$0]([nd]n@.*${KERBEROS_REALM})s/.*/${HADOOP_HDFS_USER}/ + RULE:[2:$1@$0](mapred@.*${KERBEROS_REALM})s/.*/${HADOOP_MR_USER}/ + RULE:[2:$1@$0](hdfs@.*${KERBEROS_REALM})s/.*/${HADOOP_HDFS_USER}/ + RULE:[2:$1@$0](mapredqa@.*${KERBEROS_REALM})s/.*/${HADOOP_MR_USER}/ + RULE:[2:$1@$0](hdfsqa@.*${KERBEROS_REALM})s/.*/${HADOOP_HDFS_USER}/ + DEFAULT + + + + + + hadoop.security.authentication + ${SECURITY_TYPE} + + Set the authentication for the cluster. Valid values are: simple or + kerberos. + + + + + hadoop.security.authorization + ${SECURITY} + + Enable authorization for different protocols. + + + + + hadoop.security.groups.cache.secs + 14400 + + + + hadoop.kerberos.kinit.command + ${KINIT} + + + + hadoop.http.filter.initializers + org.apache.hadoop.http.lib.StaticUserWebFilter + + diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-env.sh b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-env.sh new file mode 100644 index 0000000000..1739452457 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-env.sh @@ -0,0 +1,54 @@ +# Set Hadoop-specific environment variables here. + +# The only required environment variable is JAVA_HOME. All others are +# optional. When running a distributed configuration it is best to +# set JAVA_HOME in this file, so that it is correctly defined on +# remote nodes. + +# The java implementation to use. +export JAVA_HOME=${JAVA_HOME} +export HADOOP_CONF_DIR=${HADOOP_CONF_DIR:-"/etc/hadoop"} + +# Extra Java CLASSPATH elements. Automatically insert capacity-scheduler. +for f in $HADOOP_HOME/contrib/capacity-scheduler/*.jar; do + if [ "$HADOOP_CLASSPATH" ]; then + export HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$f + else + export HADOOP_CLASSPATH=$f + fi +done + +# The maximum amount of heap to use, in MB. Default is 1000. +#export HADOOP_HEAPSIZE= +#export HADOOP_NAMENODE_INIT_HEAPSIZE="" + +# Extra Java runtime options. Empty by default. +export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true $HADOOP_OPTS" + +# Command specific options appended to HADOOP_OPTS when specified +export HADOOP_NAMENODE_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dhdfs.audit.logger=INFO,DRFAAUDIT ${HADOOP_NAMENODE_OPTS}" +HADOOP_JOBTRACKER_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dmapred.audit.logger=INFO,MRAUDIT -Dmapred.jobsummary.logger=INFO,JSA ${HADOOP_JOBTRACKER_OPTS}" +HADOOP_TASKTRACKER_OPTS="-Dsecurity.audit.logger=ERROR,console -Dmapred.audit.logger=ERROR,console ${HADOOP_TASKTRACKER_OPTS}" +HADOOP_DATANODE_OPTS="-Dsecurity.audit.logger=ERROR,DRFAS ${HADOOP_DATANODE_OPTS}" + +export HADOOP_SECONDARYNAMENODE_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dhdfs.audit.logger=INFO,DRFAAUDIT ${HADOOP_SECONDARYNAMENODE_OPTS}" + +# The following applies to multiple commands (fs, dfs, fsck, distcp etc) +export HADOOP_CLIENT_OPTS="-Xmx128m ${HADOOP_CLIENT_OPTS}" +#HADOOP_JAVA_PLATFORM_OPTS="-XX:-UsePerfData ${HADOOP_JAVA_PLATFORM_OPTS}" + +# On secure datanodes, user to run the datanode as after dropping privileges +export HADOOP_SECURE_DN_USER=${HADOOP_HDFS_USER} + +# Where log files are stored. $HADOOP_HOME/logs by default. +export HADOOP_LOG_DIR=${HADOOP_LOG_DIR}/$USER + +# Where log files are stored in the secure data environment. +export HADOOP_SECURE_DN_LOG_DIR=${HADOOP_LOG_DIR}/${HADOOP_HDFS_USER} + +# The directory where pid files are stored. /tmp by default. +export HADOOP_PID_DIR=${HADOOP_PID_DIR} +export HADOOP_SECURE_DN_PID_DIR=${HADOOP_PID_DIR} + +# A string representing this instance of hadoop. $USER by default. +export HADOOP_IDENT_STRING=$USER diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml new file mode 100644 index 0000000000..fc5c756928 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml @@ -0,0 +1,118 @@ + + + + + + + + security.client.protocol.acl + * + ACL for ClientProtocol, which is used by user code + via the DistributedFileSystem. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.client.datanode.protocol.acl + * + ACL for ClientDatanodeProtocol, the client-to-datanode protocol + for block recovery. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.datanode.protocol.acl + * + ACL for DatanodeProtocol, which is used by datanodes to + communicate with the namenode. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.inter.datanode.protocol.acl + * + ACL for InterDatanodeProtocol, the inter-datanode protocol + for updating generation timestamp. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.namenode.protocol.acl + * + ACL for NamenodeProtocol, the protocol used by the secondary + namenode to communicate with the namenode. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.inter.tracker.protocol.acl + * + ACL for InterTrackerProtocol, used by the tasktrackers to + communicate with the jobtracker. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.job.submission.protocol.acl + * + ACL for JobSubmissionProtocol, used by job clients to + communciate with the jobtracker for job submission, querying job status etc. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.task.umbilical.protocol.acl + * + ACL for TaskUmbilicalProtocol, used by the map and reduce + tasks to communicate with the parent tasktracker. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.admin.operations.protocol.acl + ${HADOOP_HDFS_USER} + ACL for AdminOperationsProtocol. Used for admin commands. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + security.refresh.usertogroups.mappings.protocol.acl + ${HADOOP_HDFS_USER} + ACL for RefreshUserMappingsProtocol. Used to refresh + users mappings. The ACL is a comma-separated list of user and + group names. The user and group list is separated by a blank. For + e.g. "alice,bob users,wheel". A special value of "*" means all + users are allowed. + + + + security.refresh.policy.protocol.acl + ${HADOOP_HDFS_USER} + ACL for RefreshAuthorizationPolicyProtocol, used by the + dfsadmin and mradmin commands to refresh the security policy in-effect. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + + + + + diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hdfs-site.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hdfs-site.xml new file mode 100644 index 0000000000..ca9dae7d8b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hdfs-site.xml @@ -0,0 +1,225 @@ + + + + + + + + + dfs.name.dir + ${HADOOP_NN_DIR} + Determines where on the local filesystem the DFS name node + should store the name table. If this is a comma-delimited list + of directories then the name table is replicated in all of the + directories, for redundancy. + true + + + + dfs.data.dir + ${HADOOP_DN_DIR} + Determines where on the local filesystem an DFS data node + should store its blocks. If this is a comma-delimited + list of directories, then data will be stored in all named + directories, typically on different devices. + Directories that do not exist are ignored. + + true + + + + dfs.safemode.threshold.pct + 1.0f + + Specifies the percentage of blocks that should satisfy + the minimal replication requirement defined by dfs.replication.min. + Values less than or equal to 0 mean not to start in safe mode. + Values greater than 1 will make safe mode permanent. + + + + + dfs.datanode.address + ${HADOOP_DN_ADDR} + + + + dfs.datanode.http.address + ${HADOOP_DN_HTTP_ADDR} + + + + dfs.http.address + ${HADOOP_NN_HOST}:50070 + The name of the default file system. Either the + literal string "local" or a host:port for NDFS. + + true + + + + + dfs.umaskmode + 077 + + The octal umask used when creating files and directories. + + + + + dfs.block.access.token.enable + ${SECURITY} + + Are access tokens are used as capabilities for accessing datanodes. + + + + + dfs.namenode.kerberos.principal + nn/_HOST@${local.realm} + + Kerberos principal name for the NameNode + + + + + dfs.secondary.namenode.kerberos.principal + nn/_HOST@${local.realm} + + Kerberos principal name for the secondary NameNode. + + + + + + dfs.namenode.kerberos.https.principal + host/_HOST@${local.realm} + + The Kerberos principal for the host that the NameNode runs on. + + + + + dfs.secondary.namenode.kerberos.https.principal + host/_HOST@${local.realm} + + The Kerberos principal for the hostthat the secondary NameNode runs on. + + + + + dfs.secondary.https.port + 50490 + The https port where secondary-namenode binds + + + + + dfs.datanode.kerberos.principal + dn/_HOST@${local.realm} + + The Kerberos principal that the DataNode runs as. "_HOST" is replaced by + the real host name. + + + + + dfs.namenode.keytab.file + /etc/security/keytabs/nn.service.keytab + + Combined keytab file containing the namenode service and host principals. + + + + + dfs.secondary.namenode.keytab.file + /etc/security/keytabs/nn.service.keytab + + Combined keytab file containing the namenode service and host principals. + + + + + dfs.datanode.keytab.file + /etc/security/keytabs/dn.service.keytab + + The filename of the keytab file for the DataNode. + + + + + dfs.https.port + 50470 + The https port where namenode binds + + + + dfs.https.address + ${HADOOP_NN_HOST}:50470 + The https address where namenode binds + + + + dfs.datanode.data.dir.perm + 700 + The permissions that should be there on dfs.data.dir + directories. The datanode will not come up if the permissions are + different on existing dfs.data.dir directories. If the directories + don't exist, they will be created with this permission. + + + + + dfs.cluster.administrators + ${HADOOP_HDFS_USER} + ACL for who all can view the default servlets in the HDFS + + + + dfs.permissions.superusergroup + ${HADOOP_GROUP} + The name of the group of super-users. + + + + dfs.namenode.http-address + ${HADOOP_NN_HOST}:50070 + + The address and the base port where the dfs namenode web ui will listen on. + If the port is 0 then the server will start on a free port. + + + + + dfs.namenode.https-address + ${HADOOP_NN_HOST}:50470 + + + + dfs.secondary.http.address + ${HADOOP_SNN_HOST}:50090 + + The secondary namenode http server address and port. + If the port is 0 then the server will start on a free port. + + + + + dfs.hosts + ${HADOOP_CONF_DIR}/dfs.include + Names a file that contains a list of hosts that are + permitted to connect to the namenode. The full pathname of the file + must be specified. If the value is empty, all hosts are + permitted. + + + + dfs.hosts.exclude + ${HADOOP_CONF_DIR}/dfs.exclude + Names a file that contains a list of hosts that are + not permitted to connect to the namenode. The full pathname of the + file must be specified. If the value is empty, no hosts are + excluded. + + + diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-queue-acls.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-queue-acls.xml new file mode 100644 index 0000000000..f3ab194ae2 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-queue-acls.xml @@ -0,0 +1,12 @@ + + + + +mapred.queue.default.acl-submit-job +* + + +mapred.queue.default.acl-administer-jobs +* + + diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-site.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-site.xml new file mode 100644 index 0000000000..9cee589165 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-site.xml @@ -0,0 +1,268 @@ + + + + + + + + + mapred.tasktracker.tasks.sleeptime-before-sigkill + 250 + Normally, this is the amount of time before killing + processes, and the recommended-default is 5.000 seconds - a value of + 5000 here. In this case, we are using it solely to blast tasks before + killing them, and killing them very quickly (1/4 second) to guarantee + that we do not leave VMs around for later jobs. + + + + + mapred.system.dir + /mapred/mapredsystem + true + + + + mapred.job.tracker + ${HADOOP_JT_HOST}:9000 + true + + + + mapred.job.tracker.http.address + ${HADOOP_JT_HOST}:50030 + true + + + + mapred.local.dir + ${HADOOP_MAPRED_DIR} + true + + + + mapreduce.cluster.administrators + ${HADOOP_MR_USER} + + + + mapred.map.tasks.speculative.execution + false + If true, then multiple instances of some map tasks + may be executed in parallel. + + + + mapred.reduce.tasks.speculative.execution + false + If true, then multiple instances of some reduce tasks + may be executed in parallel. + + + + mapred.output.compression.type + BLOCK + If the job outputs are to compressed as SequenceFiles, how + should they be compressed? Should be one of NONE, RECORD or BLOCK. + + + + + jetty.connector + org.mortbay.jetty.nio.SelectChannelConnector + + + + mapred.task.tracker.task-controller + ${TASK_CONTROLLER} + + + + mapred.child.root.logger + INFO,TLA + + + + stream.tmpdir + ${mapred.temp.dir} + + + + + mapred.child.java.opts + -server -Xmx640m -Djava.net.preferIPv4Stack=true + + + + mapred.child.ulimit + 8388608 + + + + mapred.job.tracker.persist.jobstatus.active + true + Indicates if persistency of job status information is + active or not. + + + + + mapred.job.tracker.persist.jobstatus.dir + file:///${HADOOP_LOG_DIR}/${HADOOP_MR_USER}/jobstatus + The directory where the job status information is persisted + in a file system to be available after it drops of the memory queue and + between jobtracker restarts. + + + + + mapred.job.tracker.history.completed.location + /mapred/history/done + + + + mapred.heartbeats.in.second + 200 + to enable HADOOP:5784 + + + + mapreduce.tasktracker.outofband.heartbeat + true + to enable MAPREDUCE:270 + + + + mapred.jobtracker.maxtasks.per.job + 200000 + true + The maximum number of tasks for a single job. + A value of -1 indicates that there is no maximum. + + + + + mapreduce.jobtracker.kerberos.principal + jt/_HOST@${local.realm} + + JT principal + + + + + mapreduce.tasktracker.kerberos.principal + tt/_HOST@${local.realm} + + TT principal. + + + + + + hadoop.job.history.user.location + none + + + + mapreduce.jobtracker.keytab.file + /etc/security/keytabs/jt.service.keytab + + The keytab for the jobtracker principal. + + + + + mapreduce.tasktracker.keytab.file + /etc/security/keytabs/tt.service.keytab + The filename of the keytab for the task tracker + + + + mapreduce.jobtracker.staging.root.dir + /user + The Path prefix for where the staging directories should be + placed. The next level is always the user's + name. It is a path in the default file system. + + + + + + mapreduce.job.acl-modify-job + + + + + mapreduce.job.acl-view-job + Dr.Who + + + + mapreduce.tasktracker.group + ${HADOOP_GROUP} + The group that the task controller uses for accessing the + task controller. The mapred user must be a member and users should *not* + be members. + + + + + mapred.acls.enabled + true + + + + mapred.jobtracker.taskScheduler + org.apache.hadoop.mapred.CapacityTaskScheduler + + + mapred.queue.names + default + + + + + mapreduce.history.server.embedded + false + + + mapreduce.history.server.http.address + ${HADOOP_JT_HOST}:51111 + + + mapreduce.jobhistory.kerberos.principal + jt/_HOST@${local.realm} + history server principal + + + mapreduce.jobhistory.keytab.file + /etc/security/keytabs/jt.service.keytab + + The keytab for the jobtracker principal. + + + + + mapred.hosts + ${HADOOP_CONF_DIR}/mapred.include + Names a file that contains the list of nodes that may + connect to the jobtracker. If the value is empty, all hosts are + permitted. + + + + mapred.hosts.exclude + ${HADOOP_CONF_DIR}/mapred.exclude + Names a file that contains the list of hosts that + should be excluded by the jobtracker. If the value is empty, no + hosts are excluded. + + + mapred.jobtracker.retirejob.check + 10000 + + + mapred.jobtracker.retirejob.interval + 0 + + diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/taskcontroller.cfg b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/taskcontroller.cfg new file mode 100644 index 0000000000..428e46295b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/taskcontroller.cfg @@ -0,0 +1,3 @@ +mapreduce.cluster.local.dir=${HADOOP_MAPRED_DIR} +mapreduce.tasktracker.group=${HADOOP_GROUP} +hadoop.log.dir=${HADOOP_LOG_DIR}/${HADOOP_MR_USER} diff --git a/hadoop-common-project/hadoop-common/src/main/packages/update-hadoop-env.sh b/hadoop-common-project/hadoop-common/src/main/packages/update-hadoop-env.sh index eb17344dfc..183cce342a 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/update-hadoop-env.sh +++ b/hadoop-common-project/hadoop-common/src/main/packages/update-hadoop-env.sh @@ -134,7 +134,9 @@ if [ "${UNINSTALL}" -eq "1" ]; then rm -rf ${HADOOP_PREFIX}/etc/hadoop fi rm -f /etc/default/hadoop-env.sh - rm -f /etc/profile.d/hadoop-env.sh + if [ -d /etc/profile.d ]; then + rm -f /etc/profile.d/hadoop-env.sh + fi else # Create symlinks if [ "${HADOOP_CONF_DIR}" != "${HADOOP_PREFIX}/etc/hadoop" ]; then @@ -142,7 +144,9 @@ else ln -sf ${HADOOP_CONF_DIR} ${HADOOP_PREFIX}/etc/hadoop fi ln -sf ${HADOOP_CONF_DIR}/hadoop-env.sh /etc/default/hadoop-env.sh - ln -sf ${HADOOP_CONF_DIR}/hadoop-env.sh /etc/profile.d/hadoop-env.sh + if [ -d /etc/profile.d ]; then + ln -sf ${HADOOP_CONF_DIR}/hadoop-env.sh /etc/profile.d/hadoop-env.sh + fi mkdir -p ${HADOOP_LOG_DIR} chown root:hadoop ${HADOOP_LOG_DIR} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/serializer/TestSerializationFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/serializer/TestSerializationFactory.java new file mode 100644 index 0000000000..18c2637ec5 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/serializer/TestSerializationFactory.java @@ -0,0 +1,44 @@ +/** + * 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.io.serializer; + +import org.junit.Test; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertNotNull; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Writable; + +public class TestSerializationFactory { + + @Test + public void testSerializerAvailability() { + Configuration conf = new Configuration(); + SerializationFactory factory = new SerializationFactory(conf); + // Test that a valid serializer class is returned when its present + assertNotNull("A valid class must be returned for default Writable Serde", + factory.getSerializer(Writable.class)); + assertNotNull("A valid class must be returned for default Writable serDe", + factory.getDeserializer(Writable.class)); + // Test that a null is returned when none can be found. + assertNull("A null should be returned if there are no serializers found.", + factory.getSerializer(TestSerializationFactory.class)); + assertNull("A null should be returned if there are no deserializers found", + factory.getDeserializer(TestSerializationFactory.class)); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index ea81b8034e..84aca2d256 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -5,6 +5,11 @@ Trunk (unreleased changes) HDFS-395. DFS Scalability: Incremental block reports. (Tomasz Nykiel via hairong) + HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http + access to HDFS. (szetszwo) + + HDFS-2317. Support read access to HDFS in webhdfs. (szetszwo) + IMPROVEMENTS HADOOP-7524 Change RPC to allow multiple protocols including multuple versions of the same protocol (sanjay Radia) @@ -1029,7 +1034,9 @@ Release 0.23.0 - Unreleased (todd) HDFS-2289. Ensure jsvc is bundled with the HDFS distribution artifact. - (Alejandro Abdelnur via acmurthy) + (Alejandro Abdelnur via acmurthy) + + HDFS-2323. start-dfs.sh script fails for tarball install (tomwhite) BREAKDOWN OF HDFS-1073 SUBTASKS diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh index 32dcf860cd..d6ed5f99e6 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh @@ -51,7 +51,7 @@ NAMENODES=$($HADOOP_PREFIX/bin/hdfs getconf -namenodes) echo "Starting namenodes on [$NAMENODES]" -"$HADOOP_PREFIX/bin/hadoop-daemons.sh" \ +"$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \ --config "$HADOOP_CONF_DIR" \ --hostnames "$NAMENODES" \ --script "$bin/hdfs" start namenode $nameStartOpt @@ -64,7 +64,7 @@ if [ -n "$HADOOP_SECURE_DN_USER" ]; then "Attempting to start secure cluster, skipping datanodes. " \ "Run start-secure-dns.sh as root to complete startup." else - "$HADOOP_PREFIX/bin/hadoop-daemons.sh" \ + "$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \ --config "$HADOOP_CONF_DIR" \ --script "$bin/hdfs" start datanode $dataStartOpt fi @@ -84,7 +84,7 @@ if [ "$SECONDARY_NAMENODES" = '0.0.0.0' ] ; then else echo "Starting secondary namenodes [$SECONDARY_NAMENODES]" - "$HADOOP_PREFIX/bin/hadoop-daemons.sh" \ + "$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \ --config "$HADOOP_CONF_DIR" \ --hostnames "$SECONDARY_NAMENODES" \ --script "$bin/hdfs" start secondarynamenode diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-secure-dns.sh b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-secure-dns.sh index c4190dbde0..bcb9b8f82d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-secure-dns.sh +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-secure-dns.sh @@ -25,7 +25,7 @@ bin=`cd "$bin"; pwd` . "$bin"/../libexec/hdfs-config.sh if [ "$EUID" -eq 0 ] && [ -n "$HADOOP_SECURE_DN_USER" ]; then - "$HADOOP_PREFIX"/bin/hadoop-daemons.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs start datanode $dataStartOpt + "$HADOOP_PREFIX"/sbin/hadoop-daemons.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs start datanode $dataStartOpt else echo $usage fi diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh index 7158ca6bda..e64a5ea879 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh @@ -27,7 +27,7 @@ NAMENODES=$($HADOOP_PREFIX/bin/hdfs getconf -namenodes) echo "Stopping namenodes on [$NAMENODES]" -"$HADOOP_PREFIX/bin/hadoop-daemons.sh" \ +"$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \ --config "$HADOOP_CONF_DIR" \ --hostnames "$NAMENODES" \ --script "$bin/hdfs" stop namenode @@ -40,7 +40,7 @@ if [ -n "$HADOOP_SECURE_DN_USER" ]; then "Attempting to stop secure cluster, skipping datanodes. " \ "Run stop-secure-dns.sh as root to complete shutdown." else - "$HADOOP_PREFIX/bin/hadoop-daemons.sh" \ + "$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \ --config "$HADOOP_CONF_DIR" \ --script "$bin/hdfs" stop datanode fi @@ -60,7 +60,7 @@ if [ "$SECONDARY_NAMENODES" = '0.0.0.0' ] ; then else echo "Stopping secondary namenodes [$SECONDARY_NAMENODES]" - "$HADOOP_PREFIX/bin/hadoop-daemons.sh" \ + "$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \ --config "$HADOOP_CONF_DIR" \ --hostnames "$SECONDARY_NAMENODES" \ --script "$bin/hdfs" stop secondarynamenode diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-secure-dns.sh b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-secure-dns.sh index 63854c4495..a0ae1728d4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-secure-dns.sh +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-secure-dns.sh @@ -25,7 +25,7 @@ bin=`cd "$bin"; pwd` . "$bin"/../libexec/hdfs-config.sh if [ "$EUID" -eq 0 ] && [ -n "$HADOOP_SECURE_DN_USER" ]; then - "$HADOOP_PREFIX"/bin/hadoop-daemons.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs stop datanode + "$HADOOP_PREFIX"/sbin/hadoop-daemons.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs stop datanode else echo $usage fi diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java index fcdf6cb564..945f4ac033 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java @@ -33,7 +33,7 @@ * is made on the successive read(). The normal input stream functions are * connected to the currently active input stream. */ -class ByteRangeInputStream extends FSInputStream { +public class ByteRangeInputStream extends FSInputStream { /** * This class wraps a URL to allow easy mocking when testing. The URL class @@ -71,7 +71,8 @@ enum StreamStatus { StreamStatus status = StreamStatus.SEEK; - ByteRangeInputStream(final URL url) { + /** Create an input stream with the URL. */ + public ByteRangeInputStream(final URL url) { this(new URLOpener(url), new URLOpener(null)); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 85639afc1b..8a29928249 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -802,7 +802,7 @@ private DFSOutputStream callAppend(HdfsFileStatus stat, String src, * * @see ClientProtocol#append(String, String) */ - DFSOutputStream append(String src, int buffersize, Progressable progress) + public DFSOutputStream append(String src, int buffersize, Progressable progress) throws IOException { checkOpen(); HdfsFileStatus stat = getFileInfo(src); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index c330297cd3..4ab8dfc54f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -98,7 +98,7 @@ * datanode from the original pipeline. The DataStreamer now * starts sending packets from the dataQueue. ****************************************************************/ -class DFSOutputStream extends FSOutputSummer implements Syncable { +public class DFSOutputStream extends FSOutputSummer implements Syncable { private final DFSClient dfsClient; private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB private Socket s; @@ -1707,7 +1707,7 @@ synchronized void setTestFilename(String newname) { /** * Returns the size of a file as it was when this stream was opened */ - long getInitialLen() { + public long getInitialLen() { return initialFileSize; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 68f8616941..68ecf0f4af 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -229,12 +229,11 @@ public boolean recoverLease(Path f) throws IOException { return dfs.recoverLease(getPathName(f)); } - @SuppressWarnings("deprecation") @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { statistics.incrementReadOps(1); return new DFSClient.DFSDataInputStream( - dfs.open(getPathName(f), bufferSize, verifyChecksum, statistics)); + dfs.open(getPathName(f), bufferSize, verifyChecksum)); } /** This optional operation is not yet supported. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java index 98650b5e4c..d049dd2b6f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java @@ -250,7 +250,7 @@ public URI getUri() { * @return namenode URL referring to the given path * @throws IOException on error constructing the URL */ - URL getNamenodeURL(String path, String query) throws IOException { + protected URL getNamenodeURL(String path, String query) throws IOException { final URL url = new URL("http", nnAddr.getHostName(), nnAddr.getPort(), path + '?' + query); if (LOG.isTraceEnabled()) { @@ -317,6 +317,7 @@ protected String addDelegationTokenParam(String query) throws IOException { @Override public FSDataInputStream open(Path f, int buffersize) throws IOException { + f = f.makeQualified(getUri(), getWorkingDirectory()); String path = "/data" + ServletUtil.encodePath(f.toUri().getPath()); String query = addDelegationTokenParam("ugi=" + getEncodedUgiParameter()); URL u = getNamenodeURL(path, query); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index edc57fd797..81cdc85111 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -18,7 +18,50 @@ package org.apache.hadoop.hdfs.server.datanode; -import static org.apache.hadoop.hdfs.DFSConfigKeys.*; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY; import static org.apache.hadoop.hdfs.server.common.Util.now; import java.io.BufferedOutputStream; @@ -93,9 +136,11 @@ import org.apache.hadoop.hdfs.server.datanode.FSDataset.VolumeInfo; import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources; import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics; +import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets; import org.apache.hadoop.hdfs.server.namenode.StreamFile; +import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; @@ -109,7 +154,8 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand; -import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand; +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; +import org.apache.hadoop.hdfs.web.resources.Param; import org.apache.hadoop.http.HttpServer; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.ProtocolSignature; @@ -502,6 +548,11 @@ conf, new AccessControlList(conf.get(DFS_ADMIN, " ")), this.infoServer.setAttribute(JspHelper.CURRENT_CONF, conf); this.infoServer.addServlet(null, "/blockScannerReport", DataBlockScanner.Servlet.class); + + infoServer.addJerseyResourcePackage( + DatanodeWebHdfsMethods.class.getPackage().getName() + + ";" + Param.class.getPackage().getName(), + "/" + WebHdfsFileSystem.PATH_PREFIX + "/*"); this.infoServer.start(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java new file mode 100644 index 0000000000..e270a961bf --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java @@ -0,0 +1,228 @@ +/** + * 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.datanode.web.resources; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.EnumSet; + +import javax.servlet.ServletContext; +import javax.ws.rs.Consumes; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.StreamingOutput; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.hdfs.DFSClient; +import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream; +import org.apache.hadoop.hdfs.DFSOutputStream; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; +import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; +import org.apache.hadoop.hdfs.web.resources.BufferSizeParam; +import org.apache.hadoop.hdfs.web.resources.GetOpParam; +import org.apache.hadoop.hdfs.web.resources.LengthParam; +import org.apache.hadoop.hdfs.web.resources.OffsetParam; +import org.apache.hadoop.hdfs.web.resources.OverwriteParam; +import org.apache.hadoop.hdfs.web.resources.Param; +import org.apache.hadoop.hdfs.web.resources.PermissionParam; +import org.apache.hadoop.hdfs.web.resources.PostOpParam; +import org.apache.hadoop.hdfs.web.resources.PutOpParam; +import org.apache.hadoop.hdfs.web.resources.ReplicationParam; +import org.apache.hadoop.hdfs.web.resources.UriFsPathParam; +import org.apache.hadoop.io.IOUtils; + +/** Web-hdfs DataNode implementation. */ +@Path("") +public class DatanodeWebHdfsMethods { + public static final Log LOG = LogFactory.getLog(DatanodeWebHdfsMethods.class); + + private @Context ServletContext context; + + /** Handle HTTP PUT request. */ + @PUT + @Path("{" + UriFsPathParam.NAME + ":.*}") + @Consumes({"*/*"}) + @Produces({MediaType.APPLICATION_JSON}) + public Response put( + final InputStream in, + @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, + @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT) + final PutOpParam op, + @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT) + final PermissionParam permission, + @QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT) + final OverwriteParam overwrite, + @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) + final BufferSizeParam bufferSize, + @QueryParam(ReplicationParam.NAME) @DefaultValue(ReplicationParam.DEFAULT) + final ReplicationParam replication, + @QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT) + final BlockSizeParam blockSize + ) throws IOException, URISyntaxException { + + if (LOG.isTraceEnabled()) { + LOG.trace(op + ": " + path + + Param.toSortedString(", ", permission, overwrite, bufferSize, + replication, blockSize)); + } + + final String fullpath = path.getAbsolutePath(); + final DataNode datanode = (DataNode)context.getAttribute("datanode"); + + switch(op.getValue()) { + case CREATE: + { + final Configuration conf = new Configuration(datanode.getConf()); + final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf); + final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf); + final FSDataOutputStream out = new FSDataOutputStream(dfsclient.create( + fullpath, permission.getFsPermission(), + overwrite.getValue() ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE) + : EnumSet.of(CreateFlag.CREATE), + replication.getValue(), blockSize.getValue(), null, + bufferSize.getValue()), null); + try { + IOUtils.copyBytes(in, out, bufferSize.getValue()); + } finally { + out.close(); + } + final InetSocketAddress nnHttpAddr = NameNode.getHttpAddress(conf); + final URI uri = new URI(WebHdfsFileSystem.SCHEME, null, + nnHttpAddr.getHostName(), nnHttpAddr.getPort(), fullpath, null, null); + return Response.created(uri).type(MediaType.APPLICATION_JSON).build(); + } + default: + throw new UnsupportedOperationException(op + " is not supported"); + } + } + + /** Handle HTTP POST request. */ + @POST + @Path("{" + UriFsPathParam.NAME + ":.*}") + @Consumes({"*/*"}) + @Produces({MediaType.APPLICATION_JSON}) + public Response post( + final InputStream in, + @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, + @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT) + final PostOpParam op, + @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) + final BufferSizeParam bufferSize + ) throws IOException, URISyntaxException { + + if (LOG.isTraceEnabled()) { + LOG.trace(op + ": " + path + + Param.toSortedString(", ", bufferSize)); + } + + final String fullpath = path.getAbsolutePath(); + final DataNode datanode = (DataNode)context.getAttribute("datanode"); + + switch(op.getValue()) { + case APPEND: + { + final Configuration conf = new Configuration(datanode.getConf()); + final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf); + final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf); + final DFSOutputStream dfsout = dfsclient.append(fullpath, + bufferSize.getValue(), null); + final FSDataOutputStream out = new FSDataOutputStream(dfsout, null, + dfsout.getInitialLen()); + try { + IOUtils.copyBytes(in, out, bufferSize.getValue()); + } finally { + out.close(); + } + return Response.ok().type(MediaType.APPLICATION_JSON).build(); + } + default: + throw new UnsupportedOperationException(op + " is not supported"); + } + } + + /** Handle HTTP GET request. */ + @GET + @Path("{" + UriFsPathParam.NAME + ":.*}") + @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON}) + public Response get( + @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, + @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT) + final GetOpParam op, + @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT) + final OffsetParam offset, + @QueryParam(LengthParam.NAME) @DefaultValue(LengthParam.DEFAULT) + final LengthParam length, + @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) + final BufferSizeParam bufferSize + ) throws IOException, URISyntaxException { + + if (LOG.isTraceEnabled()) { + LOG.trace(op + ": " + path + + Param.toSortedString(", ", offset, length, bufferSize)); + } + + final String fullpath = path.getAbsolutePath(); + final DataNode datanode = (DataNode)context.getAttribute("datanode"); + + switch(op.getValue()) { + case OPEN: + { + final Configuration conf = new Configuration(datanode.getConf()); + final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf); + final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf); + final DFSDataInputStream in = new DFSClient.DFSDataInputStream( + dfsclient.open(fullpath, bufferSize.getValue(), true)); + in.seek(offset.getValue()); + + final StreamingOutput streaming = new StreamingOutput() { + @Override + public void write(final OutputStream out) throws IOException { + final Long n = length.getValue(); + if (n == null) { + IOUtils.copyBytes(in, out, bufferSize.getValue()); + } else { + IOUtils.copyBytes(in, out, n, false); + } + } + }; + return Response.ok(streaming).type(MediaType.APPLICATION_OCTET_STREAM).build(); + } + default: + throw new UnsupportedOperationException(op + " is not supported"); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index ed972c8f96..6db96924c2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -335,6 +335,11 @@ protected void setRpcServerAddress(Configuration conf, } protected InetSocketAddress getHttpServerAddress(Configuration conf) { + return getHttpAddress(conf); + } + + /** @return the NameNode HTTP address set in the conf. */ + public static InetSocketAddress getHttpAddress(Configuration conf) { return NetUtils.createSocketAddr( conf.get(DFS_NAMENODE_HTTP_ADDRESS_KEY, DFS_NAMENODE_HTTP_ADDRESS_DEFAULT)); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java index fc911fd944..d8a0c50054 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java @@ -24,18 +24,20 @@ import javax.servlet.ServletContext; import org.apache.commons.logging.Log; +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.server.common.JspHelper; +import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods; +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; +import org.apache.hadoop.hdfs.web.resources.Param; import org.apache.hadoop.http.HttpServer; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; -import org.apache.hadoop.classification.InterfaceAudience; - /** * Encapsulates the HTTP server started by the NameNode. */ @@ -179,6 +181,11 @@ private static void setupServlets(HttpServer httpServer) { FileChecksumServlets.RedirectServlet.class, false); httpServer.addInternalServlet("contentSummary", "/contentSummary/*", ContentSummaryServlet.class, false); + + httpServer.addJerseyResourcePackage( + NamenodeWebHdfsMethods.class.getPackage().getName() + + ";" + Param.class.getPackage().getName(), + "/" + WebHdfsFileSystem.PATH_PREFIX + "/*"); } public static FSImage getFsImageFromContext(ServletContext context) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java new file mode 100644 index 0000000000..362060a22d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java @@ -0,0 +1,400 @@ +/** + * 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.web.resources; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.PrintStream; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.EnumSet; + +import javax.servlet.ServletContext; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.StreamingOutput; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Options; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.common.JspHelper; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; +import org.apache.hadoop.hdfs.web.JsonUtil; +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; +import org.apache.hadoop.hdfs.web.resources.AccessTimeParam; +import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; +import org.apache.hadoop.hdfs.web.resources.BufferSizeParam; +import org.apache.hadoop.hdfs.web.resources.DeleteOpParam; +import org.apache.hadoop.hdfs.web.resources.DstPathParam; +import org.apache.hadoop.hdfs.web.resources.GetOpParam; +import org.apache.hadoop.hdfs.web.resources.GroupParam; +import org.apache.hadoop.hdfs.web.resources.HttpOpParam; +import org.apache.hadoop.hdfs.web.resources.LengthParam; +import org.apache.hadoop.hdfs.web.resources.ModificationTimeParam; +import org.apache.hadoop.hdfs.web.resources.OffsetParam; +import org.apache.hadoop.hdfs.web.resources.OverwriteParam; +import org.apache.hadoop.hdfs.web.resources.OwnerParam; +import org.apache.hadoop.hdfs.web.resources.Param; +import org.apache.hadoop.hdfs.web.resources.PermissionParam; +import org.apache.hadoop.hdfs.web.resources.PostOpParam; +import org.apache.hadoop.hdfs.web.resources.PutOpParam; +import org.apache.hadoop.hdfs.web.resources.RecursiveParam; +import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam; +import org.apache.hadoop.hdfs.web.resources.ReplicationParam; +import org.apache.hadoop.hdfs.web.resources.UriFsPathParam; +import org.apache.hadoop.net.NodeBase; + +/** Web-hdfs NameNode implementation. */ +@Path("") +public class NamenodeWebHdfsMethods { + private static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class); + + private @Context ServletContext context; + + private static DatanodeInfo chooseDatanode(final NameNode namenode, + final String path, final HttpOpParam.Op op, final long openOffset + ) throws IOException { + if (op == GetOpParam.Op.OPEN || op == PostOpParam.Op.APPEND) { + final NamenodeProtocols np = namenode.getRpcServer(); + final HdfsFileStatus status = np.getFileInfo(path); + final long len = status.getLen(); + if (op == GetOpParam.Op.OPEN && (openOffset < 0L || openOffset >= len)) { + throw new IOException("Offset=" + openOffset + " out of the range [0, " + + len + "); " + op + ", path=" + path); + } + + if (len > 0) { + final long offset = op == GetOpParam.Op.OPEN? openOffset: len - 1; + final LocatedBlocks locations = np.getBlockLocations(path, offset, 1); + final int count = locations.locatedBlockCount(); + if (count > 0) { + return JspHelper.bestNode(locations.get(0)); + } + } + } + + return (DatanodeDescriptor)namenode.getNamesystem().getBlockManager( + ).getDatanodeManager().getNetworkTopology().chooseRandom( + NodeBase.ROOT); + } + + private static URI redirectURI(final NameNode namenode, + final String path, final HttpOpParam.Op op, final long openOffset, + final Param... parameters) throws URISyntaxException, IOException { + final DatanodeInfo dn = chooseDatanode(namenode, path, op, openOffset); + final String query = op.toQueryString() + Param.toSortedString("&", parameters); + final String uripath = "/" + WebHdfsFileSystem.PATH_PREFIX + path; + + final URI uri = new URI("http", null, dn.getHostName(), dn.getInfoPort(), + uripath, query, null); + if (LOG.isTraceEnabled()) { + LOG.trace("redirectURI=" + uri); + } + return uri; + } + + /** Handle HTTP PUT request. */ + @PUT + @Path("{" + UriFsPathParam.NAME + ":.*}") + @Consumes({"*/*"}) + @Produces({MediaType.APPLICATION_JSON}) + public Response put( + final InputStream in, + @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, + @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT) + final PutOpParam op, + @QueryParam(DstPathParam.NAME) @DefaultValue(DstPathParam.DEFAULT) + final DstPathParam dstPath, + @QueryParam(OwnerParam.NAME) @DefaultValue(OwnerParam.DEFAULT) + final OwnerParam owner, + @QueryParam(GroupParam.NAME) @DefaultValue(GroupParam.DEFAULT) + final GroupParam group, + @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT) + final PermissionParam permission, + @QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT) + final OverwriteParam overwrite, + @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) + final BufferSizeParam bufferSize, + @QueryParam(ReplicationParam.NAME) @DefaultValue(ReplicationParam.DEFAULT) + final ReplicationParam replication, + @QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT) + final BlockSizeParam blockSize, + @QueryParam(ModificationTimeParam.NAME) @DefaultValue(ModificationTimeParam.DEFAULT) + final ModificationTimeParam modificationTime, + @QueryParam(AccessTimeParam.NAME) @DefaultValue(AccessTimeParam.DEFAULT) + final AccessTimeParam accessTime, + @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT) + final RenameOptionSetParam renameOptions + ) throws IOException, URISyntaxException { + + if (LOG.isTraceEnabled()) { + LOG.trace(op + ": " + path + + Param.toSortedString(", ", dstPath, owner, group, permission, + overwrite, bufferSize, replication, blockSize, + modificationTime, accessTime, renameOptions)); + } + + final String fullpath = path.getAbsolutePath(); + final NameNode namenode = (NameNode)context.getAttribute("name.node"); + final NamenodeProtocols np = namenode.getRpcServer(); + + switch(op.getValue()) { + case CREATE: + { + final URI uri = redirectURI(namenode, fullpath, op.getValue(), -1L, + permission, overwrite, bufferSize, replication, blockSize); + return Response.temporaryRedirect(uri).build(); + } + case MKDIRS: + { + final boolean b = np.mkdirs(fullpath, permission.getFsPermission(), true); + final String js = JsonUtil.toJsonString(PutOpParam.Op.MKDIRS, b); + return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); + } + case RENAME: + { + final EnumSet s = renameOptions.getValue(); + if (s.isEmpty()) { + @SuppressWarnings("deprecation") + final boolean b = np.rename(fullpath, dstPath.getValue()); + final String js = JsonUtil.toJsonString(PutOpParam.Op.RENAME, b); + return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); + } else { + np.rename(fullpath, dstPath.getValue(), + s.toArray(new Options.Rename[s.size()])); + return Response.ok().type(MediaType.APPLICATION_JSON).build(); + } + } + case SETREPLICATION: + { + final boolean b = np.setReplication(fullpath, replication.getValue()); + final String js = JsonUtil.toJsonString(PutOpParam.Op.SETREPLICATION, b); + return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); + } + case SETOWNER: + { + np.setOwner(fullpath, owner.getValue(), group.getValue()); + return Response.ok().type(MediaType.APPLICATION_JSON).build(); + } + case SETPERMISSION: + { + np.setPermission(fullpath, permission.getFsPermission()); + return Response.ok().type(MediaType.APPLICATION_JSON).build(); + } + case SETTIMES: + { + np.setTimes(fullpath, modificationTime.getValue(), accessTime.getValue()); + return Response.ok().type(MediaType.APPLICATION_JSON).build(); + } + default: + throw new UnsupportedOperationException(op + " is not supported"); + } + } + + /** Handle HTTP POST request. */ + @POST + @Path("{" + UriFsPathParam.NAME + ":.*}") + @Consumes({"*/*"}) + @Produces({MediaType.APPLICATION_JSON}) + public Response post( + final InputStream in, + @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, + @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT) + final PostOpParam op, + @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) + final BufferSizeParam bufferSize + ) throws IOException, URISyntaxException { + + if (LOG.isTraceEnabled()) { + LOG.trace(op + ": " + path + + Param.toSortedString(", ", bufferSize)); + } + + final String fullpath = path.getAbsolutePath(); + final NameNode namenode = (NameNode)context.getAttribute("name.node"); + + switch(op.getValue()) { + case APPEND: + { + final URI uri = redirectURI(namenode, fullpath, op.getValue(), -1L, + bufferSize); + return Response.temporaryRedirect(uri).build(); + } + default: + throw new UnsupportedOperationException(op + " is not supported"); + } + } + + private static final UriFsPathParam ROOT = new UriFsPathParam(""); + + /** Handle HTTP GET request for the root. */ + @GET + @Path("/") + @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON}) + public Response root( + @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT) + final GetOpParam op, + @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT) + final OffsetParam offset, + @QueryParam(LengthParam.NAME) @DefaultValue(LengthParam.DEFAULT) + final LengthParam length, + @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) + final BufferSizeParam bufferSize + ) throws IOException, URISyntaxException { + return get(ROOT, op, offset, length, bufferSize); + } + + /** Handle HTTP GET request. */ + @GET + @Path("{" + UriFsPathParam.NAME + ":.*}") + @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON}) + public Response get( + @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, + @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT) + final GetOpParam op, + @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT) + final OffsetParam offset, + @QueryParam(LengthParam.NAME) @DefaultValue(LengthParam.DEFAULT) + final LengthParam length, + @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) + final BufferSizeParam bufferSize + ) throws IOException, URISyntaxException { + + if (LOG.isTraceEnabled()) { + LOG.trace(op + ", " + path + + Param.toSortedString(", ", offset, length, bufferSize)); + } + + final NameNode namenode = (NameNode)context.getAttribute("name.node"); + final String fullpath = path.getAbsolutePath(); + final NamenodeProtocols np = namenode.getRpcServer(); + + switch(op.getValue()) { + case OPEN: + { + final URI uri = redirectURI(namenode, fullpath, op.getValue(), + offset.getValue(), offset, length, bufferSize); + return Response.temporaryRedirect(uri).build(); + } + case GETFILESTATUS: + { + final HdfsFileStatus status = np.getFileInfo(fullpath); + final String js = JsonUtil.toJsonString(status); + return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); + } + case LISTSTATUS: + { + final StreamingOutput streaming = getListingStream(np, fullpath); + return Response.ok(streaming).type(MediaType.APPLICATION_JSON).build(); + } + default: + throw new UnsupportedOperationException(op + " is not supported"); + } + } + + private static DirectoryListing getDirectoryListing(final NamenodeProtocols np, + final String p, byte[] startAfter) throws IOException { + final DirectoryListing listing = np.getListing(p, startAfter, false); + if (listing == null) { // the directory does not exist + throw new FileNotFoundException("File " + p + " does not exist."); + } + return listing; + } + + private static StreamingOutput getListingStream(final NamenodeProtocols np, + final String p) throws IOException { + final DirectoryListing first = getDirectoryListing(np, p, + HdfsFileStatus.EMPTY_NAME); + + return new StreamingOutput() { + @Override + public void write(final OutputStream outstream) throws IOException { + final PrintStream out = new PrintStream(outstream); + out.print('['); + + final HdfsFileStatus[] partial = first.getPartialListing(); + if (partial.length > 0) { + out.print(JsonUtil.toJsonString(partial[0])); + } + for(int i = 1; i < partial.length; i++) { + out.println(','); + out.print(JsonUtil.toJsonString(partial[i])); + } + + for(DirectoryListing curr = first; curr.hasMore(); ) { + curr = getDirectoryListing(np, p, curr.getLastName()); + for(HdfsFileStatus s : curr.getPartialListing()) { + out.println(','); + out.print(JsonUtil.toJsonString(s)); + } + } + + out.println(']'); + } + }; + } + + /** Handle HTTP DELETE request. */ + @DELETE + @Path("{path:.*}") + @Produces(MediaType.APPLICATION_JSON) + public Response delete( + @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, + @QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT) + final DeleteOpParam op, + @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT) + final RecursiveParam recursive + ) throws IOException { + + if (LOG.isTraceEnabled()) { + LOG.trace(op + ", " + path + + Param.toSortedString(", ", recursive)); + } + + switch(op.getValue()) { + case DELETE: + final NameNode namenode = (NameNode)context.getAttribute("name.node"); + final String fullpath = path.getAbsolutePath(); + final boolean b = namenode.getRpcServer().delete(fullpath, recursive.getValue()); + final String js = JsonUtil.toJsonString(DeleteOpParam.Op.DELETE, b); + return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); + + default: + throw new UnsupportedOperationException(op + " is not supported"); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java index fd61bac478..b9631430d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java @@ -227,7 +227,10 @@ void printOut(String message) { void printList(List list) { StringBuilder buffer = new StringBuilder(); for (InetSocketAddress address : list) { - buffer.append(address.getHostName()).append(" "); + if (buffer.length() > 0) { + buffer.append(" "); + } + buffer.append(address.getHostName()); } printOut(buffer.toString()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java new file mode 100644 index 0000000000..1c18dc334e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java @@ -0,0 +1,133 @@ +/** + * 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.web; + +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.ipc.RemoteException; +import org.mortbay.util.ajax.JSON; + +/** JSON Utilities */ +public class JsonUtil { + private static final ThreadLocal> jsonMap + = new ThreadLocal>() { + @Override + protected Map initialValue() { + return new TreeMap(); + } + + @Override + public Map get() { + final Map m = super.get(); + m.clear(); + return m; + } + }; + + /** Convert an exception object to a Json string. */ + public static String toJsonString(final Exception e) { + final Map m = jsonMap.get(); + m.put("className", e.getClass().getName()); + m.put("message", e.getMessage()); + return JSON.toString(m); + } + + /** Convert a Json map to a RemoteException. */ + public static RemoteException toRemoteException(final Map m) { + final String className = (String)m.get("className"); + final String message = (String)m.get("message"); + return new RemoteException(className, message); + } + + /** Convert a key-value pair to a Json string. */ + public static String toJsonString(final Object key, final Object value) { + final Map m = jsonMap.get(); + m.put(key instanceof String ? (String) key : key.toString(), value); + return JSON.toString(m); + } + + /** Convert a FsPermission object to a string. */ + public static String toString(final FsPermission permission) { + return String.format("%o", permission.toShort()); + } + + /** Convert a string to a FsPermission object. */ + public static FsPermission toFsPermission(final String s) { + return new FsPermission(Short.parseShort(s, 8)); + } + + /** Convert a HdfsFileStatus object to a Json string. */ + public static String toJsonString(final HdfsFileStatus status) { + final Map m = jsonMap.get(); + if (status == null) { + m.put("isNull", true); + } else { + m.put("isNull", false); + m.put("localName", status.getLocalName()); + m.put("isDir", status.isDir()); + m.put("isSymlink", status.isSymlink()); + if (status.isSymlink()) { + m.put("symlink", status.getSymlink()); + } + + m.put("len", status.getLen()); + m.put("owner", status.getOwner()); + m.put("group", status.getGroup()); + m.put("permission", toString(status.getPermission())); + m.put("accessTime", status.getAccessTime()); + m.put("modificationTime", status.getModificationTime()); + m.put("blockSize", status.getBlockSize()); + m.put("replication", status.getReplication()); + } + return JSON.toString(m); + } + + @SuppressWarnings("unchecked") + static Map parse(String jsonString) { + return (Map) JSON.parse(jsonString); + } + + /** Convert a Json string to a HdfsFileStatus object. */ + public static HdfsFileStatus toFileStatus(final Map m) { + if ((Boolean)m.get("isNull")) { + return null; + } + + final String localName = (String) m.get("localName"); + final boolean isDir = (Boolean) m.get("isDir"); + final boolean isSymlink = (Boolean) m.get("isSymlink"); + final byte[] symlink = isSymlink? + DFSUtil.string2Bytes((String)m.get("symlink")): null; + + final long len = (Long) m.get("len"); + final String owner = (String) m.get("owner"); + final String group = (String) m.get("group"); + final FsPermission permission = toFsPermission((String) m.get("permission")); + final long aTime = (Long) m.get("accessTime"); + final long mTime = (Long) m.get("modificationTime"); + final long blockSize = (Long) m.get("blockSize"); + final short replication = (short) (long) (Long) m.get("replication"); + return new HdfsFileStatus(len, isDir, replication, blockSize, mTime, aTime, + permission, owner, group, + symlink, DFSUtil.string2Bytes(localName)); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java new file mode 100644 index 0000000000..72d86a011c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -0,0 +1,386 @@ +/** + * 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.web; + +import java.io.BufferedOutputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Options; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.ByteRangeInputStream; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.HftpFileSystem; +import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; +import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.server.namenode.SafeModeException; +import org.apache.hadoop.hdfs.web.resources.AccessTimeParam; +import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; +import org.apache.hadoop.hdfs.web.resources.BufferSizeParam; +import org.apache.hadoop.hdfs.web.resources.DeleteOpParam; +import org.apache.hadoop.hdfs.web.resources.DstPathParam; +import org.apache.hadoop.hdfs.web.resources.GetOpParam; +import org.apache.hadoop.hdfs.web.resources.GroupParam; +import org.apache.hadoop.hdfs.web.resources.HttpOpParam; +import org.apache.hadoop.hdfs.web.resources.ModificationTimeParam; +import org.apache.hadoop.hdfs.web.resources.OverwriteParam; +import org.apache.hadoop.hdfs.web.resources.OwnerParam; +import org.apache.hadoop.hdfs.web.resources.Param; +import org.apache.hadoop.hdfs.web.resources.PermissionParam; +import org.apache.hadoop.hdfs.web.resources.PostOpParam; +import org.apache.hadoop.hdfs.web.resources.PutOpParam; +import org.apache.hadoop.hdfs.web.resources.RecursiveParam; +import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam; +import org.apache.hadoop.hdfs.web.resources.ReplicationParam; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.security.AccessControlException; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Progressable; +import org.mortbay.util.ajax.JSON; + +/** A FileSystem for HDFS over the web. */ +public class WebHdfsFileSystem extends HftpFileSystem { + /** File System URI: {SCHEME}://namenode:port/path/to/file */ + public static final String SCHEME = "webhdfs"; + /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */ + public static final String PATH_PREFIX = SCHEME; + + private UserGroupInformation ugi; + protected Path workingDir; + + @Override + public void initialize(URI uri, Configuration conf) throws IOException { + super.initialize(uri, conf); + setConf(conf); + + ugi = UserGroupInformation.getCurrentUser(); + this.workingDir = getHomeDirectory(); + } + + @Override + public URI getUri() { + try { + return new URI(SCHEME, null, nnAddr.getHostName(), nnAddr.getPort(), + null, null, null); + } catch (URISyntaxException e) { + return null; + } + } + + @Override + public Path getHomeDirectory() { + return makeQualified(new Path("/user/" + ugi.getShortUserName())); + } + + @Override + public synchronized Path getWorkingDirectory() { + return workingDir; + } + + @Override + public synchronized void setWorkingDirectory(final Path dir) { + String result = makeAbsolute(dir).toUri().getPath(); + if (!DFSUtil.isValidName(result)) { + throw new IllegalArgumentException("Invalid DFS directory name " + + result); + } + workingDir = makeAbsolute(dir); + } + + private Path makeAbsolute(Path f) { + return f.isAbsolute()? f: new Path(workingDir, f); + } + + @SuppressWarnings("unchecked") + private static T jsonParse(final InputStream in) throws IOException { + if (in == null) { + throw new IOException("The input stream is null."); + } + return (T)JSON.parse(new InputStreamReader(in)); + } + + private static void validateResponse(final HttpOpParam.Op op, + final HttpURLConnection conn) throws IOException { + final int code = conn.getResponseCode(); + if (code != op.getExpectedHttpResponseCode()) { + final Map m; + try { + m = jsonParse(conn.getErrorStream()); + } catch(IOException e) { + throw new IOException("Unexpected HTTP response: code=" + code + " != " + + op.getExpectedHttpResponseCode() + ", " + op.toQueryString() + + ", message=" + conn.getResponseMessage(), e); + } + + final RemoteException re = JsonUtil.toRemoteException(m); + throw re.unwrapRemoteException(AccessControlException.class, + DSQuotaExceededException.class, + FileAlreadyExistsException.class, + FileNotFoundException.class, + ParentNotDirectoryException.class, + SafeModeException.class, + NSQuotaExceededException.class, + UnresolvedPathException.class); + } + } + + private URL toUrl(final HttpOpParam.Op op, final Path fspath, + final Param... parameters) throws IOException { + //initialize URI path and query + final String path = "/" + PATH_PREFIX + + makeQualified(fspath).toUri().getPath(); + final String query = op.toQueryString() + + Param.toSortedString("&", parameters); + final URL url = getNamenodeURL(path, query); + if (LOG.isTraceEnabled()) { + LOG.trace("url=" + url); + } + return url; + } + + private HttpURLConnection httpConnect(final HttpOpParam.Op op, final Path fspath, + final Param... parameters) throws IOException { + final URL url = toUrl(op, fspath, parameters); + + //connect and get response + final HttpURLConnection conn = (HttpURLConnection)url.openConnection(); + try { + conn.setRequestMethod(op.getType().toString()); + conn.setDoOutput(op.getDoOutput()); + if (op.getDoOutput()) { + conn.setRequestProperty("Expect", "100-Continue"); + conn.setInstanceFollowRedirects(true); + } + conn.connect(); + return conn; + } catch(IOException e) { + conn.disconnect(); + throw e; + } + } + + /** + * Run a http operation. + * Connect to the http server, validate response, and obtain the JSON output. + * + * @param op http operation + * @param fspath file system path + * @param parameters parameters for the operation + * @return a JSON object, e.g. Object[], Map, etc. + * @throws IOException + */ + private T run(final HttpOpParam.Op op, final Path fspath, + final Param... parameters) throws IOException { + final HttpURLConnection conn = httpConnect(op, fspath, parameters); + validateResponse(op, conn); + try { + return jsonParse(conn.getInputStream()); + } finally { + conn.disconnect(); + } + } + + private FsPermission applyUMask(FsPermission permission) { + if (permission == null) { + permission = FsPermission.getDefault(); + } + return permission.applyUMask(FsPermission.getUMask(getConf())); + } + + private HdfsFileStatus getHdfsFileStatus(Path f) throws IOException { + final HttpOpParam.Op op = GetOpParam.Op.GETFILESTATUS; + final Map json = run(op, f); + final HdfsFileStatus status = JsonUtil.toFileStatus(json); + if (status == null) { + throw new FileNotFoundException("File does not exist: " + f); + } + return status; + } + + @Override + public FileStatus getFileStatus(Path f) throws IOException { + statistics.incrementReadOps(1); + return makeQualified(getHdfsFileStatus(f), f); + } + + private FileStatus makeQualified(HdfsFileStatus f, Path parent) { + return new FileStatus(f.getLen(), f.isDir(), f.getReplication(), + f.getBlockSize(), f.getModificationTime(), + f.getAccessTime(), + f.getPermission(), f.getOwner(), f.getGroup(), + f.getFullPath(parent).makeQualified(getUri(), getWorkingDirectory())); + } + + @Override + public boolean mkdirs(Path f, FsPermission permission) throws IOException { + statistics.incrementWriteOps(1); + final HttpOpParam.Op op = PutOpParam.Op.MKDIRS; + final Map json = run(op, f, + new PermissionParam(applyUMask(permission))); + return (Boolean)json.get(op.toString()); + } + + @Override + public boolean rename(final Path src, final Path dst) throws IOException { + statistics.incrementWriteOps(1); + final HttpOpParam.Op op = PutOpParam.Op.RENAME; + final Map json = run(op, src, + new DstPathParam(makeQualified(dst).toUri().getPath())); + return (Boolean)json.get(op.toString()); + } + + @SuppressWarnings("deprecation") + @Override + public void rename(final Path src, final Path dst, + final Options.Rename... options) throws IOException { + statistics.incrementWriteOps(1); + final HttpOpParam.Op op = PutOpParam.Op.RENAME; + run(op, src, new DstPathParam(makeQualified(dst).toUri().getPath()), + new RenameOptionSetParam(options)); + } + + @Override + public void setOwner(final Path p, final String owner, final String group + ) throws IOException { + if (owner == null && group == null) { + throw new IOException("owner == null && group == null"); + } + + statistics.incrementWriteOps(1); + final HttpOpParam.Op op = PutOpParam.Op.SETOWNER; + run(op, p, new OwnerParam(owner), new GroupParam(group)); + } + + @Override + public void setPermission(final Path p, final FsPermission permission + ) throws IOException { + statistics.incrementWriteOps(1); + final HttpOpParam.Op op = PutOpParam.Op.SETPERMISSION; + run(op, p, new PermissionParam(permission)); + } + + @Override + public boolean setReplication(final Path p, final short replication + ) throws IOException { + statistics.incrementWriteOps(1); + final HttpOpParam.Op op = PutOpParam.Op.SETREPLICATION; + final Map json = run(op, p, + new ReplicationParam(replication)); + return (Boolean)json.get(op.toString()); + } + + @Override + public void setTimes(final Path p, final long mtime, final long atime + ) throws IOException { + statistics.incrementWriteOps(1); + final HttpOpParam.Op op = PutOpParam.Op.SETTIMES; + run(op, p, new ModificationTimeParam(mtime), new AccessTimeParam(atime)); + } + + private FSDataOutputStream write(final HttpOpParam.Op op, + final HttpURLConnection conn, final int bufferSize) throws IOException { + return new FSDataOutputStream(new BufferedOutputStream( + conn.getOutputStream(), bufferSize), statistics) { + @Override + public void close() throws IOException { + try { + super.close(); + } finally { + validateResponse(op, conn); + } + } + }; + } + + @Override + public FSDataOutputStream create(final Path f, final FsPermission permission, + final boolean overwrite, final int bufferSize, final short replication, + final long blockSize, final Progressable progress) throws IOException { + statistics.incrementWriteOps(1); + + final HttpOpParam.Op op = PutOpParam.Op.CREATE; + final HttpURLConnection conn = httpConnect(op, f, + new PermissionParam(applyUMask(permission)), + new OverwriteParam(overwrite), + new BufferSizeParam(bufferSize), + new ReplicationParam(replication), + new BlockSizeParam(blockSize)); + return write(op, conn, bufferSize); + } + + @Override + public FSDataOutputStream append(final Path f, final int bufferSize, + final Progressable progress) throws IOException { + statistics.incrementWriteOps(1); + + final HttpOpParam.Op op = PostOpParam.Op.APPEND; + final HttpURLConnection conn = httpConnect(op, f, + new BufferSizeParam(bufferSize)); + return write(op, conn, bufferSize); + } + + @Override + public boolean delete(Path f, boolean recursive) throws IOException { + final HttpOpParam.Op op = DeleteOpParam.Op.DELETE; + final Map json = run(op, f, new RecursiveParam(recursive)); + return (Boolean)json.get(op.toString()); + } + + @Override + public FSDataInputStream open(final Path f, final int buffersize + ) throws IOException { + statistics.incrementReadOps(1); + final HttpOpParam.Op op = GetOpParam.Op.OPEN; + final URL url = toUrl(op, f, new BufferSizeParam(buffersize)); + return new FSDataInputStream(new ByteRangeInputStream(url)); + } + + @Override + public FileStatus[] listStatus(final Path f) throws IOException { + statistics.incrementReadOps(1); + + final HttpOpParam.Op op = GetOpParam.Op.LISTSTATUS; + final Object[] array = run(op, f); + + //convert FileStatus + final FileStatus[] statuses = new FileStatus[array.length]; + for(int i = 0; i < array.length; i++) { + @SuppressWarnings("unchecked") + final Map m = (Map)array[i]; + statuses[i] = makeQualified(JsonUtil.toFileStatus(m), f); + } + return statuses; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java new file mode 100644 index 0000000000..830e5cd32d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AccessTimeParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Access time parameter. */ +public class AccessTimeParam extends LongParam { + /** Parameter name. */ + public static final String NAME = "accessTime"; + /** Default parameter value. */ + public static final String DEFAULT = "-1"; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public AccessTimeParam(final Long value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public AccessTimeParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java new file mode 100644 index 0000000000..0f83519e55 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BlockSizeParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Block size parameter. */ +public class BlockSizeParam extends LongParam { + /** Parameter name. */ + public static final String NAME = "blockSize"; + /** Default parameter value. */ + public static final String DEFAULT = NULL; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public BlockSizeParam(final Long value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public BlockSizeParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java new file mode 100644 index 0000000000..14dfdf5334 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BooleanParam.java @@ -0,0 +1,51 @@ +/** + * 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.web.resources; + +/** Boolean parameter. */ +abstract class BooleanParam extends Param { + static final String TRUE = "true"; + static final String FALSE = "false"; + + BooleanParam(final Domain domain, final Boolean value) { + super(domain, value); + } + + /** The domain of the parameter. */ + static final class Domain extends Param.Domain { + Domain(final String paramName) { + super(paramName); + } + + @Override + public String getDomain() { + return "<" + NULL + " | boolean>"; + } + + @Override + Boolean parse(final String str) { + if (TRUE.equalsIgnoreCase(str)) { + return true; + } else if (FALSE.equalsIgnoreCase(str)) { + return false; + } + throw new IllegalArgumentException("Failed to parse \"" + str + + "\" to Boolean."); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BufferSizeParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BufferSizeParam.java new file mode 100644 index 0000000000..bc2e74118d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/BufferSizeParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Buffer size parameter. */ +public class BufferSizeParam extends IntegerParam { + /** Parameter name. */ + public static final String NAME = "bufferSize"; + /** Default parameter value. */ + public static final String DEFAULT = NULL; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public BufferSizeParam(final Integer value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public BufferSizeParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java new file mode 100644 index 0000000000..e61e858ee4 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java @@ -0,0 +1,74 @@ +/** + * 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.web.resources; + +import java.net.HttpURLConnection; + +/** Http DELETE operation parameter. */ +public class DeleteOpParam extends HttpOpParam { + /** Parameter name. */ + public static final String NAME = "deleteOp"; + + /** Delete operations. */ + public static enum Op implements HttpOpParam.Op { + DELETE(HttpURLConnection.HTTP_OK), + + NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED); + + final int expectedHttpResponseCode; + + Op(final int expectedHttpResponseCode) { + this.expectedHttpResponseCode = expectedHttpResponseCode; + } + + @Override + public HttpOpParam.Type getType() { + return HttpOpParam.Type.DELETE; + } + + @Override + public boolean getDoOutput() { + return false; + } + + @Override + public int getExpectedHttpResponseCode() { + return expectedHttpResponseCode; + } + + @Override + public String toQueryString() { + return NAME + "=" + this; + } + } + + private static final Domain DOMAIN = new Domain(NAME, Op.class); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public DeleteOpParam(final String str) { + super(DOMAIN, DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DstPathParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DstPathParam.java new file mode 100644 index 0000000000..7d522a3877 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DstPathParam.java @@ -0,0 +1,43 @@ +/** + * 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.web.resources; + +import org.apache.hadoop.fs.Path; + +/** Destination path parameter. */ +public class DstPathParam extends StringParam { + /** Parameter name. */ + public static final String NAME = "dstPath"; + /** Default parameter value. */ + public static final String DEFAULT = ""; + + private static final Domain DOMAIN = new Domain(NAME, null); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public DstPathParam(final String str) { + super(DOMAIN, str == null || str.equals(DEFAULT)? null: new Path(str).toUri().getPath()); + } + + @Override + public String getName() { + return NAME; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java new file mode 100644 index 0000000000..1703e3b25d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumParam.java @@ -0,0 +1,46 @@ +/** + * 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.web.resources; + +import java.util.Arrays; + +abstract class EnumParam> extends Param> { + EnumParam(final Domain domain, final E value) { + super(domain, value); + } + + /** The domain of the parameter. */ + static final class Domain> extends Param.Domain { + private final Class enumClass; + + Domain(String name, final Class enumClass) { + super(name); + this.enumClass = enumClass; + } + + @Override + public final String getDomain() { + return Arrays.asList(enumClass.getEnumConstants()).toString(); + } + + @Override + final E parse(final String str) { + return Enum.valueOf(enumClass, str.toUpperCase()); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java new file mode 100644 index 0000000000..feb4128e51 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/EnumSetParam.java @@ -0,0 +1,86 @@ +/** + * 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.web.resources; + +import java.util.Arrays; +import java.util.EnumSet; +import java.util.Iterator; + +abstract class EnumSetParam> extends Param, EnumSetParam.Domain> { + /** Convert an EnumSet to a string of comma separated values. */ + static > String toString(EnumSet set) { + if (set == null || set.isEmpty()) { + return ""; + } else { + final StringBuilder b = new StringBuilder(); + final Iterator i = set.iterator(); + b.append(i.next()); + for(; i.hasNext(); ) { + b.append(',').append(i.next()); + } + return b.toString(); + } + } + + static > EnumSet toEnumSet(final Class clazz, + final E... values) { + final EnumSet set = EnumSet.noneOf(clazz); + set.addAll(Arrays.asList(values)); + return set; + } + + EnumSetParam(final Domain domain, final EnumSet value) { + super(domain, value); + } + + @Override + public String toString() { + return getName() + "=" + toString(value); + } + + + /** The domain of the parameter. */ + static final class Domain> extends Param.Domain> { + private final Class enumClass; + + Domain(String name, final Class enumClass) { + super(name); + this.enumClass = enumClass; + } + + @Override + public final String getDomain() { + return Arrays.asList(enumClass.getEnumConstants()).toString(); + } + + /** The string contains a comma separated values. */ + @Override + final EnumSet parse(final String str) { + final EnumSet set = EnumSet.noneOf(enumClass); + if (!str.isEmpty()) { + for(int i, j = 0; j >= 0; ) { + i = j; + j = str.indexOf(',', i+1); + final String sub = j >= 0? str.substring(i, j): str.substring(i); + set.add(Enum.valueOf(enumClass, sub.trim().toUpperCase())); + } + } + return set; + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java new file mode 100644 index 0000000000..8a04c4ad91 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java @@ -0,0 +1,59 @@ +/** + * 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.web.resources; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.ext.ExceptionMapper; +import javax.ws.rs.ext.Provider; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hdfs.web.JsonUtil; + +/** Handle exceptions. */ +@Provider +public class ExceptionHandler implements ExceptionMapper { + public static final Log LOG = LogFactory.getLog(ExceptionHandler.class); + + @Override + public Response toResponse(final Exception e) { + if (LOG.isTraceEnabled()) { + LOG.trace("GOT EXCEPITION", e); + } + + final Response.Status s; + if (e instanceof SecurityException) { + s = Response.Status.UNAUTHORIZED; + } else if (e instanceof FileNotFoundException) { + s = Response.Status.NOT_FOUND; + } else if (e instanceof IOException) { + s = Response.Status.FORBIDDEN; + } else if (e instanceof UnsupportedOperationException) { + s = Response.Status.BAD_REQUEST; + } else { + s = Response.Status.INTERNAL_SERVER_ERROR; + } + + final String js = JsonUtil.toJsonString(e); + return Response.status(s).type(MediaType.APPLICATION_JSON).entity(js).build(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java new file mode 100644 index 0000000000..6f11871ebb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java @@ -0,0 +1,77 @@ +/** + * 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.web.resources; + +import java.net.HttpURLConnection; + +/** Http GET operation parameter. */ +public class GetOpParam extends HttpOpParam { + /** Parameter name. */ + public static final String NAME = "getOp"; + + /** Get operations. */ + public static enum Op implements HttpOpParam.Op { + OPEN(HttpURLConnection.HTTP_OK), + + GETFILESTATUS(HttpURLConnection.HTTP_OK), + LISTSTATUS(HttpURLConnection.HTTP_OK), + + NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED); + + final int expectedHttpResponseCode; + + Op(final int expectedHttpResponseCode) { + this.expectedHttpResponseCode = expectedHttpResponseCode; + } + + @Override + public HttpOpParam.Type getType() { + return HttpOpParam.Type.GET; + } + + @Override + public boolean getDoOutput() { + return false; + } + + @Override + public int getExpectedHttpResponseCode() { + return expectedHttpResponseCode; + } + + @Override + public String toQueryString() { + return NAME + "=" + this; + } + } + + private static final Domain DOMAIN = new Domain(NAME, Op.class); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public GetOpParam(final String str) { + super(DOMAIN, DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java new file mode 100644 index 0000000000..c0429cc3d2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GroupParam.java @@ -0,0 +1,41 @@ +/** + * 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.web.resources; + +/** Group parameter. */ +public class GroupParam extends StringParam { + /** Parameter name. */ + public static final String NAME = "group"; + /** Default parameter value. */ + public static final String DEFAULT = ""; + + private static final Domain DOMAIN = new Domain(NAME, null); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public GroupParam(final String str) { + super(DOMAIN, str == null || str.equals(DEFAULT)? null: str); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java new file mode 100644 index 0000000000..204e15b7c0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java @@ -0,0 +1,52 @@ +/** + * 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.web.resources; + +/** Http operation parameter. */ +public abstract class HttpOpParam & HttpOpParam.Op> extends EnumParam { + /** Default parameter value. */ + public static final String DEFAULT = NULL; + + /** Http operation types */ + public static enum Type { + GET, PUT, POST, DELETE; + } + + /** Http operation interface. */ + public static interface Op { + /** @return the Http operation type. */ + public Type getType(); + + /** @return true if the operation has output. */ + public boolean getDoOutput(); + + /** @return true if the operation has output. */ + public int getExpectedHttpResponseCode(); + + /** @return a URI query string. */ + public String toQueryString(); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + HttpOpParam(final Domain domain, final E value) { + super(domain, value); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/IntegerParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/IntegerParam.java new file mode 100644 index 0000000000..5e89087610 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/IntegerParam.java @@ -0,0 +1,60 @@ +/** + * 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.web.resources; + +/** Integer parameter. */ +abstract class IntegerParam extends Param { + IntegerParam(final Domain domain, final Integer value) { + super(domain, value); + } + + @Override + public String toString() { + return getName() + "=" + domain.toString(getValue()); + } + + /** The domain of the parameter. */ + static final class Domain extends Param.Domain { + /** The radix of the number. */ + final int radix; + + Domain(final String paramName) { + this(paramName, 10); + } + + Domain(final String paramName, final int radix) { + super(paramName); + this.radix = radix; + } + + @Override + public String getDomain() { + return "<" + NULL + " | int in radix " + radix + ">"; + } + + @Override + Integer parse(final String str) { + return NULL.equals(str)? null: Integer.parseInt(str, radix); + } + + /** Convert an Integer to a String. */ + String toString(final Integer n) { + return n == null? NULL: Integer.toString(n, radix); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/LengthParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/LengthParam.java new file mode 100644 index 0000000000..90d4f6289d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/LengthParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Length parameter. */ +public class LengthParam extends LongParam { + /** Parameter name. */ + public static final String NAME = "length"; + /** Default parameter value. */ + public static final String DEFAULT = NULL; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public LengthParam(final Long value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public LengthParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/LongParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/LongParam.java new file mode 100644 index 0000000000..8a3e0f5e41 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/LongParam.java @@ -0,0 +1,60 @@ +/** + * 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.web.resources; + +/** Long parameter. */ +abstract class LongParam extends Param { + LongParam(final Domain domain, final Long value) { + super(domain, value); + } + + @Override + public String toString() { + return getName() + "=" + domain.toString(getValue()); + } + + /** The domain of the parameter. */ + static final class Domain extends Param.Domain { + /** The radix of the number. */ + final int radix; + + Domain(final String paramName) { + this(paramName, 10); + } + + Domain(final String paramName, final int radix) { + super(paramName); + this.radix = radix; + } + + @Override + public String getDomain() { + return "<" + NULL + " | short in radix " + radix + ">"; + } + + @Override + Long parse(final String str) { + return NULL.equals(str)? null: Long.parseLong(str, radix); + } + + /** Convert a Short to a String. */ + String toString(final Long n) { + return n == null? NULL: Long.toString(n, radix); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ModificationTimeParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ModificationTimeParam.java new file mode 100644 index 0000000000..d43da07328 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ModificationTimeParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Modification time parameter. */ +public class ModificationTimeParam extends LongParam { + /** Parameter name. */ + public static final String NAME = "modificationTime"; + /** Default parameter value. */ + public static final String DEFAULT = "-1"; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public ModificationTimeParam(final Long value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public ModificationTimeParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OffsetParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OffsetParam.java new file mode 100644 index 0000000000..8b3654dbd8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OffsetParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Offset parameter. */ +public class OffsetParam extends LongParam { + /** Parameter name. */ + public static final String NAME = "offset"; + /** Default parameter value. */ + public static final String DEFAULT = "0"; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public OffsetParam(final Long value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public OffsetParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java new file mode 100644 index 0000000000..6639ece7b2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OverwriteParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Recursive parameter. */ +public class OverwriteParam extends BooleanParam { + /** Parameter name. */ + public static final String NAME = "overwrite"; + /** Default parameter value. */ + public static final String DEFAULT = FALSE; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public OverwriteParam(final Boolean value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public OverwriteParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OwnerParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OwnerParam.java new file mode 100644 index 0000000000..a1c10aac6f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/OwnerParam.java @@ -0,0 +1,41 @@ +/** + * 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.web.resources; + +/** Owner parameter. */ +public class OwnerParam extends StringParam { + /** Parameter name. */ + public static final String NAME = "owner"; + /** Default parameter value. */ + public static final String DEFAULT = ""; + + private static final Domain DOMAIN = new Domain(NAME, null); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public OwnerParam(final String str) { + super(DOMAIN, str == null || str.equals(DEFAULT)? null: str); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/Param.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/Param.java new file mode 100644 index 0000000000..b5fd1da241 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/Param.java @@ -0,0 +1,104 @@ +/** + * 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.web.resources; + +import java.util.Arrays; +import java.util.Comparator; + + +/** Base class of parameters. */ +public abstract class Param> { + static final String NULL = "null"; + + static final Comparator> NAME_CMP = new Comparator>() { + @Override + public int compare(Param left, Param right) { + return left.getName().compareTo(right.getName()); + } + }; + + /** Convert the parameters to a sorted String. */ + public static String toSortedString(final String separator, + final Param... parameters) { + Arrays.sort(parameters, NAME_CMP); + final StringBuilder b = new StringBuilder(); + for(Param p : parameters) { + if (p.getValue() != null) { + b.append(separator).append(p); + } + } + return b.toString(); + } + + /** The domain of the parameter. */ + final D domain; + /** The actual parameter value. */ + final T value; + + Param(final D domain, final T value) { + this.domain = domain; + this.value = value; + } + + /** @return the parameter value. */ + public final T getValue() { + return value; + } + + /** @return the parameter name. */ + public abstract String getName(); + + @Override + public String toString() { + return getName() + "=" + value; + } + + /** Base class of parameter domains. */ + static abstract class Domain { + /** Parameter name. */ + final String paramName; + + Domain(final String paramName) { + this.paramName = paramName; + } + + /** @return the parameter name. */ + public final String getParamName() { + return paramName; + } + + /** @return a string description of the domain of the parameter. */ + public abstract String getDomain(); + + /** @return the parameter value represented by the string. */ + abstract T parse(String str); + + /** Parse the given string. + * @return the parameter value represented by the string. + */ + public final T parse(final String varName, final String str) { + try { + return str != null && str.trim().length() > 0 ? parse(str) : null; + } catch(Exception e) { + throw new IllegalArgumentException("Failed to parse \"" + str + + "\" for the parameter " + varName + + ". The value must be in the domain " + getDomain(), e); + } + } + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PermissionParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PermissionParam.java new file mode 100644 index 0000000000..264e60226b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PermissionParam.java @@ -0,0 +1,57 @@ +/** + * 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.web.resources; + +import org.apache.hadoop.fs.permission.FsPermission; + +/** Permission parameter, use a Short to represent a FsPermission. */ +public class PermissionParam extends ShortParam { + /** Parameter name. */ + public static final String NAME = "permission"; + /** Default parameter value. */ + public static final String DEFAULT = NULL; + + private static final Domain DOMAIN = new Domain(NAME, 8); + + /** + * Constructor. + * @param value the parameter value. + */ + public PermissionParam(final FsPermission value) { + super(DOMAIN, value == null? null: value.toShort()); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public PermissionParam(final String str) { + super(DOMAIN, DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } + + /** @return the represented FsPermission. */ + public FsPermission getFsPermission() { + final Short mode = getValue(); + return mode == null? FsPermission.getDefault(): new FsPermission(mode); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java new file mode 100644 index 0000000000..116d6af8b3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java @@ -0,0 +1,74 @@ +/** + * 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.web.resources; + +import java.net.HttpURLConnection; + +/** Http POST operation parameter. */ +public class PostOpParam extends HttpOpParam { + /** Parameter name. */ + public static final String NAME = "postOp"; + + /** Post operations. */ + public static enum Op implements HttpOpParam.Op { + APPEND(HttpURLConnection.HTTP_OK), + + NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED); + + final int expectedHttpResponseCode; + + Op(final int expectedHttpResponseCode) { + this.expectedHttpResponseCode = expectedHttpResponseCode; + } + + @Override + public Type getType() { + return Type.POST; + } + + @Override + public boolean getDoOutput() { + return true; + } + + @Override + public int getExpectedHttpResponseCode() { + return expectedHttpResponseCode; + } + + /** @return a URI query string. */ + public String toQueryString() { + return NAME + "=" + this; + } + } + + private static final Domain DOMAIN = new Domain(NAME, Op.class); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public PostOpParam(final String str) { + super(DOMAIN, DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java new file mode 100644 index 0000000000..00703fefbc --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java @@ -0,0 +1,84 @@ +/** + * 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.web.resources; + +import java.net.HttpURLConnection; + +/** Http POST operation parameter. */ +public class PutOpParam extends HttpOpParam { + /** Parameter name. */ + public static final String NAME = "putOp"; + + /** Put operations. */ + public static enum Op implements HttpOpParam.Op { + CREATE(true, HttpURLConnection.HTTP_CREATED), + + MKDIRS(false, HttpURLConnection.HTTP_OK), + RENAME(false, HttpURLConnection.HTTP_OK), + SETREPLICATION(false, HttpURLConnection.HTTP_OK), + + SETOWNER(false, HttpURLConnection.HTTP_OK), + SETPERMISSION(false, HttpURLConnection.HTTP_OK), + SETTIMES(false, HttpURLConnection.HTTP_OK), + + NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED); + + final boolean doOutput; + final int expectedHttpResponseCode; + + Op(final boolean doOutput, final int expectedHttpResponseCode) { + this.doOutput = doOutput; + this.expectedHttpResponseCode = expectedHttpResponseCode; + } + + @Override + public HttpOpParam.Type getType() { + return HttpOpParam.Type.PUT; + } + + @Override + public boolean getDoOutput() { + return doOutput; + } + + @Override + public int getExpectedHttpResponseCode() { + return expectedHttpResponseCode; + } + + @Override + public String toQueryString() { + return NAME + "=" + this; + } + } + + private static final Domain DOMAIN = new Domain(NAME, Op.class); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public PutOpParam(final String str) { + super(DOMAIN, DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java new file mode 100644 index 0000000000..4890a615f1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RecursiveParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Recursive parameter. */ +public class RecursiveParam extends BooleanParam { + /** Parameter name. */ + public static final String NAME = "recursive"; + /** Default parameter value. */ + public static final String DEFAULT = FALSE; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public RecursiveParam(final Boolean value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public RecursiveParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java new file mode 100644 index 0000000000..ec66a51c78 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RenameOptionSetParam.java @@ -0,0 +1,52 @@ +/** + * 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.web.resources; + +import org.apache.hadoop.fs.Options; + +/** Rename option set parameter. */ +public class RenameOptionSetParam extends EnumSetParam { + /** Parameter name. */ + public static final String NAME = "renameOptions"; + /** Default parameter value. */ + public static final String DEFAULT = ""; + + private static final Domain DOMAIN = new Domain( + NAME, Options.Rename.class); + + /** + * Constructor. + * @param options rename options. + */ + public RenameOptionSetParam(final Options.Rename... options) { + super(DOMAIN, toEnumSet(Options.Rename.class, options)); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public RenameOptionSetParam(final String str) { + super(DOMAIN, DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java new file mode 100644 index 0000000000..e13aec8115 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ReplicationParam.java @@ -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. + */ +package org.apache.hadoop.hdfs.web.resources; + +/** Replication parameter. */ +public class ReplicationParam extends ShortParam { + /** Parameter name. */ + public static final String NAME = "replication"; + /** Default parameter value. */ + public static final String DEFAULT = NULL; + + private static final Domain DOMAIN = new Domain(NAME); + + /** + * Constructor. + * @param value the parameter value. + */ + public ReplicationParam(final Short value) { + super(DOMAIN, value); + } + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public ReplicationParam(final String str) { + this(DOMAIN.parse(str)); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java new file mode 100644 index 0000000000..af3e72f687 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ShortParam.java @@ -0,0 +1,60 @@ +/** + * 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.web.resources; + +/** Short parameter. */ +abstract class ShortParam extends Param { + ShortParam(final Domain domain, final Short value) { + super(domain, value); + } + + @Override + public String toString() { + return getName() + "=" + domain.toString(getValue()); + } + + /** The domain of the parameter. */ + static final class Domain extends Param.Domain { + /** The radix of the number. */ + final int radix; + + Domain(final String paramName) { + this(paramName, 10); + } + + Domain(final String paramName, final int radix) { + super(paramName); + this.radix = radix; + } + + @Override + public String getDomain() { + return "<" + NULL + " | short in radix " + radix + ">"; + } + + @Override + Short parse(final String str) { + return NULL.equals(str)? null: Short.parseShort(str, radix); + } + + /** Convert a Short to a String. */ + String toString(final Short n) { + return n == null? NULL: Integer.toString(n, radix); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java new file mode 100644 index 0000000000..d4303f1407 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/StringParam.java @@ -0,0 +1,54 @@ +/** + * 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.web.resources; + +import java.util.regex.Pattern; + +/** String parameter. */ +abstract class StringParam extends Param { + StringParam(final Domain domain, String str) { + super(domain, domain.parse(str)); + } + + /** The domain of the parameter. */ + static final class Domain extends Param.Domain { + /** The pattern defining the domain; null . */ + private final Pattern pattern; + + Domain(final String paramName, final Pattern pattern) { + super(paramName); + this.pattern = pattern; + } + + @Override + public final String getDomain() { + return pattern == null ? "" : pattern.pattern(); + } + + @Override + final String parse(final String str) { + if (pattern != null) { + if (!pattern.matcher(str).matches()) { + throw new IllegalArgumentException("Invalid value: \"" + str + + "\" does not belong to the domain " + getDomain()); + } + } + return str; + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java new file mode 100644 index 0000000000..2e12697076 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UriFsPathParam.java @@ -0,0 +1,45 @@ +/** + * 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.web.resources; + +/** The FileSystem path parameter. */ +public class UriFsPathParam extends StringParam { + /** Parameter name. */ + public static final String NAME = "path"; + + private static final Domain DOMAIN = new Domain(NAME, null); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public UriFsPathParam(String str) { + super(DOMAIN, str); + } + + @Override + public String getName() { + return NAME; + } + + /** @return the absolute path. */ + public final String getAbsolutePath() { + final String path = getValue(); //The first / has been stripped out. + return path == null? null: "/" + path; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java new file mode 100644 index 0000000000..ec056628c9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java @@ -0,0 +1,41 @@ +/** + * 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.web.resources; + +/** User parameter. */ +public class UserParam extends StringParam { + /** Parameter name. */ + public static final String NAME = "user.name"; + /** Default parameter value. */ + public static final String DEFAULT = ""; + + private static final Domain DOMAIN = new Domain(NAME, null); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public UserParam(final String str) { + super(DOMAIN, str == null || str.equals(DEFAULT)? null: str); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java new file mode 100644 index 0000000000..43c66e4186 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java @@ -0,0 +1,73 @@ +/** + * 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.web.resources; + +import java.lang.reflect.Type; +import java.security.Principal; + +import javax.ws.rs.core.Context; +import javax.ws.rs.ext.Provider; + +import com.sun.jersey.api.core.HttpContext; +import com.sun.jersey.core.spi.component.ComponentContext; +import com.sun.jersey.core.spi.component.ComponentScope; +import com.sun.jersey.server.impl.inject.AbstractHttpContextInjectable; +import com.sun.jersey.spi.inject.Injectable; +import com.sun.jersey.spi.inject.InjectableProvider; + +@Provider +public class UserProvider extends AbstractHttpContextInjectable + implements InjectableProvider { + + @Override + public Principal getValue(final HttpContext context) { + //get principal from the request + final Principal principal = context.getRequest().getUserPrincipal(); + if (principal != null) { + return principal; + } + + //get username from the parameter + final String username = context.getRequest().getQueryParameters().getFirst( + UserParam.NAME); + if (username != null) { + final UserParam userparam = new UserParam(username); + return new Principal() { + @Override + public String getName() { + return userparam.getValue(); + } + }; + } + + //user not found + return null; + } + + @Override + public ComponentScope getScope() { + return ComponentScope.PerRequest; + } + + @Override + public Injectable getInjectable( + final ComponentContext componentContext, final Context context, + final Type type) { + return type.equals(Principal.class)? this : null; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index e51401cfc0..8ebae4e57a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -815,6 +815,8 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, long[] simulatedCapacities, boolean setupHostsFile, boolean checkDataNodeAddrConfig) throws IOException { + conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1"); + int curDatanodesNum = dataNodes.size(); // for mincluster's the default initialDelay for BRs is 0 if (conf.get(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java new file mode 100644 index 0000000000..1addbf00de --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.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.web; + +import java.net.URI; + +import org.apache.commons.logging.impl.Log4JLogger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSMainOperationsBaseTest; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods; +import org.apache.hadoop.hdfs.web.resources.ExceptionHandler; +import org.apache.log4j.Level; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestFSMainOperationsWebHdfs extends FSMainOperationsBaseTest { + { + ((Log4JLogger)ExceptionHandler.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)DatanodeWebHdfsMethods.LOG).getLogger().setLevel(Level.ALL); + } + + private static MiniDFSCluster cluster = null; + private static Path defaultWorkingDirectory; + + @BeforeClass + public static void setupCluster() { + Configuration conf = new Configuration(); + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + cluster.waitActive(); + + final String uri = WebHdfsFileSystem.SCHEME + "://" + + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY); + fSys = FileSystem.get(new URI(uri), conf); + defaultWorkingDirectory = fSys.getWorkingDirectory(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @AfterClass + public static void shutdownCluster() { + if (cluster != null) { + cluster.shutdown(); + cluster = null; + } + } + + @Override + protected Path getDefaultWorkingDirectory() { + return defaultWorkingDirectory; + } + + //The following test failed since WebHdfsFileSystem did not support + //authentication. + //Disable it. + @Test + public void testListStatusThrowsExceptionForUnreadableDir() {} +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java new file mode 100644 index 0000000000..a2000144f2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java @@ -0,0 +1,55 @@ +/** + * 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.web; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.web.JsonUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestJsonUtil { + static FileStatus toFileStatus(HdfsFileStatus f, String parent) { + return new FileStatus(f.getLen(), f.isDir(), f.getReplication(), + f.getBlockSize(), f.getModificationTime(), f.getAccessTime(), + f.getPermission(), f.getOwner(), f.getGroup(), + new Path(f.getFullName(parent))); + } + + @Test + public void testHdfsFileStatus() { + final long now = System.currentTimeMillis(); + final String parent = "/dir"; + final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L<<26, + now, now + 10, new FsPermission((short)0644), "user", "group", + DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo")); + final FileStatus fstatus = toFileStatus(status, parent); + System.out.println("status = " + status); + System.out.println("fstatus = " + fstatus); + final String json = JsonUtil.toJsonString(status); + System.out.println("json = " + json.replace(",", ",\n ")); + final HdfsFileStatus s2 = JsonUtil.toFileStatus(JsonUtil.parse(json)); + final FileStatus fs2 = toFileStatus(s2, parent); + System.out.println("s2 = " + s2); + System.out.println("fs2 = " + fs2); + Assert.assertEquals(fstatus, fs2); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java new file mode 100644 index 0000000000..39334e2693 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java @@ -0,0 +1,56 @@ +/** + * 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.web; + +import java.io.IOException; +import java.net.URI; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileSystemContractBaseTest; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; + +public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest { + private static final Configuration conf = new Configuration(); + private static final MiniDFSCluster cluster; + private String defaultWorkingDirectory; + + static { + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + cluster.waitActive(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + protected void setUp() throws Exception { + final String uri = WebHdfsFileSystem.SCHEME + "://" + + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY); + fs = FileSystem.get(new URI(uri), conf); + defaultWorkingDirectory = fs.getWorkingDirectory().toUri().getPath(); + } + + @Override + protected String getDefaultWorkingDirectory() { + return defaultWorkingDirectory; + } +} diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 58fa824442..d1364d0fd2 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -2,9 +2,29 @@ Hadoop MapReduce Change Log Trunk (unreleased changes) + INCOMPATIBLE CHANGES + + NEW FEATURES + + MAPREDUCE-2669. Add new examples for Mean, Median, and Standard Deviation. + (Plamen Jeliazkov via shv) + IMPROVEMENTS - MAPREDUCE-2887 due to HADOOP-7524 Change RPC to allow multiple protocols including multuple versions of the same protocol (sanjay Radia) + MAPREDUCE-2887 due to HADOOP-7524 Change RPC to allow multiple protocols + including multuple versions of the same protocol (sanjay Radia) + + MAPREDUCE-2934. MR portion of HADOOP-7607 - Simplify the RPC proxy cleanup + process (atm) + + BUG FIXES + + MAPREDUCE-2784. [Gridmix] Bug fixes in ExecutionSummarizer and + ResourceUsageMatcher. (amarrk) + + MAPREDUCE-2978. Fixed test-patch to make Jenkins report correct number of + findBugs, correct links to findBugs artifacts and no links to the + artifacts when there are no warnings. (Tom White via vinodkv). Release 0.23.0 - Unreleased @@ -248,6 +268,24 @@ Release 0.23.0 - Unreleased MAPREDUCE-2655. Add audit logs to ResourceManager and NodeManager. (Thomas Graves via acmurthy) + MAPREDUCE-2864. Normalize configuration variable names for YARN. (Robert + Evans via acmurthy) + + MAPREDUCE-2690. Web-page for FifoScheduler. (Eric Payne via acmurthy) + + MAPREDUCE-2711. Update TestBlockPlacementPolicyRaid for the new namesystem + and block management APIs. (szetszwo) + + MAPREDUCE-2933. Change allocate call to return ContainerStatus for + completed containers rather than Container. (acmurthy) + + MAPREDUCE-2675. Reformat JobHistory Server main page to be more + useful. (Robert Joseph Evans via vinodkv). + + MAPREDUCE-2896. Simplify all apis to in + org.apache.hadoop.yarn.api.records.* to be get/set only. Added javadocs to + all public records. (acmurthy) + OPTIMIZATIONS MAPREDUCE-2026. Make JobTracker.getJobCounters() and @@ -1221,6 +1259,59 @@ Release 0.23.0 - Unreleased MAPREDUCE-2948. Hadoop streaming test failure, post MR-2767 (mahadev) + MAPREDUCE-2908. Fix all findbugs warnings. (vinodkv via acmurthy) + + MAPREDUCE-2942. TestNMAuditLogger.testNMAuditLoggerWithIP failing (Thomas Graves + via mahadev) + + MAPREDUCE-2947. Fixed race condition in AuxiliaryServices. (vinodkv via + acmurthy) + + MAPREDUCE-2844. Fixed display of nodes in UI. (Ravi Teja Ch N V via + acmurthy) + + MAPREDUCE-2677. Fixed 404 for some links from HistoryServer. (Robert Evans + via acmurthy) + + MAPREDUCE-2937. Ensure reason for application failure is displayed to the + user. (mahadev via acmurthy) + + MAPREDUCE-2953. Fix a race condition on submission which caused client to + incorrectly assume application was gone by making submission synchronous + for RMAppManager. (Thomas Graves via acmurthy) + + MAPREDUCE-2963. Fix hang in TestMRJobs. (Siddharth Seth via acmurthy) + + MAPREDUCE-2954. Fixed a deadlock in NM caused due to wrong synchronization + in protocol buffer records. (Siddharth Seth via vinodkv) + + MAPREDUCE-2975. Fixed YARNRunner to use YarnConfiguration rather than + Configuration. (mahadev via acmurthy) + + MAPREDUCE-2971. ant build mapreduce fails protected access jc.displayJobList + (jobs) (Thomas Graves via mahadev) + + MAPREDUCE-2691. Finishing up the cleanup of distributed cache file resources + and related tests. (Siddharth Seth via vinodkv) + + MAPREDUCE-2749. Ensure NM registers with RM after starting all its services + correctly. (Thomas Graves via acmurthy) + + MAPREDUCE-2979. Removed the needless ClientProtocolProvider configuration + from the hadoop-mapreduce-client-core module. (Siddharth Seth via vinodkv) + + MAPREDUCE-2985. Fixed findbugs warnings in ResourceLocalizationService. + (Thomas Graves via acmurthy) + + MAPREDUCE-2874. Fix formatting of ApplicationId in web-ui. (Eric Payne via + acmurthy) + + MAPREDUCE-2995. Better handling of expired containers in MapReduce + ApplicationMaster. (vinodkv via acmurthy) + + MAPREDUCE-2995. Fixed race condition in ContainerLauncher. (vinodkv via + acmurthy) + Release 0.22.0 - Unreleased INCOMPATIBLE CHANGES diff --git a/hadoop-mapreduce-project/INSTALL b/hadoop-mapreduce-project/INSTALL index ed81fa084b..16db5b6dbe 100644 --- a/hadoop-mapreduce-project/INSTALL +++ b/hadoop-mapreduce-project/INSTALL @@ -56,12 +56,12 @@ export YARN_CONF_DIR=$HADOOP_CONF_DIR Step 9) Setup config: for running mapreduce applications, which now are in user land, you need to setup nodemanager with the following configuration in your yarn-site.xml before you start the nodemanager. - nodemanager.auxiluary.services + yarn.nodemanager.aux-services mapreduce.shuffle - nodemanager.aux.service.mapreduce.shuffle.class + yarn.nodemanager.aux-services.mapreduce.shuffle.class org.apache.hadoop.mapred.ShuffleHandler diff --git a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml index 92b0549a35..0e2c41f732 100644 --- a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml +++ b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml @@ -138,6 +138,11 @@ + + + + + @@ -155,7 +160,8 @@ - + + @@ -166,6 +172,14 @@ + + + + + + + + diff --git a/hadoop-mapreduce-project/dev-support/test-patch.properties b/hadoop-mapreduce-project/dev-support/test-patch.properties index 4d6032d4c5..c59f3895ee 100644 --- a/hadoop-mapreduce-project/dev-support/test-patch.properties +++ b/hadoop-mapreduce-project/dev-support/test-patch.properties @@ -14,5 +14,5 @@ # limitations under the License. OK_RELEASEAUDIT_WARNINGS=2 -OK_FINDBUGS_WARNINGS=13 +OK_FINDBUGS_WARNINGS=0 OK_JAVADOC_WARNINGS=0 diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java index 422f6ff6b3..c3508f8694 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java @@ -42,7 +42,6 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; import org.apache.hadoop.mapreduce.v2.app.AppContext; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java index c12c60cb5f..016245cbef 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java @@ -34,10 +34,10 @@ import org.apache.hadoop.ipc.RPC.Server; import org.apache.hadoop.ipc.VersionedProtocol; import org.apache.hadoop.mapred.SortedRanges.Range; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener; import org.apache.hadoop.mapreduce.v2.app.TaskHeartbeatHandler; @@ -104,8 +104,8 @@ protected void startRpcServer() { try { server = RPC.getServer(TaskUmbilicalProtocol.class, this, "0.0.0.0", 0, - conf.getInt(AMConstants.AM_TASK_LISTENER_THREADS, - AMConstants.DEFAULT_AM_TASK_LISTENER_THREADS), + conf.getInt(MRJobConfig.MR_AM_TASK_LISTENER_THREAD_COUNT, + MRJobConfig.DEFAULT_MR_AM_TASK_LISTENER_THREAD_COUNT), false, conf, jobTokenSecretManager); server.start(); InetSocketAddress listenerAddress = server.getListenerAddress(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/WrappedProgressSplitsBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/WrappedProgressSplitsBlock.java index 3b452a4ae9..f79580e65b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/WrappedProgressSplitsBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/WrappedProgressSplitsBlock.java @@ -20,9 +20,6 @@ // Workaround for ProgressSplitBlock being package access public class WrappedProgressSplitsBlock extends ProgressSplitsBlock { - - public static final int DEFAULT_NUMBER_PROGRESS_SPLITS = 12; - private WrappedPeriodicStatsAccumulator wrappedProgressWallclockTime; private WrappedPeriodicStatsAccumulator wrappedProgressCPUTime; private WrappedPeriodicStatsAccumulator wrappedProgressVirtualMemoryKbytes; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java index 27fd448074..9650d821c3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java @@ -37,12 +37,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.mapreduce.JobCounter; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.Counter; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.JobState; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo; import org.apache.hadoop.security.UserGroupInformation; @@ -140,7 +140,7 @@ public void init(Configuration conf) { LOG.info("Creating intermediate history logDir: [" + doneDirPath + "] + based on conf. Should ideally be created by the JobHistoryServer: " - + JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY); + + MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR); mkdir( doneDirFS, doneDirPath, @@ -154,7 +154,7 @@ public void init(Configuration conf) { String message = "Not creating intermediate history logDir: [" + doneDirPath + "] based on conf: " - + JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY + + MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR + ". Either set to true or pre-create this directory with appropriate permissions"; LOG.error(message); throw new YarnException(message); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AMConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AMConstants.java deleted file mode 100644 index fbe30370c5..0000000000 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AMConstants.java +++ /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. -*/ - -package org.apache.hadoop.mapreduce.v2.app; - -import org.apache.hadoop.mapreduce.v2.MRConstants; - -public interface AMConstants { - - public static final String CONTAINERLAUNCHER_THREADPOOL_SIZE = - "yarn.mapreduce.containerlauncher.threadpool-size"; - - public static final String AM_RM_SCHEDULE_INTERVAL = - "yarn.appMaster.scheduler.interval"; - - public static final int DEFAULT_AM_RM_SCHEDULE_INTERVAL = 2000; - - public static final String AM_TASK_LISTENER_THREADS = - MRConstants.YARN_MR_PREFIX + "task.listener.threads"; - - public static final int DEFAULT_AM_TASK_LISTENER_THREADS = 10; - - public static final String AM_JOB_CLIENT_THREADS = - MRConstants.YARN_MR_PREFIX + "job.client.threads"; - - public static final int DEFAULT_AM_JOB_CLIENT_THREADS = 1; - - public static final String SPECULATOR_CLASS = - MRConstants.YARN_MR_PREFIX + "speculator.class"; - - public static final String TASK_RUNTIME_ESTIMATOR_CLASS = - MRConstants.YARN_MR_PREFIX + "task.runtime.estimator.class"; - - public static final String TASK_ATTEMPT_PROGRESS_RUNTIME_LINEARIZER_CLASS = - MRConstants.YARN_MR_PREFIX + "task.runtime.linearizer.class"; - - public static final String EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS = - MRConstants.YARN_MR_PREFIX - + "task.runtime.estimator.exponential.smooth.lambda"; - - public static final String EXPONENTIAL_SMOOTHING_SMOOTH_RATE = - MRConstants.YARN_MR_PREFIX - + "task.runtime.estimator.exponential.smooth.smoothsrate"; - - public static final String RECOVERY_ENABLE = MRConstants.YARN_MR_PREFIX - + "recovery.enable"; - - public static final float DEFAULT_REDUCE_RAMP_UP_LIMIT = 0.5f; - public static final String REDUCE_RAMPUP_UP_LIMIT = MRConstants.YARN_MR_PREFIX - + "reduce.rampup.limit"; - - public static final float DEFAULT_REDUCE_PREEMPTION_LIMIT = 0.5f; - public static final String REDUCE_PREEMPTION_LIMIT = MRConstants.YARN_MR_PREFIX - + "reduce.preemption.limit"; - - public static final String NODE_BLACKLISTING_ENABLE = MRConstants.YARN_MR_PREFIX - + "node.blacklisting.enable"; - -} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java index 7e925d58be..4d7a9eafb5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java @@ -154,7 +154,7 @@ public void init(final Configuration conf) { // for an app later appName = conf.get(MRJobConfig.JOB_NAME, ""); - if (conf.getBoolean(AMConstants.RECOVERY_ENABLE, false) + if (conf.getBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, false) && startCount > 1) { LOG.info("Recovery is enabled. Will try to recover from previous life."); Recovery recoveryServ = new RecoveryService(appID, clock, startCount); @@ -349,7 +349,7 @@ protected Speculator createSpeculator(Configuration conf, AppContext context) { try { speculatorClass // "yarn.mapreduce.job.speculator.class" - = conf.getClass(AMConstants.SPECULATOR_CLASS, + = conf.getClass(MRJobConfig.MR_AM_JOB_SPECULATOR, DefaultSpeculator.class, Speculator.class); Constructor speculatorConstructor @@ -360,19 +360,19 @@ protected Speculator createSpeculator(Configuration conf, AppContext context) { return result; } catch (InstantiationException ex) { LOG.error("Can't make a speculator -- check " - + AMConstants.SPECULATOR_CLASS + " " + ex); + + MRJobConfig.MR_AM_JOB_SPECULATOR, ex); throw new YarnException(ex); } catch (IllegalAccessException ex) { LOG.error("Can't make a speculator -- check " - + AMConstants.SPECULATOR_CLASS + " " + ex); + + MRJobConfig.MR_AM_JOB_SPECULATOR, ex); throw new YarnException(ex); } catch (InvocationTargetException ex) { LOG.error("Can't make a speculator -- check " - + AMConstants.SPECULATOR_CLASS + " " + ex); + + MRJobConfig.MR_AM_JOB_SPECULATOR, ex); throw new YarnException(ex); } catch (NoSuchMethodException ex) { LOG.error("Can't make a speculator -- check " - + AMConstants.SPECULATOR_CLASS + " " + ex); + + MRJobConfig.MR_AM_JOB_SPECULATOR, ex); throw new YarnException(ex); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java index 504a941abc..73359bb12a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java @@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.JobACL; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol; import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest; import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptResponse; @@ -59,7 +60,6 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Task; @@ -141,8 +141,8 @@ public void start() { server = rpc.getServer(MRClientProtocol.class, protocolHandler, address, conf, secretManager, - conf.getInt(AMConstants.AM_JOB_CLIENT_THREADS, - AMConstants.DEFAULT_AM_JOB_CLIENT_THREADS)); + conf.getInt(MRJobConfig.MR_AM_JOB_CLIENT_THREAD_COUNT, + MRJobConfig.DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT)); server.start(); this.bindAddress = NetUtils.createSocketAddr(hostNameResolved.getHostAddress() diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java index 98cf9fbd9f..3d4dcb5ed0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java @@ -94,7 +94,6 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent; import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent; import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; import org.apache.hadoop.mapreduce.v2.util.MRApps; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.Credentials; @@ -138,6 +137,7 @@ public abstract class TaskAttemptImpl implements protected final Configuration conf; protected final Path jobFile; protected final int partition; + @SuppressWarnings("rawtypes") protected final EventHandler eventHandler; private final TaskAttemptId attemptId; private final Clock clock; @@ -204,6 +204,11 @@ TaskAttemptEventType.TA_FAILMSG, new DeallocateContainerTransition( .addTransition(TaskAttemptState.ASSIGNED, TaskAttemptState.FAILED, TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED, new DeallocateContainerTransition(TaskAttemptState.FAILED, false)) + .addTransition(TaskAttemptState.ASSIGNED, + TaskAttemptState.FAIL_CONTAINER_CLEANUP, + TaskAttemptEventType.TA_CONTAINER_COMPLETED, + CLEANUP_CONTAINER_TRANSITION) + // ^ If RM kills the container due to expiry, preemption etc. .addTransition(TaskAttemptState.ASSIGNED, TaskAttemptState.KILL_CONTAINER_CLEANUP, TaskAttemptEventType.TA_KILL, CLEANUP_CONTAINER_TRANSITION) @@ -432,7 +437,8 @@ TaskAttemptEventType.TA_CONTAINER_CLEANED, new TaskCleanupTransition()) //this is the last status reported by the REMOTE running attempt private TaskAttemptStatus reportedStatus; - public TaskAttemptImpl(TaskId taskId, int i, EventHandler eventHandler, + public TaskAttemptImpl(TaskId taskId, int i, + @SuppressWarnings("rawtypes") EventHandler eventHandler, TaskAttemptListener taskAttemptListener, Path jobFile, int partition, Configuration conf, String[] dataLocalHosts, OutputCommitter committer, Token jobToken, @@ -528,6 +534,13 @@ private ContainerLaunchContext createContainerLaunchContext() { ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class); + // Application resources + Map localResources = + new HashMap(); + + // Application environment + Map environment = new HashMap(); + try { FileSystem remoteFS = FileSystem.get(conf); @@ -536,7 +549,7 @@ private ContainerLaunchContext createContainerLaunchContext() { Path remoteJobJar = (new Path(remoteTask.getConf().get( MRJobConfig.JAR))).makeQualified(remoteFS.getUri(), remoteFS.getWorkingDirectory()); - container.setLocalResource( + localResources.put( MRConstants.JOB_JAR, createLocalResource(remoteFS, recordFactory, remoteJobJar, LocalResourceType.FILE, LocalResourceVisibility.APPLICATION)); @@ -558,7 +571,7 @@ private ContainerLaunchContext createContainerLaunchContext() { new Path(path, oldJobId.toString()); Path remoteJobConfPath = new Path(remoteJobSubmitDir, MRConstants.JOB_CONF_FILE); - container.setLocalResource( + localResources.put( MRConstants.JOB_CONF_FILE, createLocalResource(remoteFS, recordFactory, remoteJobConfPath, LocalResourceType.FILE, LocalResourceVisibility.APPLICATION)); @@ -566,9 +579,14 @@ private ContainerLaunchContext createContainerLaunchContext() { + remoteJobConfPath.toUri().toASCIIString()); // //////////// End of JobConf setup + // Setup DistributedCache - setupDistributedCache(remoteFS, conf, container); + setupDistributedCache(remoteFS, conf, localResources, environment); + // Set local-resources and environment + container.setLocalResources(localResources); + container.setEnv(environment); + // Setup up tokens Credentials taskCredentials = new Credentials(); @@ -595,12 +613,12 @@ private ContainerLaunchContext createContainerLaunchContext() { // Add shuffle token LOG.info("Putting shuffle token in serviceData"); - container - .setServiceData( - ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID, - ShuffleHandler.serializeServiceData(jobToken)); + Map serviceData = new HashMap(); + serviceData.put(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID, + ShuffleHandler.serializeServiceData(jobToken)); + container.setServiceData(serviceData); - MRApps.addToClassPath(container.getAllEnv(), getInitialClasspath()); + MRApps.addToClassPath(container.getEnv(), getInitialClasspath()); } catch (IOException e) { throw new YarnException(e); } @@ -623,11 +641,11 @@ private ContainerLaunchContext createContainerLaunchContext() { classPaths.add(workDir.toString()); // TODO // Construct the actual Container - container.addAllCommands(MapReduceChildJVM.getVMCommand( + container.setCommands(MapReduceChildJVM.getVMCommand( taskAttemptListener.getAddress(), remoteTask, javaHome, workDir.toString(), containerLogDir, childTmpDir, jvmID)); - MapReduceChildJVM.setVMEnv(container.getAllEnv(), classPaths, + MapReduceChildJVM.setVMEnv(container.getEnv(), classPaths, workDir.toString(), containerLogDir, nmLdLibraryPath, remoteTask, localizedApplicationTokensFile); @@ -649,11 +667,15 @@ private static long[] parseTimeStamps(String[] strs) { return result; } - private void setupDistributedCache(FileSystem remoteFS, Configuration conf, - ContainerLaunchContext container) throws IOException { + private void setupDistributedCache(FileSystem remoteFS, + Configuration conf, + Map localResources, + Map env) + throws IOException { // Cache archives - parseDistributedCacheArtifacts(remoteFS, container, LocalResourceType.ARCHIVE, + parseDistributedCacheArtifacts(remoteFS, localResources, env, + LocalResourceType.ARCHIVE, DistributedCache.getCacheArchives(conf), parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)), getFileSizes(conf, MRJobConfig.CACHE_ARCHIVES_SIZES), @@ -661,7 +683,9 @@ private void setupDistributedCache(FileSystem remoteFS, Configuration conf, DistributedCache.getArchiveClassPaths(conf)); // Cache files - parseDistributedCacheArtifacts(remoteFS, container, LocalResourceType.FILE, + parseDistributedCacheArtifacts(remoteFS, + localResources, env, + LocalResourceType.FILE, DistributedCache.getCacheFiles(conf), parseTimeStamps(DistributedCache.getFileTimestamps(conf)), getFileSizes(conf, MRJobConfig.CACHE_FILES_SIZES), @@ -673,7 +697,10 @@ private void setupDistributedCache(FileSystem remoteFS, Configuration conf, // Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[], // long[], boolean[], Path[], FileType) private void parseDistributedCacheArtifacts( - FileSystem remoteFS, ContainerLaunchContext container, LocalResourceType type, + FileSystem remoteFS, + Map localResources, + Map env, + LocalResourceType type, URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[], Path[] pathsToPutOnClasspath) throws IOException { @@ -710,9 +737,9 @@ private void parseDistributedCacheArtifacts( throw new IllegalArgumentException("Resource name must be relative"); } String linkName = name.toUri().getPath(); - container.setLocalResource( + localResources.put( linkName, - BuilderUtils.newLocalResource(recordFactory, + BuilderUtils.newLocalResource( p.toUri(), type, visibilities[i] ? LocalResourceVisibility.PUBLIC @@ -720,8 +747,7 @@ private void parseDistributedCacheArtifacts( sizes[i], timestamps[i]) ); if (classPaths.containsKey(u.getPath())) { - Map environment = container.getAllEnv(); - MRApps.addToClassPath(environment, linkName); + MRApps.addToClassPath(env, linkName); } } } @@ -893,6 +919,7 @@ public TaskAttemptState getState() { } } + @SuppressWarnings("unchecked") @Override public void handle(TaskAttemptEvent event) { LOG.info("Processing " + event.getTaskAttemptID() + @@ -903,7 +930,8 @@ public void handle(TaskAttemptEvent event) { try { stateMachine.doTransition(event.getType(), event); } catch (InvalidStateTransitonException e) { - LOG.error("Can't handle this event at current state", e); + LOG.error("Can't handle this event at current state for " + + this.attemptId, e); eventHandler.handle(new JobDiagnosticsUpdateEvent( this.attemptId.getTaskId().getJobId(), "Invalid event " + event.getType() + " on TaskAttempt " + this.attemptId)); @@ -981,8 +1009,8 @@ private WrappedProgressSplitsBlock getProgressSplitBlock() { try { if (progressSplitBlock == null) { progressSplitBlock = new WrappedProgressSplitsBlock(conf.getInt( - JHConfig.JOBHISTORY_TASKPROGRESS_NUMBER_SPLITS_KEY, - WrappedProgressSplitsBlock.DEFAULT_NUMBER_PROGRESS_SPLITS)); + MRJobConfig.MR_AM_NUM_PROGRESS_SPLITS, + MRJobConfig.DEFAULT_MR_AM_NUM_PROGRESS_SPLITS)); } return progressSplitBlock; } finally { @@ -1035,6 +1063,7 @@ private static class RequestContainerTransition implements public RequestContainerTransition(boolean rescheduled) { this.rescheduled = rescheduled; } + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { @@ -1063,6 +1092,7 @@ public void transition(TaskAttemptImpl taskAttempt, private static class ContainerAssignedTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(final TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { @@ -1112,6 +1142,7 @@ private static class DeallocateContainerTransition implements this.finalState = finalState; this.withdrawsContainerRequest = withdrawsContainerRequest; } + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { @@ -1158,6 +1189,7 @@ public void transition(TaskAttemptImpl taskAttempt, private static class LaunchedContainerTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent evnt) { @@ -1208,6 +1240,7 @@ public void transition(TaskAttemptImpl taskAttempt, private static class CommitPendingTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { @@ -1219,6 +1252,7 @@ public void transition(TaskAttemptImpl taskAttempt, private static class TaskCleanupTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { @@ -1234,6 +1268,7 @@ public void transition(TaskAttemptImpl taskAttempt, private static class SucceededTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { @@ -1263,6 +1298,7 @@ public void transition(TaskAttemptImpl taskAttempt, private static class FailedTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { // set the finish time @@ -1287,6 +1323,7 @@ public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { } } + @SuppressWarnings({ "unchecked" }) private void logAttemptFinishedEvent(TaskAttemptState state) { //Log finished events only if an attempt started. if (getLaunchTime() == 0) return; @@ -1320,6 +1357,7 @@ private void logAttemptFinishedEvent(TaskAttemptState state) { private static class TooManyFetchFailureTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { //add to diagnostic @@ -1347,6 +1385,7 @@ public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { private static class KilledTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { @@ -1373,6 +1412,7 @@ public void transition(TaskAttemptImpl taskAttempt, private static class CleanupContainerTransition implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { @@ -1399,6 +1439,7 @@ private void addDiagnosticInfo(String diag) { private static class StatusUpdater implements SingleArcTransition { + @SuppressWarnings("unchecked") @Override public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java index cd2a540b97..71f8823e68 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java @@ -528,7 +528,8 @@ public void handle(TaskEvent event) { try { stateMachine.doTransition(event.getType(), event); } catch (InvalidStateTransitonException e) { - LOG.error("Can't handle this event at current state", e); + LOG.error("Can't handle this event at current state for " + + this.taskId, e); internalError(event.getType()); } if (oldState != getState()) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java index bc6322cb0e..982f7d334a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java @@ -21,9 +21,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.security.PrivilegedAction; -import java.util.HashMap; -import java.util.Map; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -33,8 +33,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.ShuffleHandler; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent; @@ -79,8 +79,8 @@ public class ContainerLauncherImpl extends AbstractService implements private RecordFactory recordFactory; //have a cache/map of UGIs so as to avoid creating too many RPC //client connection objects to the same NodeManager - private Map ugiMap = - new HashMap(); + private ConcurrentMap ugiMap = + new ConcurrentHashMap(); public ContainerLauncherImpl(AppContext context) { super(ContainerLauncherImpl.class.getName()); @@ -102,7 +102,7 @@ public synchronized void init(Configuration conf) { public void start() { launcherPool = new ThreadPoolExecutor(getConfig().getInt( - AMConstants.CONTAINERLAUNCHER_THREADPOOL_SIZE, 10), + MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT, 10), Integer.MAX_VALUE, 1, TimeUnit.HOURS, new LinkedBlockingQueue()); launcherPool.prestartAllCoreThreads(); // Wait for work. @@ -142,22 +142,19 @@ protected ContainerManager getCMProxy(ContainerId containerID, UserGroupInformation user = UserGroupInformation.getCurrentUser(); - // TODO: Synchronization problems!! if (UserGroupInformation.isSecurityEnabled()) { - if(!ugiMap.containsKey(containerManagerBindAddr)) { - Token token = - new Token( - containerToken.getIdentifier().array(), - containerToken.getPassword().array(), new Text( - containerToken.getKind()), new Text( - containerToken.getService())); - //the user in createRemoteUser in this context is not important - user = UserGroupInformation.createRemoteUser(containerManagerBindAddr); - user.addToken(token); - ugiMap.put(containerManagerBindAddr, user); - } else { - user = ugiMap.get(containerManagerBindAddr); - } + + Token token = new Token( + containerToken.getIdentifier().array(), containerToken + .getPassword().array(), new Text(containerToken.getKind()), + new Text(containerToken.getService())); + // the user in createRemoteUser in this context is not important + UserGroupInformation ugi = UserGroupInformation + .createRemoteUser(containerManagerBindAddr); + ugi.addToken(token); + ugiMap.putIfAbsent(containerManagerBindAddr, ugi); + + user = ugiMap.get(containerManagerBindAddr); } ContainerManager proxy = user.doAs(new PrivilegedAction() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java index dfe9b8e02c..18a0f2d5a6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java @@ -70,7 +70,7 @@ public void handle(ContainerAllocatorEvent event) { if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) { LOG.info("Processing the event " + event.toString()); ContainerId cID = recordFactory.newRecordInstance(ContainerId.class); - cID.setAppId(appID); + cID.setApplicationAttemptId(applicationAttemptId); // use negative ids to denote that these are local. Need a better way ?? cID.setId((-1) * containerCount.getAndIncrement()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java index dfebd27119..db4a60b1dc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java @@ -25,12 +25,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.JobState; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.client.ClientService; import org.apache.hadoop.mapreduce.v2.app.job.Job; @@ -98,8 +97,8 @@ public RMCommunicator(ClientService clientService, AppContext context) { public void init(Configuration conf) { super.init(conf); rmPollInterval = - conf.getInt(AMConstants.AM_RM_SCHEDULE_INTERVAL, - AMConstants.DEFAULT_AM_RM_SCHEDULE_INTERVAL); + conf.getInt(MRJobConfig.MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS, + MRJobConfig.DEFAULT_MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS); } @Override @@ -226,8 +225,8 @@ protected AMRMProtocol createSchedulerProxy() { final YarnRPC rpc = YarnRPC.create(getConfig()); final Configuration conf = new Configuration(getConfig()); final String serviceAddr = conf.get( - YarnConfiguration.SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_SCHEDULER_BIND_ADDRESS); + YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS); UserGroupInformation currentUser; try { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java index fad43bd6a7..ff232104bd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java @@ -40,7 +40,6 @@ import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.client.ClientService; import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent; @@ -54,6 +53,7 @@ import org.apache.hadoop.yarn.api.records.AMResponse; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.util.RackResolver; @@ -137,11 +137,11 @@ public void init(Configuration conf) { MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART, DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART); maxReduceRampupLimit = conf.getFloat( - AMConstants.REDUCE_RAMPUP_UP_LIMIT, - AMConstants.DEFAULT_REDUCE_RAMP_UP_LIMIT); + MRJobConfig.MR_AM_JOB_REDUCE_RAMPUP_UP_LIMIT, + MRJobConfig.DEFAULT_MR_AM_JOB_REDUCE_RAMP_UP_LIMIT); maxReducePreemptionLimit = conf.getFloat( - AMConstants.REDUCE_PREEMPTION_LIMIT, - AMConstants.DEFAULT_REDUCE_PREEMPTION_LIMIT); + MRJobConfig.MR_AM_JOB_REDUCE_PREEMPTION_LIMIT, + MRJobConfig.DEFAULT_MR_AM_JOB_REDUCE_PREEMPTION_LIMIT); RackResolver.init(conf); } @@ -415,8 +415,8 @@ private List getResources() throws Exception { int headRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;//first time it would be null AMResponse response = makeRemoteRequest(); int newHeadRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0; - List newContainers = response.getNewContainerList(); - List finishedContainers = response.getFinishedContainerList(); + List newContainers = response.getAllocatedContainers(); + List finishedContainers = response.getCompletedContainersStatuses(); if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) { //something changed recalculateReduceSchedule = true; @@ -427,12 +427,12 @@ private List getResources() throws Exception { allocatedContainers.add(cont); LOG.debug("Received new Container :" + cont); } - for (Container cont : finishedContainers) { + for (ContainerStatus cont : finishedContainers) { LOG.info("Received completed container " + cont); - TaskAttemptId attemptID = assignedRequests.get(cont.getId()); + TaskAttemptId attemptID = assignedRequests.get(cont.getContainerId()); if (attemptID == null) { LOG.error("Container complete event for unknown container id " - + cont.getId()); + + cont.getContainerId()); } else { assignedRequests.remove(attemptID); if (attemptID.getTaskId().getTaskType().equals(TaskType.MAP)) { @@ -444,7 +444,7 @@ private List getResources() throws Exception { eventHandler.handle(new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_CONTAINER_COMPLETED)); // Send the diagnostics - String diagnostics = cont.getContainerStatus().getDiagnostics(); + String diagnostics = cont.getDiagnostics(); eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptID, diagnostics)); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java index 024bac1bb7..b9f0c6ee45 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java @@ -31,13 +31,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.client.ClientService; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.records.AMResponse; -import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; @@ -102,7 +100,7 @@ public ContainerRequest(ContainerRequestEvent event, Priority priority) { public void init(Configuration conf) { super.init(conf); nodeBlacklistingEnabled = - conf.getBoolean(AMConstants.NODE_BLACKLISTING_ENABLE, true); + conf.getBoolean(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, true); LOG.info("nodeBlacklistingEnabled:" + nodeBlacklistingEnabled); maxTaskFailuresPerNode = conf.getInt(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, 3); @@ -124,10 +122,11 @@ protected AMResponse makeRemoteRequest() throws YarnRemoteException { availableResources = response.getAvailableResources(); LOG.info("getResources() for " + applicationId + ":" + " ask=" - + ask.size() + " release= " + release.size() + " newContainers=" - + response.getNewContainerCount() + " finishedContainers=" - + response.getFinishedContainerCount() - + " resourcelimit=" + availableResources); + + ask.size() + " release= " + release.size() + + " newContainers=" + response.getAllocatedContainers().size() + + " finishedContainers=" + + response.getCompletedContainersStatuses().size() + + " resourcelimit=" + availableResources); ask.clear(); release.clear(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java index a51a3e7b43..feb019fe16 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java @@ -34,13 +34,13 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Task; @@ -117,7 +117,7 @@ public DefaultSpeculator(Configuration conf, AppContext context, Clock clock) { try { // "yarn.mapreduce.job.task.runtime.estimator.class" Class estimatorClass - = conf.getClass(AMConstants.TASK_RUNTIME_ESTIMATOR_CLASS, + = conf.getClass(MRJobConfig.MR_AM_TASK_ESTIMATOR, LegacyTaskRuntimeEstimator.class, TaskRuntimeEstimator.class); @@ -128,16 +128,16 @@ public DefaultSpeculator(Configuration conf, AppContext context, Clock clock) { estimator.contextualize(conf, context); } catch (InstantiationException ex) { - LOG.error("Can't make a speculation runtime extimator" + ex); + LOG.error("Can't make a speculation runtime extimator", ex); throw new YarnException(ex); } catch (IllegalAccessException ex) { - LOG.error("Can't make a speculation runtime extimator" + ex); + LOG.error("Can't make a speculation runtime extimator", ex); throw new YarnException(ex); } catch (InvocationTargetException ex) { - LOG.error("Can't make a speculation runtime extimator" + ex); + LOG.error("Can't make a speculation runtime extimator", ex); throw new YarnException(ex); } catch (NoSuchMethodException ex) { - LOG.error("Can't make a speculation runtime extimator" + ex); + LOG.error("Can't make a speculation runtime extimator", ex); throw new YarnException(ex); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/ExponentiallySmoothedTaskRuntimeEstimator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/ExponentiallySmoothedTaskRuntimeEstimator.java index ff50bc2f1d..cb6b441743 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/ExponentiallySmoothedTaskRuntimeEstimator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/ExponentiallySmoothedTaskRuntimeEstimator.java @@ -23,8 +23,8 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; -import org.apache.hadoop.mapreduce.v2.app.AMConstants; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus; @@ -129,18 +129,15 @@ private EstimateVector getEstimateVector(TaskAttemptId attemptID) { return vectorRef.get(); } - private static final long DEFAULT_EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS - = 1000L * 60; - @Override public void contextualize(Configuration conf, AppContext context) { super.contextualize(conf, context); lambda - = conf.getLong(AMConstants.EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS, - DEFAULT_EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS); + = conf.getLong(MRJobConfig.MR_AM_TASK_ESTIMATOR_SMOOTH_LAMBDA_MS, + MRJobConfig.DEFAULT_MR_AM_TASK_ESTIMATOR_SMNOOTH_LAMBDA_MS); smoothedValue - = conf.getBoolean(AMConstants.EXPONENTIAL_SMOOTHING_SMOOTH_RATE, true) + = conf.getBoolean(MRJobConfig.MR_AM_TASK_EXTIMATOR_EXPONENTIAL_RATE_ENABLE, true) ? SmoothedValue.RATE : SmoothedValue.TIME_PER_UNIT_PROGRESS; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/App.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/App.java index bb28ce68cc..c22ff0538b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/App.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/App.java @@ -28,11 +28,27 @@ @RequestScoped public class App { final AppContext context; - Job job; - Task task; + private Job job; + private Task task; @Inject App(AppContext ctx) { context = ctx; } + + void setJob(Job job) { + this.job = job; + } + + public Job getJob() { + return job; + } + + void setTask(Task task) { + this.task = task; + } + + public Task getTask() { + return task; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java index adc0e14584..3c3ba74651 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java @@ -22,6 +22,8 @@ import java.util.Locale; +import javax.servlet.http.HttpServletResponse; + import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.v2.api.records.JobId; @@ -31,9 +33,13 @@ import org.apache.hadoop.yarn.util.Apps; import org.apache.hadoop.yarn.util.Times; import org.apache.hadoop.yarn.webapp.Controller; +import org.apache.hadoop.yarn.webapp.View; import com.google.inject.Inject; +/** + * This class renders the various pages that the web app supports. + */ public class AppController extends Controller implements AMParams { final App app; @@ -41,7 +47,7 @@ protected AppController(App app, Configuration conf, RequestContext ctx, String title) { super(ctx); this.app = app; - set(APP_ID, Apps.toString(app.context.getApplicationID())); + set(APP_ID, app.context.getApplicationID().toString()); set(RM_WEB, YarnConfiguration.getRMWebAppURL(conf)); } @@ -50,10 +56,16 @@ protected AppController(App app, Configuration conf, RequestContext ctx) { this(app, conf, ctx, "am"); } + /** + * Render the default(index.html) page for the Application Controller + */ @Override public void index() { setTitle(join("MapReduce Application ", $(APP_ID))); } + /** + * Render the /info page with an overview of current application. + */ public void info() { info("Application Master Overview"). _("Application ID:", $(APP_ID)). @@ -65,22 +77,52 @@ public void info() { render(InfoPage.class); } + /** + * @return The class that will render the /job page + */ + protected Class jobPage() { + return JobPage.class; + } + + /** + * Render the /job page + */ public void job() { requireJob(); - render(JobPage.class); + render(jobPage()); } + /** + * @return the class that will render the /jobcounters page + */ + protected Class countersPage() { + return CountersPage.class; + } + + /** + * Render the /jobcounters page + */ public void jobCounters() { requireJob(); - if (app.job != null) { + if (app.getJob() != null) { setTitle(join("Counters for ", $(JOB_ID))); } - render(CountersPage.class); + render(countersPage()); } + /** + * @return the class that will render the /tasks page + */ + protected Class tasksPage() { + return TasksPage.class; + } + + /** + * Render the /tasks page + */ public void tasks() { requireJob(); - if (app.job != null) { + if (app.getJob() != null) { try { String tt = $(TASK_TYPE); tt = tt.isEmpty() ? "All" : StringUtils.capitalize(MRApps.taskType(tt). @@ -90,20 +132,40 @@ public void tasks() { badRequest(e.getMessage()); } } - render(TasksPage.class); + render(tasksPage()); } + /** + * @return the class that will render the /task page + */ + protected Class taskPage() { + return TaskPage.class; + } + + /** + * Render the /task page + */ public void task() { requireTask(); - if (app.task != null) { + if (app.getTask() != null) { setTitle(join("Attempts for ", $(TASK_ID))); } - render(TaskPage.class); + render(taskPage()); } + /** + * @return the class that will render the /attempts page + */ + protected Class attemptsPage() { + return AttemptsPage.class; + } + + /** + * Render the attempts page + */ public void attempts() { requireJob(); - if (app.job != null) { + if (app.getJob() != null) { try { String taskType = $(TASK_TYPE); if (taskType.isEmpty()) { @@ -119,27 +181,38 @@ public void attempts() { badRequest(e.getMessage()); } } - render(AttemptsPage.class); + render(attemptsPage()); } + /** + * Render a BAD_REQUEST error. + * @param s the error message to include. + */ void badRequest(String s) { - setStatus(response().SC_BAD_REQUEST); + setStatus(HttpServletResponse.SC_BAD_REQUEST); setTitle(join("Bad request: ", s)); } + /** + * Render a NOT_FOUND error. + * @param s the error message to include. + */ void notFound(String s) { - setStatus(response().SC_NOT_FOUND); + setStatus(HttpServletResponse.SC_NOT_FOUND); setTitle(join("Not found: ", s)); } + /** + * Ensure that a JOB_ID was passed into the page. + */ void requireJob() { try { if ($(JOB_ID).isEmpty()) { throw new RuntimeException("missing job ID"); } JobId jobID = MRApps.toJobID($(JOB_ID)); - app.job = app.context.getJob(jobID); - if (app.job == null) { + app.setJob(app.context.getJob(jobID)); + if (app.getJob() == null) { notFound($(JOB_ID)); } } catch (Exception e) { @@ -147,18 +220,21 @@ void requireJob() { } } + /** + * Ensure that a TASK_ID was passed into the page. + */ void requireTask() { try { if ($(TASK_ID).isEmpty()) { throw new RuntimeException("missing task ID"); } TaskId taskID = MRApps.toTaskID($(TASK_ID)); - app.job = app.context.getJob(taskID.getJobId()); - if (app.job == null) { + app.setJob(app.context.getJob(taskID.getJobId())); + if (app.getJob() == null) { notFound(MRApps.toString(taskID.getJobId())); } else { - app.task = app.job.getTask(taskID); - if (app.task == null) { + app.setTask(app.getJob().getTask(taskID)); + if (app.getTask() == null) { notFound($(TASK_ID)); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java index d0cd23e686..92ae7b6f84 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java @@ -57,7 +57,7 @@ protected Collection getTaskAttempts() { String attemptStateStr = $(ATTEMPT_STATE); TaskAttemptStateUI neededState = MRApps .taskAttemptState(attemptStateStr); - for (Task task : super.app.job.getTasks(taskType).values()) { + for (Task task : super.app.getJob().getTasks(taskType).values()) { Map attempts = task.getAttempts(); for (TaskAttempt attempt : attempts.values()) { if (neededState.correspondsTo(attempt.getState())) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java index c7d6751300..5bbfc24f2b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java @@ -45,8 +45,8 @@ public class NavBlock extends HtmlBlock { ul(). li().a(url("app/info"), "About")._(). li().a(url("app"), "Jobs")._()._(); - if (app.job != null) { - String jobid = MRApps.toString(app.job.getID()); + if (app.getJob() != null) { + String jobid = MRApps.toString(app.getJob().getID()); nav. h3("Job"). ul(). diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java index 3671df01e4..d9884d146a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java @@ -98,11 +98,11 @@ protected void render(Block html) { } protected boolean isValidRequest() { - return app.task != null; + return app.getTask() != null; } protected Collection getTaskAttempts() { - return app.task.getAttempts().values(); + return app.getTask().getAttempts().values(); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java index c44453bfe9..7247761d65 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java @@ -42,7 +42,7 @@ public class TasksBlock extends HtmlBlock { } @Override protected void render(Block html) { - if (app.job == null) { + if (app.getJob() == null) { html. h2($(TITLE)); return; @@ -63,7 +63,7 @@ public class TasksBlock extends HtmlBlock { th("Finish Time"). th("Elapsed Time")._()._(). tbody(); - for (Task task : app.job.getTasks().values()) { + for (Task task : app.getJob().getTasks().values()) { if (type != null && task.getType() != type) { continue; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java index 1151b76610..499831c5f8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java @@ -35,7 +35,6 @@ import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent; import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo; -import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.JobReport; import org.apache.hadoop.mapreduce.v2.api.records.JobState; @@ -62,10 +61,8 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent; import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleaner; import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.Clock; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -134,8 +131,8 @@ public MRApp(int maps, int reduces, boolean autoComplete, String testName, boole public Job submit(Configuration conf) throws Exception { String user = conf.get(MRJobConfig.USER_NAME, "mapred"); conf.set(MRJobConfig.USER_NAME, user); - conf.set(MRConstants.APPS_STAGING_DIR_KEY, testAbsPath.toString()); - conf.setBoolean(JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY, true); + conf.set(MRJobConfig.MR_AM_STAGING_DIR, testAbsPath.toString()); + conf.setBoolean(MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR, true); //TODO: fix the bug where the speculator gets events with //not-fully-constructed objects. For now, disable speculative exec LOG.info("****DISABLING SPECULATIVE EXECUTION*****"); @@ -326,7 +323,7 @@ protected ContainerAllocator createContainerAllocator( @Override public void handle(ContainerAllocatorEvent event) { ContainerId cId = recordFactory.newRecordInstance(ContainerId.class); - cId.setAppId(getContext().getApplicationID()); + cId.setApplicationAttemptId(getContext().getApplicationAttemptId()); cId.setId(containerCount++); Container container = recordFactory.newRecordInstance(Container.class); container.setId(cId); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java index 74bb1a89bf..3615c27b15 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java @@ -30,6 +30,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator; import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; @@ -124,12 +125,15 @@ public void run() { try { if (concurrentRunningTasks < maxConcurrentRunningTasks) { event = eventQueue.take(); - ContainerId cId = recordFactory.newRecordInstance(ContainerId.class); - cId.setAppId(getContext().getApplicationID()); + ContainerId cId = + recordFactory.newRecordInstance(ContainerId.class); + cId.setApplicationAttemptId( + getContext().getApplicationAttemptId()); cId.setId(containerCount++); //System.out.println("Allocating " + containerCount); - Container container = recordFactory.newRecordInstance(Container.class); + Container container = + recordFactory.newRecordInstance(Container.class); container.setId(cId); NodeId nodeId = recordFactory.newRecordInstance(NodeId.class); nodeId.setHost("dummy"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java index ce160b8f13..9f69386034 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java @@ -51,6 +51,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.MockApps; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.Records; @@ -235,7 +236,11 @@ public boolean isFinished() { @Override public ContainerId getAssignedContainerID() { ContainerId id = Records.newRecord(ContainerId.class); - id.setAppId(taid.getTaskId().getJobId().getAppId()); + ApplicationAttemptId appAttemptId = + Records.newRecord(ApplicationAttemptId.class); + appAttemptId.setApplicationId(taid.getTaskId().getJobId().getAppId()); + appAttemptId.setAttemptId(0); + id.setApplicationAttemptId(appAttemptId); return id; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java index 4744035f5e..e08d4f2a31 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.v2.app; +import java.io.IOException; import java.util.Iterator; import java.util.Map; @@ -36,6 +37,12 @@ import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType; +import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher; +import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent; +import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl; +import org.apache.hadoop.yarn.api.ContainerManager; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerToken; import org.junit.Test; /** @@ -160,6 +167,74 @@ public void testTimedOutTask() throws Exception { } } + @Test + public void testTaskFailWithUnusedContainer() throws Exception { + MRApp app = new FailingTaskWithUnusedContainer(); + Configuration conf = new Configuration(); + int maxAttempts = 1; + conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, maxAttempts); + // disable uberization (requires entire job to be reattempted, so max for + // subtask attempts is overridden to 1) + conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); + Job job = app.submit(conf); + app.waitForState(job, JobState.RUNNING); + Map tasks = job.getTasks(); + Assert.assertEquals("Num tasks is not correct", 1, tasks.size()); + Task task = tasks.values().iterator().next(); + app.waitForState(task, TaskState.SCHEDULED); + Map attempts = tasks.values().iterator() + .next().getAttempts(); + Assert.assertEquals("Num attempts is not correct", maxAttempts, attempts + .size()); + TaskAttempt attempt = attempts.values().iterator().next(); + app.waitForState(attempt, TaskAttemptState.ASSIGNED); + app.getDispatcher().getEventHandler().handle( + new TaskAttemptEvent(attempt.getID(), + TaskAttemptEventType.TA_CONTAINER_COMPLETED)); + app.waitForState(job, JobState.FAILED); + } + + static class FailingTaskWithUnusedContainer extends MRApp { + + public FailingTaskWithUnusedContainer() { + super(1, 0, false, "TaskFailWithUnsedContainer", true); + } + + protected ContainerLauncher createContainerLauncher(AppContext context, + boolean isLocal) { + return new ContainerLauncherImpl(context) { + @Override + public void handle(ContainerLauncherEvent event) { + + switch (event.getType()) { + case CONTAINER_REMOTE_LAUNCH: + super.handle(event); + break; + case CONTAINER_REMOTE_CLEANUP: + getContext().getEventHandler().handle( + new TaskAttemptEvent(event.getTaskAttemptID(), + TaskAttemptEventType.TA_CONTAINER_CLEANED)); + break; + } + } + + @Override + protected ContainerManager getCMProxy(ContainerId containerID, + String containerManagerBindAddr, ContainerToken containerToken) + throws IOException { + try { + synchronized (this) { + wait(); // Just hang the thread simulating a very slow NM. + } + } catch (InterruptedException e) { + e.printStackTrace(); + } + return null; + } + }; + }; + } + static class TimeOutTaskMRApp extends MRApp { TimeOutTaskMRApp(int maps, int reduces) { super(maps, reduces, false, "TimeOutTaskMRApp", true); @@ -232,5 +307,6 @@ public static void main(String[] args) throws Exception { t.testTimedOutTask(); t.testMapFailureMaxPercent(); t.testReduceFailureMaxPercent(); + t.testTaskFailWithUnusedContainer(); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java index 448af9b473..75db751480 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java @@ -131,7 +131,7 @@ public void testCrashed() throws Exception { //in rerun the 1st map will be recovered from previous run app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), false, ++runCount); conf = new Configuration(); - conf.setBoolean(AMConstants.RECOVERY_ENABLE, true); + conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true); conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); job = app.submit(conf); app.waitForState(job, JobState.RUNNING); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java index 474e39d40d..c5938e8ca1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java @@ -108,7 +108,7 @@ public long getStartTime() { Injector injector = WebAppTests.createMockInjector(AppContext.class, ctx); AppController controller = injector.getInstance(AppController.class); controller.index(); - assertEquals(Apps.toString(ctx.appID), controller.get(APP_ID,"")); + assertEquals(ctx.appID.toString(), controller.get(APP_ID,"")); } @Test public void testAppView() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/MRConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/MRConstants.java index 326261c42e..6ac05361dc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/MRConstants.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/MRConstants.java @@ -25,9 +25,6 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public interface MRConstants { - - public static final String YARN_MR_PREFIX = "yarn.mapreduce.job."; - // This should be the directory where splits file gets localized on the node // running ApplicationMaster. public static final String JOB_SUBMIT_DIR = "jobSubmitDir"; @@ -45,8 +42,6 @@ public interface MRConstants { public static final String YARN_MAPREDUCE_APP_JAR_PATH = "$YARN_HOME/modules/" + HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME; - public static final String APPS_STAGING_DIR_KEY = "yarn.apps.stagingDir"; - // The token file for the application. Should contain tokens for access to // remote file system and may optionally contain application specific tokens. // For now, generated by the AppManagers and used by NodeManagers and the diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java new file mode 100644 index 0000000000..a726a005b5 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java @@ -0,0 +1,111 @@ +/** +* 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.v2.jobhistory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Stores Job History configuration keys that can be set by administrators of + * the Job History server. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class JHAdminConfig { + /** The prefix to all Job History configuration properties.*/ + public static final String MR_HISTORY_PREFIX = "mapreduce.jobhistory."; + + /** host:port address for History Server API.*/ + public static final String MR_HISTORY_ADDRESS = MR_HISTORY_PREFIX + "address"; + public static final String DEFAULT_MR_HISTORY_ADDRESS = "0.0.0.0:10020"; + + /** If history cleaning should be enabled or not.*/ + public static final String MR_HISTORY_CLEANER_ENABLE = + MR_HISTORY_PREFIX + "cleaner.enable"; + + /** Run the History Cleaner every X ms.*/ + public static final String MR_HISTORY_CLEANER_INTERVAL_MS = + MR_HISTORY_PREFIX + "cleaner.interval-ms"; + + /** The number of threads to handle client API requests.*/ + public static final String MR_HISTORY_CLIENT_THREAD_COUNT = + MR_HISTORY_PREFIX + "client.thread-count"; + public static final int DEFAULT_MR_HISTORY_CLIENT_THREAD_COUNT = 10; + + /** + * Size of the date string cache. Effects the number of directories + * which will be scanned to find a job. + */ + public static final String MR_HISTORY_DATESTRING_CACHE_SIZE = + MR_HISTORY_PREFIX + "datestring.cache.size"; + + /** Equivalent to 0.20 mapreduce.jobhistory.debug.mode */ + public static final String MR_HISTORY_DEBUG_MODE = + MR_HISTORY_PREFIX + "debug-mode"; + + /** Path where history files should be stored for DONE jobs. **/ + public static final String MR_HISTORY_DONE_DIR = + MR_HISTORY_PREFIX + "done-dir"; + + /** + * Path where history files should be stored after a job finished and before + * they are pulled into the job history server. + **/ + public static final String MR_HISTORY_INTERMEDIATE_DONE_DIR = + MR_HISTORY_PREFIX + "intermediate-done-dir"; + + /** Size of the job list cache.*/ + public static final String MR_HISTORY_JOBLIST_CACHE_SIZE = + MR_HISTORY_PREFIX + "joblist.cache.size"; + + /** The location of the Kerberos keytab file.*/ + public static final String MR_HISTORY_KEYTAB = MR_HISTORY_PREFIX + "keytab"; + + /** Size of the loaded job cache.*/ + public static final String MR_HISTORY_LOADED_JOB_CACHE_SIZE = + MR_HISTORY_PREFIX + "loadedjobs.cache.size"; + + /** + * The maximum age of a job history file before it is deleted from the history + * server. + */ + public static final String MR_HISTORY_MAX_AGE_MS = + MR_HISTORY_PREFIX + "max-age-ms"; + + /** + * Scan for history files to more from intermediate done dir to done dir + * every X ms. + */ + public static final String MR_HISTORY_MOVE_INTERVAL_MS = + MR_HISTORY_PREFIX + "move.interval-ms"; + + /** The number of threads used to move files.*/ + public static final String MR_HISTORY_MOVE_THREAD_COUNT = + MR_HISTORY_PREFIX + "move.thread-count"; + + /** The Kerberos principal for the history server.*/ + public static final String MR_HISTORY_PRINCIPAL = + MR_HISTORY_PREFIX + "principal"; + + /**The address the history server webapp is on.*/ + public static final String MR_HISTORY_WEBAPP_ADDRESS = + MR_HISTORY_PREFIX + "webapp.address"; + public static final String DEFAULT_MR_HISTORY_WEBAPP_ADDRESS = + "0.0.0.0:19888"; +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHConfig.java deleted file mode 100644 index 681961aa50..0000000000 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHConfig.java +++ /dev/null @@ -1,117 +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.mapreduce.v2.jobhistory; - -public class JHConfig { - public static final String HS_PREFIX = "yarn.server.historyserver."; - /** host:port address to which to bind to **/ - public static final String HS_BIND_ADDRESS = HS_PREFIX + "address"; - - public static final String HS_USER_NAME = HS_PREFIX + "kerberos.principal"; - - public static final String HS_KEYTAB_FILE = HS_PREFIX + "jeytab.file"; - - public static final String DEFAULT_HS_BIND_ADDRESS = "0.0.0.0:10020"; - - /** Done Dir for for AppMaster **/ - public static final String HISTORY_INTERMEDIATE_DONE_DIR_KEY = - "yarn.historyfile.intermediateDoneDir"; - - /** Done Dir for for AppMaster **/ - public static final String HISTORY_DONE_DIR_KEY = - "yarn.historyfile.doneDir"; - - /** - * Boolean. Create the base dirs in the JobHistoryEventHandler - * Set to false for multi-user clusters. - */ - public static final String CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY = - "yarn.history.create.intermediate.base.dir"; - - /** Done Dir for history server. **/ - public static final String HISTORY_SERVER_DONE_DIR_KEY = - HS_PREFIX + "historyfile.doneDir"; - - /** - * Size of the job list cache. - */ - public static final String HISTORY_SERVER_JOBLIST_CACHE_SIZE_KEY = - HS_PREFIX + "joblist.cache.size"; - - /** - * Size of the loaded job cache. - */ - public static final String HISTORY_SERVER_LOADED_JOB_CACHE_SIZE_KEY = - HS_PREFIX + "loadedjobs.cache.size"; - - /** - * Size of the date string cache. Effects the number of directories - * which will be scanned to find a job. - */ - public static final String HISTORY_SERVER_DATESTRING_CACHE_SIZE_KEY = - HS_PREFIX + "datestring.cache.size"; - - /** - * The time interval in milliseconds for the history server - * to wake up and scan for files to be moved. - */ - public static final String HISTORY_SERVER_MOVE_THREAD_INTERVAL = - HS_PREFIX + "move.thread.interval"; - - /** - * The number of threads used to move files. - */ - public static final String HISTORY_SERVER_NUM_MOVE_THREADS = - HS_PREFIX + "move.threads.count"; - - // Equivalent to 0.20 mapreduce.jobhistory.debug.mode - public static final String HISTORY_DEBUG_MODE_KEY = HS_PREFIX + "debug.mode"; - - public static final String HISTORY_MAXAGE = - "yarn.historyfile.maxage"; - - //TODO Move some of the HistoryServer specific out into a separate configuration class. - public static final String HS_KEYTAB_KEY = HS_PREFIX + "keytab"; - - public static final String HS_SERVER_PRINCIPAL_KEY = "yarn.historyserver.principal"; - - public static final String RUN_HISTORY_CLEANER_KEY = - HS_PREFIX + "cleaner.run"; - - /** - * Run interval for the History Cleaner thread. - */ - public static final String HISTORY_CLEANER_RUN_INTERVAL = - HS_PREFIX + "cleaner.run.interval"; - - public static final String HS_WEBAPP_BIND_ADDRESS = HS_PREFIX + - "address.webapp"; - public static final String DEFAULT_HS_WEBAPP_BIND_ADDRESS = - "0.0.0.0:19888"; - - public static final String HS_CLIENT_THREADS = - HS_PREFIX + "client.threads"; - public static final int DEFAULT_HS_CLIENT_THREADS = 10; - -//From JTConfig. May need to be moved elsewhere. - public static final String JOBHISTORY_TASKPROGRESS_NUMBER_SPLITS_KEY = - "mapreduce.jobtracker.jobhistory.task.numberprogresssplits"; - - public static int DEFAULT_NUMBER_PROGRESS_SPLITS = 12; -} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java index ee3e60e77a..dcddd126cc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java @@ -38,8 +38,8 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TypeConverter; -import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.util.MRApps; import org.apache.hadoop.net.NetUtils; @@ -184,9 +184,9 @@ public static PathFilter getHistoryFileFilter() { public static String getConfiguredHistoryIntermediateDoneDirPrefix( Configuration conf) { String doneDirPrefix = conf - .get(JHConfig.HISTORY_INTERMEDIATE_DONE_DIR_KEY); + .get(JHAdminConfig.MR_HISTORY_INTERMEDIATE_DONE_DIR); if (doneDirPrefix == null) { - doneDirPrefix = conf.get(MRConstants.APPS_STAGING_DIR_KEY) + doneDirPrefix = conf.get(MRJobConfig.MR_AM_STAGING_DIR) + "/history/done_intermediate"; } return doneDirPrefix; @@ -199,9 +199,9 @@ public static String getConfiguredHistoryIntermediateDoneDirPrefix( */ public static String getConfiguredHistoryServerDoneDirPrefix( Configuration conf) { - String doneDirPrefix = conf.get(JHConfig.HISTORY_DONE_DIR_KEY); + String doneDirPrefix = conf.get(JHAdminConfig.MR_HISTORY_DONE_DIR); if (doneDirPrefix == null) { - doneDirPrefix = conf.get(MRConstants.APPS_STAGING_DIR_KEY) + doneDirPrefix = conf.get(MRJobConfig.MR_AM_STAGING_DIR) + "/history/done"; } return doneDirPrefix; @@ -220,7 +220,7 @@ public static String getHistoryIntermediateDoneDirForUser(Configuration conf) th public static boolean shouldCreateNonUserDirectory(Configuration conf) { // Returning true by default to allow non secure single node clusters to work // without any configuration change. - return conf.getBoolean(JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY, true); + return conf.getBoolean(MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR, true); } /** @@ -478,8 +478,8 @@ private static Path[] filteredStat2Paths(List stats, boolean dirs, public static String getHistoryUrl(Configuration conf, ApplicationId appId) throws UnknownHostException { //construct the history url for job - String hsAddress = conf.get(JHConfig.HS_WEBAPP_BIND_ADDRESS, - JHConfig.DEFAULT_HS_WEBAPP_BIND_ADDRESS); + String hsAddress = conf.get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS, + JHAdminConfig.DEFAULT_MR_HISTORY_WEBAPP_ADDRESS); InetSocketAddress address = NetUtils.createSocketAddr(hsAddress); StringBuffer sb = new StringBuffer(); if (address.getAddress().isAnyLocalAddress() || diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java index 2f25070636..9c8c63af1d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java @@ -21,7 +21,7 @@ import java.lang.annotation.Annotation; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; +import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; import org.apache.hadoop.security.KerberosInfo; import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.token.TokenInfo; @@ -44,7 +44,7 @@ public Class annotationType() { @Override public String serverPrincipal() { - return JHConfig.HS_SERVER_PRINCIPAL_KEY; + return JHAdminConfig.MR_HISTORY_PRINCIPAL; } @Override diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java index a6a5eddaf4..5dfa1dcfe4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java @@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; @@ -157,6 +158,7 @@ public static TaskAttemptStateUI taskAttemptState(String attemptStateStr) { public static void setInitialClasspath( Map environment) throws IOException { InputStream classpathFileStream = null; + BufferedReader reader = null; try { // Get yarn mapreduce-app classpath from generated classpath // Works if compile time env is same as runtime. Mainly tests. @@ -165,8 +167,7 @@ public static void setInitialClasspath( String mrAppGeneratedClasspathFile = "mrapp-generated-classpath"; classpathFileStream = thisClassLoader.getResourceAsStream(mrAppGeneratedClasspathFile); - BufferedReader reader = - new BufferedReader(new InputStreamReader(classpathFileStream)); + reader = new BufferedReader(new InputStreamReader(classpathFileStream)); String cp = reader.readLine(); if (cp != null) { addToClassPath(environment, cp.trim()); @@ -198,6 +199,9 @@ public static void setInitialClasspath( if (classpathFileStream != null) { classpathFileStream.close(); } + if (reader != null) { + reader.close(); + } } // TODO: Remove duplicates. } @@ -218,7 +222,7 @@ public static void addToClassPath( private static final String STAGING_CONSTANT = ".staging"; public static Path getStagingAreaDir(Configuration conf, String user) { return new Path( - conf.get(MRConstants.APPS_STAGING_DIR_KEY) + + conf.get(MRJobConfig.MR_AM_STAGING_DIR) + Path.SEPARATOR + user + Path.SEPARATOR + STAGING_CONSTANT); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java index 77fa446d58..7a2ee00a92 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java @@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; @@ -113,7 +114,7 @@ public class TestMRApps { @Test public void testGetJobFileWithUser() { Configuration conf = new Configuration(); - conf.set(MRConstants.APPS_STAGING_DIR_KEY, "/my/path/to/staging"); + conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/my/path/to/staging"); String jobFile = MRApps.getJobFile(conf, "dummy-user", new JobID("dummy-job", 12345)); assertNotNull("getJobFile results in null.", jobFile); assertEquals("jobFile with specified user is not as expected.", diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java index 194b80caf1..8a45bef3e6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java @@ -868,15 +868,6 @@ protected long getCounter(org.apache.hadoop.mapreduce.Counters cntrs, Counters counters = Counters.downgrade(cntrs); return counters.findCounter(counterGroupName, counterName).getValue(); } - - void displayJobList(JobStatus[] jobs) { - System.out.printf("JobId\tState\tStartTime\tUserName\tQueue\tPriority\tSchedulingInfo\n"); - for (JobStatus job : jobs) { - System.out.printf("%s\t%d\t%d\t%s\t%s\t%s\t%s\n", job.getJobID(), job.getRunState(), - job.getStartTime(), job.getUsername(), job.getQueue(), - job.getJobPriority().name(), job.getSchedulingInfo()); - } - } /** * Get status information about the max available Maps in the cluster. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java index 55ab70f759..33884bb82e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java @@ -301,4 +301,103 @@ public interface MRJobConfig { "mapreduce.ubertask.child.ulimit"; // or mapreduce.uber.ulimit? public static final String UBERTASK_ENV = "mapreduce.ubertask.child.env"; // or mapreduce.uber.env? + + public static final String MR_PREFIX = "yarn.app.mapreduce."; + + public static final String MR_AM_PREFIX = MR_PREFIX + "am."; + + /** The staging directory for map reduce.*/ + public static final String MR_AM_STAGING_DIR = + MR_AM_PREFIX+"staging-dir"; + + /** The amount of memory the MR app master needs.*/ + public static final String MR_AM_VMEM_MB = + MR_AM_PREFIX+"resource.mb"; + public static final int DEFAULT_MR_AM_VMEM_MB = 2048; + + /** Command line arguments passed to the MR app master.*/ + public static final String MR_AM_COMMAND_OPTS = + MR_AM_PREFIX+"command-opts"; + public static final String DEFAULT_MR_AM_COMMAND_OPTS = "-Xmx1536m"; + + /** Root Logging level passed to the MR app master.*/ + public static final String MR_AM_LOG_OPTS = + MR_AM_PREFIX+"log-opts"; + public static final String DEFAULT_MR_AM_LOG_OPTS = "INFO"; + + /**The number of splits when reporting progress in MR*/ + public static final String MR_AM_NUM_PROGRESS_SPLITS = + MR_AM_PREFIX+"num-progress-splits"; + public static final int DEFAULT_MR_AM_NUM_PROGRESS_SPLITS = 12; + + /** Number of threads user to launch containers in the app master.*/ + public static final String MR_AM_CONTAINERLAUNCHER_THREAD_COUNT = + MR_AM_PREFIX+"containerlauncher.thread-count"; + + /** Number of threads to handle job client RPC requests.*/ + public static final String MR_AM_JOB_CLIENT_THREAD_COUNT = + MR_AM_PREFIX + "job.client.thread-count"; + public static final int DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT = 1; + + /** Enable blacklisting of nodes in the job.*/ + public static final String MR_AM_JOB_NODE_BLACKLISTING_ENABLE = + MR_AM_PREFIX + "job.node.blacklisting.enable"; + + /** Enable job recovery.*/ + public static final String MR_AM_JOB_RECOVERY_ENABLE = + MR_AM_PREFIX + "job.recovery.enable"; + + /** + * Limit on the number of reducers that can be preempted to ensure that at + * least one map task can run if it needs to. Percentage between 0.0 and 1.0 + */ + public static final String MR_AM_JOB_REDUCE_PREEMPTION_LIMIT = + MR_AM_PREFIX + "job.reduce.preemption.limit"; + public static final float DEFAULT_MR_AM_JOB_REDUCE_PREEMPTION_LIMIT = 0.5f; + + /** + * Limit reduces starting until a certain percentage of maps have finished. + * Percentage between 0.0 and 1.0 + */ + public static final String MR_AM_JOB_REDUCE_RAMPUP_UP_LIMIT = + MR_AM_PREFIX + "job.reduce.rampup.limit"; + public static final float DEFAULT_MR_AM_JOB_REDUCE_RAMP_UP_LIMIT = 0.5f; + + /** The class that should be used for speculative execution calculations.*/ + public static final String MR_AM_JOB_SPECULATOR = + MR_AM_PREFIX + "job.speculator.class"; + + /** Class used to estimate task resource needs.*/ + public static final String MR_AM_TASK_ESTIMATOR = + MR_AM_PREFIX + "job.task.estimator.class"; + + /** The lambda value in the smoothing function of the task estimator.*/ + public static final String MR_AM_TASK_ESTIMATOR_SMOOTH_LAMBDA_MS = + MR_AM_PREFIX + + "job.task.estimator.exponential.smooth.lambda-ms"; + + public static final long DEFAULT_MR_AM_TASK_ESTIMATOR_SMNOOTH_LAMBDA_MS = + 1000L * 60; + + /** true if the smoothing rate should be exponential.*/ + public static final String MR_AM_TASK_EXTIMATOR_EXPONENTIAL_RATE_ENABLE = + MR_AM_PREFIX + "job.task.estimator.exponential.smooth.rate"; + + /** The number of threads used to handle task RPC calls.*/ + public static final String MR_AM_TASK_LISTENER_THREAD_COUNT = + MR_AM_PREFIX + "job.task.listener.thread-count"; + public static final int DEFAULT_MR_AM_TASK_LISTENER_THREAD_COUNT = 10; + + /** How often the AM should send heartbeats to the RM.*/ + public static final String MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS = + MR_AM_PREFIX + "scheduler.heartbeat.interval-ms"; + public static final int DEFAULT_MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS = 2000; + + /** + * Boolean. Create the base dirs in the JobHistoryEventHandler + * Set to false for multi-user clusters. This is an internal config that + * is set by the MR framework and read by it too. + */ + public static final String MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR = + MR_AM_PREFIX + "create-intermediate-jh-base-dir"; } 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 0ac7053cba..518bc4025d 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 @@ -523,7 +523,7 @@ protected void displayTasks(Job job, String type, String state) } } - protected void displayJobList(JobStatus[] jobs) + public void displayJobList(JobStatus[] jobs) throws IOException, InterruptedException { System.out.println("Total jobs:" + jobs.length); System.out.println("JobId\tState\tStartTime\t" + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java index 541773c987..23846786c9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java @@ -97,7 +97,7 @@ public abstract class ResourceCalculatorPlugin extends Configured { @InterfaceStability.Unstable public abstract ProcResourceValues getProcResourceValues(); - public class ProcResourceValues { + public static class ProcResourceValues { private final long cumulativeCpuTime; private final long physicalMemorySize; private final long virtualMemorySize; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml new file mode 100644 index 0000000000..f0d534da2a --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml @@ -0,0 +1,1182 @@ + + + + + + + + + + + + mapreduce.jobtracker.jobhistory.location + + If job tracker is static the history files are stored + in this single well known place. If No value is set here, by default, + it is in the local file system at ${hadoop.log.dir}/history. + + + + + mapreduce.jobtracker.jobhistory.task.numberprogresssplits + 12 + Every task attempt progresses from 0.0 to 1.0 [unless + it fails or is killed]. We record, for each task attempt, certain + statistics over each twelfth of the progress range. You can change + the number of intervals we divide the entire range of progress into + by setting this property. Higher values give more precision to the + recorded data, but costs more memory in the job tracker at runtime. + Each increment in this attribute costs 16 bytes per running task. + + + + + mapreduce.job.userhistorylocation + + User can specify a location to store the history files of + a particular job. If nothing is specified, the logs are stored in + output directory. The files are stored in "_logs/history/" in the directory. + User can stop logging by giving the value "none". + + + + + mapreduce.jobtracker.jobhistory.completed.location + + The completed job history files are stored at this single well + known location. If nothing is specified, the files are stored at + ${mapreduce.jobtracker.jobhistory.location}/done. + + + + + mapreduce.job.committer.setup.cleanup.needed + true + true, if job needs job-setup and job-cleanup. + false, otherwise + + + + + + mapreduce.task.io.sort.factor + 10 + The number of streams to merge at once while sorting + files. This determines the number of open file handles. + + + + mapreduce.task.io.sort.mb + 100 + The total amount of buffer memory to use while sorting + files, in megabytes. By default, gives each merge stream 1MB, which + should minimize seeks. + + + + mapreduce.map.sort.spill.percent + 0.80 + The soft limit in the serialization buffer. Once reached, a + thread will begin to spill the contents to disk in the background. Note that + collection will not block if this threshold is exceeded while a spill is + already in progress, so spills may be larger than this threshold when it is + set to less than .5 + + + + mapreduce.jobtracker.address + local + The host and port that the MapReduce job tracker runs + at. If "local", then jobs are run in-process as a single map + and reduce task. + + + + + mapreduce.local.clientfactory.class.name + org.apache.hadoop.mapred.LocalClientFactory + This the client factory that is responsible for + creating local job runner client + + + + mapreduce.jobtracker.http.address + 0.0.0.0:50030 + + The job tracker http server address and port the server will listen on. + If the port is 0 then the server will start on a free port. + + + + + mapreduce.jobtracker.handler.count + 10 + + The number of server threads for the JobTracker. This should be roughly + 4% of the number of tasktracker nodes. + + + + + mapreduce.tasktracker.report.address + 127.0.0.1:0 + The interface and port that task tracker server listens on. + Since it is only connected to by the tasks, it uses the local interface. + EXPERT ONLY. Should only be changed if your host does not have the loopback + interface. + + + + mapreduce.cluster.local.dir + ${hadoop.tmp.dir}/mapred/local + The local directory where MapReduce stores intermediate + data files. May be a comma-separated list of + directories on different devices in order to spread disk i/o. + Directories that do not exist are ignored. + + + + + mapreduce.jobtracker.system.dir + ${hadoop.tmp.dir}/mapred/system + The directory where MapReduce stores control files. + + + + + mapreduce.jobtracker.staging.root.dir + ${hadoop.tmp.dir}/mapred/staging + The root of the staging area for users' job files + In practice, this should be the directory where users' home + directories are located (usually /user) + + + + + mapreduce.cluster.temp.dir + ${hadoop.tmp.dir}/mapred/temp + A shared directory for temporary files. + + + + + mapreduce.tasktracker.local.dir.minspacestart + 0 + If the space in mapreduce.cluster.local.dir drops under this, + do not ask for more tasks. + Value in bytes. + + + + + mapreduce.tasktracker.local.dir.minspacekill + 0 + If the space in mapreduce.cluster.local.dir drops under this, + do not ask more tasks until all the current ones have finished and + cleaned up. Also, to save the rest of the tasks we have running, + kill one of them, to clean up some space. Start with the reduce tasks, + then go with the ones that have finished the least. + Value in bytes. + + + + + mapreduce.jobtracker.expire.trackers.interval + 600000 + Expert: The time-interval, in miliseconds, after which + a tasktracker is declared 'lost' if it doesn't send heartbeats. + + + + + mapreduce.tasktracker.instrumentation + org.apache.hadoop.mapred.TaskTrackerMetricsInst + Expert: The instrumentation class to associate with each TaskTracker. + + + + + mapreduce.tasktracker.resourcecalculatorplugin + + + Name of the class whose instance will be used to query resource information + on the tasktracker. + + The class must be an instance of + org.apache.hadoop.util.ResourceCalculatorPlugin. If the value is null, the + tasktracker attempts to use a class appropriate to the platform. + Currently, the only platform supported is Linux. + + + + + mapreduce.tasktracker.taskmemorymanager.monitoringinterval + 5000 + The interval, in milliseconds, for which the tasktracker waits + between two cycles of monitoring its tasks' memory usage. Used only if + tasks' memory management is enabled via mapred.tasktracker.tasks.maxmemory. + + + + + mapreduce.tasktracker.tasks.sleeptimebeforesigkill + 5000 + The time, in milliseconds, the tasktracker waits for sending a + SIGKILL to a task, after it has been sent a SIGTERM. This is currently + not used on WINDOWS where tasks are just sent a SIGTERM. + + + + + mapreduce.job.maps + 2 + The default number of map tasks per job. + Ignored when mapreduce.jobtracker.address is "local". + + + + + mapreduce.job.reduces + 1 + The default number of reduce tasks per job. Typically set to 99% + of the cluster's reduce capacity, so that if a node fails the reduces can + still be executed in a single wave. + Ignored when mapreduce.jobtracker.address is "local". + + + + + mapreduce.jobtracker.restart.recover + false + "true" to enable (job) recovery upon restart, + "false" to start afresh + + + + + mapreduce.jobtracker.jobhistory.block.size + 3145728 + The block size of the job history file. Since the job recovery + uses job history, its important to dump job history to disk as + soon as possible. Note that this is an expert level parameter. + The default value is set to 3 MB. + + + + + mapreduce.jobtracker.taskscheduler + org.apache.hadoop.mapred.JobQueueTaskScheduler + The class responsible for scheduling the tasks. + + + + + mapreduce.job.split.metainfo.maxsize + 10000000 + The maximum permissible size of the split metainfo file. + The JobTracker won't attempt to read split metainfo files bigger than + the configured value. + No limits if set to -1. + + + + + mapreduce.jobtracker.taskscheduler.maxrunningtasks.perjob + + The maximum number of running tasks for a job before + it gets preempted. No limits if undefined. + + + + + mapreduce.map.maxattempts + 4 + Expert: The maximum number of attempts per map task. + In other words, framework will try to execute a map task these many number + of times before giving up on it. + + + + + mapreduce.reduce.maxattempts + 4 + Expert: The maximum number of attempts per reduce task. + In other words, framework will try to execute a reduce task these many number + of times before giving up on it. + + + + + mapreduce.reduce.shuffle.parallelcopies + 5 + The default number of parallel transfers run by reduce + during the copy(shuffle) phase. + + + + + mapreduce.reduce.shuffle.connect.timeout + 180000 + Expert: The maximum amount of time (in milli seconds) reduce + task spends in trying to connect to a tasktracker for getting map output. + + + + + mapreduce.reduce.shuffle.read.timeout + 180000 + Expert: The maximum amount of time (in milli seconds) reduce + task waits for map output data to be available for reading after obtaining + connection. + + + + + mapreduce.task.timeout + 600000 + The number of milliseconds before a task will be + terminated if it neither reads an input, writes an output, nor + updates its status string. + + + + + mapreduce.tasktracker.map.tasks.maximum + 2 + The maximum number of map tasks that will be run + simultaneously by a task tracker. + + + + + mapreduce.tasktracker.reduce.tasks.maximum + 2 + The maximum number of reduce tasks that will be run + simultaneously by a task tracker. + + + + + mapreduce.jobtracker.retiredjobs.cache.size + 1000 + The number of retired job status to keep in the cache. + + + + + mapreduce.tasktracker.outofband.heartbeat + false + Expert: Set this to true to let the tasktracker send an + out-of-band heartbeat on task-completion for better latency. + + + + + mapreduce.jobtracker.jobhistory.lru.cache.size + 5 + The number of job history files loaded in memory. The jobs are + loaded when they are first accessed. The cache is cleared based on LRU. + + + + + mapreduce.jobtracker.instrumentation + org.apache.hadoop.mapred.JobTrackerMetricsInst + Expert: The instrumentation class to associate with each JobTracker. + + + + + mapred.child.java.opts + -Xmx200m + Java opts for the task tracker child processes. + The following symbol, if present, will be interpolated: @taskid@ is replaced + by current TaskID. Any other occurrences of '@' will go unchanged. + For example, to enable verbose gc logging to a file named for the taskid in + /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of: + -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc + + The configuration variable mapred.child.ulimit can be used to control the + maximum virtual memory of the child processes. + + + + + mapred.child.env + + User added environment variables for the task tracker child + processes. Example : + 1) A=foo This will set the env variable A to foo + 2) B=$B:c This is inherit tasktracker's B env variable. + + + + + mapred.child.ulimit + + The maximum virtual memory, in KB, of a process launched by the + Map-Reduce framework. This can be used to control both the Mapper/Reducer + tasks and applications using Hadoop Pipes, Hadoop Streaming etc. + By default it is left unspecified to let cluster admins control it via + limits.conf and other such relevant mechanisms. + + Note: mapred.child.ulimit must be greater than or equal to the -Xmx passed to + JavaVM, else the VM might not start. + + + + + mapreduce.task.tmp.dir + ./tmp + To set the value of tmp directory for map and reduce tasks. + If the value is an absolute path, it is directly assigned. Otherwise, it is + prepended with task's working directory. The java tasks are executed with + option -Djava.io.tmpdir='the absolute path of the tmp dir'. Pipes and + streaming are set with environment variable, + TMPDIR='the absolute path of the tmp dir' + + + + + mapreduce.map.log.level + INFO + The logging level for the map task. The allowed levels are: + OFF, FATAL, ERROR, WARN, INFO, DEBUG, TRACE and ALL. + + + + + mapreduce.reduce.log.level + INFO + The logging level for the reduce task. The allowed levels are: + OFF, FATAL, ERROR, WARN, INFO, DEBUG, TRACE and ALL. + + + + + mapreduce.reduce.merge.inmem.threshold + 1000 + The threshold, in terms of the number of files + for the in-memory merge process. When we accumulate threshold number of files + we initiate the in-memory merge and spill to disk. A value of 0 or less than + 0 indicates we want to DON'T have any threshold and instead depend only on + the ramfs's memory consumption to trigger the merge. + + + + + mapreduce.reduce.shuffle.merge.percent + 0.66 + The usage threshold at which an in-memory merge will be + initiated, expressed as a percentage of the total memory allocated to + storing in-memory map outputs, as defined by + mapreduce.reduce.shuffle.input.buffer.percent. + + + + + mapreduce.reduce.shuffle.input.buffer.percent + 0.70 + The percentage of memory to be allocated from the maximum heap + size to storing map outputs during the shuffle. + + + + + mapreduce.reduce.input.buffer.percent + 0.0 + The percentage of memory- relative to the maximum heap size- to + retain map outputs during the reduce. When the shuffle is concluded, any + remaining map outputs in memory must consume less than this threshold before + the reduce can begin. + + + + + mapreduce.reduce.markreset.buffer.percent + 0.0 + The percentage of memory -relative to the maximum heap size- to + be used for caching values when using the mark-reset functionality. + + + + + mapreduce.map.speculative + true + If true, then multiple instances of some map tasks + may be executed in parallel. + + + + mapreduce.reduce.speculative + true + If true, then multiple instances of some reduce tasks + may be executed in parallel. + + + mapreduce.job.speculative.speculativecap + 0.1 + The max percent (0-1) of running tasks that + can be speculatively re-executed at any time. + + + + mapreduce.job.speculative.slowtaskthreshold + 1.0The number of standard deviations by which a task's + ave progress-rates must be lower than the average of all running tasks' + for the task to be considered too slow. + + + + + + mapreduce.job.speculative.slownodethreshold + 1.0 + The number of standard deviations by which a Task + Tracker's ave map and reduce progress-rates (finishTime-dispatchTime) + must be lower than the average of all successful map/reduce task's for + the TT to be considered too slow to give a speculative task to. + + + + + mapreduce.job.jvm.numtasks + 1 + How many tasks to run per jvm. If set to -1, there is + no limit. + + + + + mapreduce.job.ubertask.enable + false + Whether to enable the small-jobs "ubertask" optimization, + which runs "sufficiently small" jobs sequentially within a single JVM. + "Small" is defined by the following maxmaps, maxreduces, and maxbytes + settings. Users may override this value. + + + + + mapreduce.job.ubertask.maxmaps + 9 + Threshold for number of maps, beyond which job is considered + too big for the ubertasking optimization. Users may override this value, + but only downward. + + + + + mapreduce.job.ubertask.maxreduces + 1 + Threshold for number of reduces, beyond which job is considered + too big for the ubertasking optimization. CURRENTLY THE CODE CANNOT SUPPORT + MORE THAN ONE REDUCE and will ignore larger values. (Zero is a valid max, + however.) Users may override this value, but only downward. + + + + + mapreduce.job.ubertask.maxbytes + + Threshold for number of input bytes, beyond which job is + considered too big for the ubertasking optimization. If no value is + specified, dfs.block.size is used as a default. Be sure to specify a + default value in mapred-site.xml if the underlying filesystem is not HDFS. + Users may override this value, but only downward. + + + + + mapreduce.input.fileinputformat.split.minsize + 0 + The minimum size chunk that map input should be split + into. Note that some file formats may have minimum split sizes that + take priority over this setting. + + + + mapreduce.jobtracker.maxtasks.perjob + -1 + The maximum number of tasks for a single job. + A value of -1 indicates that there is no maximum. + + + + mapreduce.client.submit.file.replication + 10 + The replication level for submitted job files. This + should be around the square root of the number of nodes. + + + + + + mapreduce.tasktracker.dns.interface + default + The name of the Network Interface from which a task + tracker should report its IP address. + + + + + mapreduce.tasktracker.dns.nameserver + default + The host name or IP address of the name server (DNS) + which a TaskTracker should use to determine the host name used by + the JobTracker for communication and display purposes. + + + + + mapreduce.tasktracker.http.threads + 40 + The number of worker threads that for the http server. This is + used for map output fetching + + + + + mapreduce.tasktracker.http.address + 0.0.0.0:50060 + + The task tracker http server address and port. + If the port is 0 then the server will start on a free port. + + + + + mapreduce.task.files.preserve.failedtasks + false + Should the files for failed tasks be kept. This should only be + used on jobs that are failing, because the storage is never + reclaimed. It also prevents the map outputs from being erased + from the reduce directory as they are consumed. + + + + + + + mapreduce.output.fileoutputformat.compress + false + Should the job outputs be compressed? + + + + + mapreduce.output.fileoutputformat.compression.type + RECORD + If the job outputs are to compressed as SequenceFiles, how should + they be compressed? Should be one of NONE, RECORD or BLOCK. + + + + + mapreduce.output.fileoutputformat.compression.codec + org.apache.hadoop.io.compress.DefaultCodec + If the job outputs are compressed, how should they be compressed? + + + + + mapreduce.map.output.compress + false + Should the outputs of the maps be compressed before being + sent across the network. Uses SequenceFile compression. + + + + + mapreduce.map.output.compress.codec + org.apache.hadoop.io.compress.DefaultCodec + If the map outputs are compressed, how should they be + compressed? + + + + + map.sort.class + org.apache.hadoop.util.QuickSort + The default sort class for sorting keys. + + + + + mapreduce.task.userlog.limit.kb + 0 + The maximum size of user-logs of each task in KB. 0 disables the cap. + + + + + mapreduce.job.userlog.retain.hours + 24 + The maximum time, in hours, for which the user-logs are to be + retained after the job completion. + + + + + mapreduce.jobtracker.hosts.filename + + Names a file that contains the list of nodes that may + connect to the jobtracker. If the value is empty, all hosts are + permitted. + + + + mapreduce.jobtracker.hosts.exclude.filename + + Names a file that contains the list of hosts that + should be excluded by the jobtracker. If the value is empty, no + hosts are excluded. + + + + mapreduce.jobtracker.heartbeats.in.second + 100 + Expert: Approximate number of heart-beats that could arrive + at JobTracker in a second. Assuming each RPC can be processed + in 10msec, the default value is made 100 RPCs in a second. + + + + + mapreduce.jobtracker.tasktracker.maxblacklists + 4 + The number of blacklists for a taskTracker by various jobs + after which the task tracker could be blacklisted across + all jobs. The tracker will be given a tasks later + (after a day). The tracker will become a healthy + tracker after a restart. + + + + + mapreduce.job.maxtaskfailures.per.tracker + 4 + The number of task-failures on a tasktracker of a given job + after which new tasks of that job aren't assigned to it. + + + + + mapreduce.client.output.filter + FAILED + The filter for controlling the output of the task's userlogs sent + to the console of the JobClient. + The permissible options are: NONE, KILLED, FAILED, SUCCEEDED and + ALL. + + + + + mapreduce.client.completion.pollinterval + 5000 + The interval (in milliseconds) between which the JobClient + polls the JobTracker for updates about job status. You may want to set this + to a lower value to make tests run faster on a single node system. Adjusting + this value in production may lead to unwanted client-server traffic. + + + + + mapreduce.client.progressmonitor.pollinterval + 1000 + The interval (in milliseconds) between which the JobClient + reports status to the console and checks for job completion. You may want to set this + to a lower value to make tests run faster on a single node system. Adjusting + this value in production may lead to unwanted client-server traffic. + + + + + mapreduce.jobtracker.persist.jobstatus.active + true + Indicates if persistency of job status information is + active or not. + + + + + mapreduce.jobtracker.persist.jobstatus.hours + 1 + The number of hours job status information is persisted in DFS. + The job status information will be available after it drops of the memory + queue and between jobtracker restarts. With a zero value the job status + information is not persisted at all in DFS. + + + + + mapreduce.jobtracker.persist.jobstatus.dir + /jobtracker/jobsInfo + The directory where the job status information is persisted + in a file system to be available after it drops of the memory queue and + between jobtracker restarts. + + + + + mapreduce.task.profile + false + To set whether the system should collect profiler + information for some of the tasks in this job? The information is stored + in the user log directory. The value is "true" if task profiling + is enabled. + + + + mapreduce.task.profile.maps + 0-2 + To set the ranges of map tasks to profile. + mapreduce.task.profile has to be set to true for the value to be accounted. + + + + + mapreduce.task.profile.reduces + 0-2 + To set the ranges of reduce tasks to profile. + mapreduce.task.profile has to be set to true for the value to be accounted. + + + + + mapreduce.task.skip.start.attempts + 2 + The number of Task attempts AFTER which skip mode + will be kicked off. When skip mode is kicked off, the + tasks reports the range of records which it will process + next, to the TaskTracker. So that on failures, TT knows which + ones are possibly the bad records. On further executions, + those are skipped. + + + + + mapreduce.map.skip.proc.count.autoincr + true + The flag which if set to true, + SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS is incremented + by MapRunner after invoking the map function. This value must be set to + false for applications which process the records asynchronously + or buffer the input records. For example streaming. + In such cases applications should increment this counter on their own. + + + + + mapreduce.reduce.skip.proc.count.autoincr + true + The flag which if set to true, + SkipBadRecords.COUNTER_REDUCE_PROCESSED_GROUPS is incremented + by framework after invoking the reduce function. This value must be set to + false for applications which process the records asynchronously + or buffer the input records. For example streaming. + In such cases applications should increment this counter on their own. + + + + + mapreduce.job.skip.outdir + + If no value is specified here, the skipped records are + written to the output directory at _logs/skip. + User can stop writing skipped records by giving the value "none". + + + + + mapreduce.map.skip.maxrecords + 0 + The number of acceptable skip records surrounding the bad + record PER bad record in mapper. The number includes the bad record as well. + To turn the feature of detection/skipping of bad records off, set the + value to 0. + The framework tries to narrow down the skipped range by retrying + until this threshold is met OR all attempts get exhausted for this task. + Set the value to Long.MAX_VALUE to indicate that framework need not try to + narrow down. Whatever records(depends on application) get skipped are + acceptable. + + + + + mapreduce.reduce.skip.maxgroups + 0 + The number of acceptable skip groups surrounding the bad + group PER bad group in reducer. The number includes the bad group as well. + To turn the feature of detection/skipping of bad groups off, set the + value to 0. + The framework tries to narrow down the skipped range by retrying + until this threshold is met OR all attempts get exhausted for this task. + Set the value to Long.MAX_VALUE to indicate that framework need not try to + narrow down. Whatever groups(depends on application) get skipped are + acceptable. + + + + + + + + + mapreduce.job.end-notification.retry.attempts + 0 + Indicates how many times hadoop should attempt to contact the + notification URL + + + + mapreduce.job.end-notification.retry.interval + 30000 + Indicates time in milliseconds between notification URL retry + calls + + + + + mapreduce.jobtracker.taskcache.levels + 2 + This is the max level of the task cache. For example, if + the level is 2, the tasks cached are at the host level and at the rack + level. + + + + + mapreduce.job.queuename + default + Queue to which a job is submitted. This must match one of the + queues defined in mapred-queues.xml for the system. Also, the ACL setup + for the queue must allow the current user to submit a job to the queue. + Before specifying a queue, ensure that the system is configured with + the queue, and access is allowed for submitting jobs to the queue. + + + + + mapreduce.cluster.acls.enabled + false + Specifies whether ACLs should be checked + for authorization of users for doing various queue and job level operations. + ACLs are disabled by default. If enabled, access control checks are made by + JobTracker and TaskTracker when requests are made by users for queue + operations like submit job to a queue and kill a job in the queue and job + operations like viewing the job-details (See mapreduce.job.acl-view-job) + or for modifying the job (See mapreduce.job.acl-modify-job) using + Map/Reduce APIs, RPCs or via the console and web user interfaces. + For enabling this flag(mapreduce.cluster.acls.enabled), this is to be set + to true in mapred-site.xml on JobTracker node and on all TaskTracker nodes. + + + + + mapreduce.job.acl-modify-job + + Job specific access-control list for 'modifying' the job. It + is only used if authorization is enabled in Map/Reduce by setting the + configuration property mapreduce.cluster.acls.enabled to true. + This specifies the list of users and/or groups who can do modification + operations on the job. For specifying a list of users and groups the + format to use is "user1,user2 group1,group". If set to '*', it allows all + users/groups to modify this job. If set to ' '(i.e. space), it allows + none. This configuration is used to guard all the modifications with respect + to this job and takes care of all the following operations: + o killing this job + o killing a task of this job, failing a task of this job + o setting the priority of this job + Each of these operations are also protected by the per-queue level ACL + "acl-administer-jobs" configured via mapred-queues.xml. So a caller should + have the authorization to satisfy either the queue-level ACL or the + job-level ACL. + + Irrespective of this ACL configuration, (a) job-owner, (b) the user who + started the cluster, (c) members of an admin configured supergroup + configured via mapreduce.cluster.permissions.supergroup and (d) queue + administrators of the queue to which this job was submitted to configured + via acl-administer-jobs for the specific queue in mapred-queues.xml can + do all the modification operations on a job. + + By default, nobody else besides job-owner, the user who started the cluster, + members of supergroup and queue administrators can perform modification + operations on a job. + + + + + mapreduce.job.acl-view-job + + Job specific access-control list for 'viewing' the job. It is + only used if authorization is enabled in Map/Reduce by setting the + configuration property mapreduce.cluster.acls.enabled to true. + This specifies the list of users and/or groups who can view private details + about the job. For specifying a list of users and groups the + format to use is "user1,user2 group1,group". If set to '*', it allows all + users/groups to modify this job. If set to ' '(i.e. space), it allows + none. This configuration is used to guard some of the job-views and at + present only protects APIs that can return possibly sensitive information + of the job-owner like + o job-level counters + o task-level counters + o tasks' diagnostic information + o task-logs displayed on the TaskTracker web-UI and + o job.xml showed by the JobTracker's web-UI + Every other piece of information of jobs is still accessible by any other + user, for e.g., JobStatus, JobProfile, list of jobs in the queue, etc. + + Irrespective of this ACL configuration, (a) job-owner, (b) the user who + started the cluster, (c) members of an admin configured supergroup + configured via mapreduce.cluster.permissions.supergroup and (d) queue + administrators of the queue to which this job was submitted to configured + via acl-administer-jobs for the specific queue in mapred-queues.xml can + do all the view operations on a job. + + By default, nobody else besides job-owner, the user who started the + cluster, memebers of supergroup and queue administrators can perform + view operations on a job. + + + + + mapreduce.tasktracker.indexcache.mb + 10 + The maximum memory that a task tracker allows for the + index cache that is used when serving map outputs to reducers. + + + + + mapreduce.task.merge.progress.records + 10000 + The number of records to process during merge before + sending a progress notification to the TaskTracker. + + + + + mapreduce.job.reduce.slowstart.completedmaps + 0.05 + Fraction of the number of maps in the job which should be + complete before reduces are scheduled for the job. + + + + +mapreduce.job.complete.cancel.delegation.tokens + true + if false - do not unregister/cancel delegation tokens from + renewal, because same tokens may be used by spawned jobs + + + + + mapreduce.tasktracker.taskcontroller + org.apache.hadoop.mapred.DefaultTaskController + TaskController which is used to launch and manage task execution + + + + + mapreduce.tasktracker.group + + Expert: Group to which TaskTracker belongs. If + LinuxTaskController is configured via mapreduce.tasktracker.taskcontroller, + the group owner of the task-controller binary should be same as this group. + + + + + + + mapreduce.tasktracker.healthchecker.script.path + + Absolute path to the script which is + periodicallyrun by the node health monitoring service to determine if + the node is healthy or not. If the value of this key is empty or the + file does not exist in the location configured here, the node health + monitoring service is not started. + + + + mapreduce.tasktracker.healthchecker.interval + 60000 + Frequency of the node health script to be run, + in milliseconds + + + + mapreduce.tasktracker.healthchecker.script.timeout + 600000 + Time after node health script should be killed if + unresponsive and considered that the script has failed. + + + + mapreduce.tasktracker.healthchecker.script.args + + List of arguments which are to be passed to + node health script when it is being launched comma seperated. + + + + + + + mapreduce.job.counters.limit + 120 + Limit on the number of user counters allowed per job. + + + + + mapreduce.framework.name + yarn + The runtime framework for executing MapReduce jobs. + Can be one of local, classic or yarn. + + + + + yarn.app.mapreduce.am.staging-dir + /tmp/hadoop-yarn/${user.name}/staging + The staging dir used while submitting jobs. + + + + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml index f678d6be76..b9e409cad6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml @@ -43,6 +43,12 @@ test-jar test + + org.apache.hadoop + hadoop-yarn-common + test-jar + test + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java index d29139f2df..3af30088d8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java @@ -279,7 +279,8 @@ public int getTotalReduces() { @Override public boolean isUber() { - throw new YarnException("Not yet implemented!"); + LOG.warn("isUber is not yet implemented"); + return false; } @Override diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTaskAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTaskAttempt.java index 5f303440d0..67d676ce8a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTaskAttempt.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTaskAttempt.java @@ -29,6 +29,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; @@ -82,12 +83,23 @@ public class CompletedTaskAttempt implements TaskAttempt { @Override public ContainerId getAssignedContainerID() { - //TODO ContainerId needs to be part of some historyEvent to be able to render the log directory. - ContainerId containerId = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ContainerId.class); + //TODO ContainerId needs to be part of some historyEvent to be able to + //render the log directory. + ContainerId containerId = + RecordFactoryProvider.getRecordFactory(null).newRecordInstance( + ContainerId.class); containerId.setId(-1); - containerId.setAppId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class)); - containerId.getAppId().setId(-1); - containerId.getAppId().setClusterTimestamp(-1); + ApplicationAttemptId applicationAttemptId = + RecordFactoryProvider.getRecordFactory(null).newRecordInstance( + ApplicationAttemptId.class); + applicationAttemptId.setAttemptId(-1); + ApplicationId applicationId = + RecordFactoryProvider.getRecordFactory(null).newRecordInstance( + ApplicationId.class); + applicationId.setClusterTimestamp(-1); + applicationId.setId(-1); + applicationAttemptId.setApplicationId(applicationId); + containerId.setApplicationAttemptId(applicationAttemptId); return containerId; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java index a848edcd6e..56f114adc5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java @@ -62,8 +62,8 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Task; -import org.apache.hadoop.mapreduce.v2.hs.webapp.HSWebApp; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; +import org.apache.hadoop.mapreduce.v2.hs.webapp.HsWebApp; +import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; import org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityInfo; @@ -107,8 +107,8 @@ public void start() { YarnConfiguration.YARN_SECURITY_INFO, ClientHSSecurityInfo.class, SecurityInfo.class); initializeWebApp(getConfig()); - String serviceAddr = conf.get(JHConfig.HS_BIND_ADDRESS, - JHConfig.DEFAULT_HS_BIND_ADDRESS); + String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS, + JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS); InetSocketAddress address = NetUtils.createSocketAddr(serviceAddr); InetAddress hostNameResolved = null; try { @@ -120,8 +120,8 @@ public void start() { server = rpc.getServer(MRClientProtocol.class, protocolHandler, address, conf, null, - conf.getInt(JHConfig.HS_CLIENT_THREADS, - JHConfig.DEFAULT_HS_CLIENT_THREADS)); + conf.getInt(JHAdminConfig.MR_HISTORY_CLIENT_THREAD_COUNT, + JHAdminConfig.DEFAULT_MR_HISTORY_CLIENT_THREAD_COUNT)); server.start(); this.bindAddress = NetUtils.createSocketAddr(hostNameResolved.getHostAddress() @@ -132,9 +132,9 @@ public void start() { } private void initializeWebApp(Configuration conf) { - webApp = new HSWebApp(history); - String bindAddress = conf.get(JHConfig.HS_WEBAPP_BIND_ADDRESS, - JHConfig.DEFAULT_HS_WEBAPP_BIND_ADDRESS); + webApp = new HsWebApp(history); + String bindAddress = conf.get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS, + JHAdminConfig.DEFAULT_MR_HISTORY_WEBAPP_ADDRESS); WebApps.$for("yarn", this).at(bindAddress).start(webApp); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java index 389de0bd70..5fdea87d92 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java @@ -57,7 +57,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; +import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo; import org.apache.hadoop.yarn.Clock; @@ -184,7 +184,7 @@ public void init(Configuration conf) throws YarnException { this.appAttemptID = RecordFactoryProvider.getRecordFactory(conf) .newRecordInstance(ApplicationAttemptId.class); - debugMode = conf.getBoolean(JHConfig.HISTORY_DEBUG_MODE_KEY, false); + debugMode = conf.getBoolean(JHAdminConfig.MR_HISTORY_DEBUG_MODE, false); serialNumberLowDigits = debugMode ? 1 : 3; serialNumberFormat = ("%0" + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits) + "d"); @@ -216,15 +216,15 @@ public void init(Configuration conf) throws YarnException { - jobListCacheSize = conf.getInt(JHConfig.HISTORY_SERVER_JOBLIST_CACHE_SIZE_KEY, DEFAULT_JOBLIST_CACHE_SIZE); - loadedJobCacheSize = conf.getInt(JHConfig.HISTORY_SERVER_LOADED_JOB_CACHE_SIZE_KEY, DEFAULT_LOADEDJOB_CACHE_SIZE); - dateStringCacheSize = conf.getInt(JHConfig.HISTORY_SERVER_DATESTRING_CACHE_SIZE_KEY, DEFAULT_DATESTRING_CACHE_SIZE); + jobListCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_JOBLIST_CACHE_SIZE, DEFAULT_JOBLIST_CACHE_SIZE); + loadedJobCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_LOADED_JOB_CACHE_SIZE, DEFAULT_LOADEDJOB_CACHE_SIZE); + dateStringCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_DATESTRING_CACHE_SIZE, DEFAULT_DATESTRING_CACHE_SIZE); moveThreadInterval = - conf.getLong(JHConfig.HISTORY_SERVER_MOVE_THREAD_INTERVAL, + conf.getLong(JHAdminConfig.MR_HISTORY_MOVE_INTERVAL_MS, DEFAULT_MOVE_THREAD_INTERVAL); - numMoveThreads = conf.getInt(JHConfig.HISTORY_SERVER_NUM_MOVE_THREADS, DEFAULT_MOVE_THREAD_COUNT); + numMoveThreads = conf.getInt(JHAdminConfig.MR_HISTORY_MOVE_THREAD_COUNT, DEFAULT_MOVE_THREAD_COUNT); try { - initExisting(); + initExisting(); } catch (IOException e) { throw new YarnException("Failed to intialize existing directories", e); } @@ -260,12 +260,12 @@ public void start() { moveIntermediateToDoneThread.start(); //Start historyCleaner - boolean startCleanerService = conf.getBoolean(JHConfig.RUN_HISTORY_CLEANER_KEY, true); + boolean startCleanerService = conf.getBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, true); if (startCleanerService) { - long maxAgeOfHistoryFiles = conf.getLong(JHConfig.HISTORY_MAXAGE, + long maxAgeOfHistoryFiles = conf.getLong(JHAdminConfig.MR_HISTORY_MAX_AGE_MS, DEFAULT_HISTORY_MAX_AGE); cleanerScheduledExecutor = new ScheduledThreadPoolExecutor(1); - long runInterval = conf.getLong(JHConfig.HISTORY_CLEANER_RUN_INTERVAL, + long runInterval = conf.getLong(JHAdminConfig.MR_HISTORY_CLEANER_INTERVAL_MS, DEFAULT_RUN_INTERVAL); cleanerScheduledExecutor .scheduleAtFixedRate(new HistoryCleaner(maxAgeOfHistoryFiles), @@ -319,6 +319,7 @@ public JobHistory() { */ @SuppressWarnings("unchecked") private void initExisting() throws IOException { + LOG.info("Initializing Existing Jobs..."); List timestampedDirList = findTimestampedDirectories(); Collections.sort(timestampedDirList); for (FileStatus fs : timestampedDirList) { @@ -350,6 +351,9 @@ private void removeDirectoryFromSerialNumberIndex(Path serialDirPath) { } private void addDirectoryToSerialNumberIndex(Path serialDirPath) { + if(LOG.isDebugEnabled()) { + LOG.debug("Adding "+serialDirPath+" to serial index"); + } String serialPart = serialDirPath.getName(); String timestampPart = JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString()); if (timestampPart == null) { @@ -374,9 +378,15 @@ private void addToSerialNumberIndex(String serialPart, String timestampPart) { } private void addDirectoryToJobListCache(Path path) throws IOException { + if(LOG.isDebugEnabled()) { + LOG.debug("Adding "+path+" to job list cache."); + } List historyFileList = scanDirectoryForHistoryFiles(path, doneDirFc); for (FileStatus fs : historyFileList) { + if(LOG.isDebugEnabled()) { + LOG.debug("Adding in history for "+fs.getPath()); + } JobIndexInfo jobIndexInfo = FileNameIndexUtils.getIndexInfo(fs.getPath() .getName()); String confFileName = JobHistoryUtils @@ -423,6 +433,9 @@ private List findTimestampedDirectories() throws IOException { * Adds an entry to the job list cache. Maintains the size. */ private void addToJobListCache(JobId jobId, MetaInfo metaInfo) { + if(LOG.isDebugEnabled()) { + LOG.debug("Adding "+jobId+" to job list cache with "+metaInfo.getJobIndexInfo()); + } jobListCache.put(jobId, metaInfo); if (jobListCache.size() > jobListCacheSize) { jobListCache.remove(jobListCache.firstKey()); @@ -432,7 +445,10 @@ private void addToJobListCache(JobId jobId, MetaInfo metaInfo) { /** * Adds an entry to the loaded job cache. Maintains the size. */ - private void addToLoadedJobCache(Job job) { + private void addToLoadedJobCache(Job job) { + if(LOG.isDebugEnabled()) { + LOG.debug("Adding "+job.getID()+" to loaded job cache"); + } loadedJobCache.put(job.getID(), job); if (loadedJobCache.size() > loadedJobCacheSize ) { loadedJobCache.remove(loadedJobCache.firstKey()); @@ -967,6 +983,9 @@ private Path canonicalHistoryLogPath(JobId id, long millisecondTime) { @Override public synchronized Job getJob(JobId jobId) { + if(LOG.isDebugEnabled()) { + LOG.debug("Looking for Job "+jobId); + } Job job = null; try { job = findJob(jobId); @@ -979,7 +998,9 @@ public synchronized Job getJob(JobId jobId) { @Override public Map getAllJobs(ApplicationId appID) { - LOG.info("Called getAllJobs(AppId): " + appID); + if(LOG.isDebugEnabled()) { + LOG.debug("Called getAllJobs(AppId): " + appID); + } // currently there is 1 to 1 mapping between app and job id org.apache.hadoop.mapreduce.JobID oldJobID = TypeConverter.fromYarn(appID); Map jobs = new HashMap(); @@ -1002,12 +1023,9 @@ public Map getAllJobs(ApplicationId appID) { * This does involve a DFS oepration of scanning the intermediate directory. */ public Map getAllJobs() { + LOG.debug("Called getAllJobs()"); return getAllJobsInternal(); - } - - - - + } static class MetaInfo { private Path historyFile; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java index 03bf3a4a95..73ef9feaa2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java @@ -24,7 +24,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; +import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.YarnException; @@ -68,8 +68,8 @@ public synchronized void init(Configuration conf) { } protected void doSecureLogin(Configuration conf) throws IOException { - SecurityUtil.login(conf, JHConfig.HS_KEYTAB_KEY, - JHConfig.HS_SERVER_PRINCIPAL_KEY); + SecurityUtil.login(conf, JHAdminConfig.MR_HISTORY_KEYTAB, + JHAdminConfig.MR_HISTORY_PRINCIPAL); } public static void main(String[] args) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java index b90bd338b3..71f4b027d2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java @@ -46,6 +46,8 @@ public PartialJob(JobIndexInfo jobIndexInfo, JobId jobId) { this.jobIndexInfo = jobIndexInfo; this.jobId = jobId; jobReport = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobReport.class); + jobReport.setStartTime(jobIndexInfo.getSubmitTime()); + jobReport.setFinishTime(jobIndexInfo.getFinishTime()); } @Override @@ -142,7 +144,7 @@ public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) @Override public String getUserName() { - return null; + return jobIndexInfo.getUser(); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java new file mode 100644 index 0000000000..4c9f667b5b --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java @@ -0,0 +1,52 @@ +/** +* 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.v2.hs.webapp; + +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; + +import org.apache.hadoop.util.VersionInfo; +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.view.InfoBlock; + +/** + * A Page the shows info about the history server + */ +public class HsAboutPage extends HsView { + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ + @Override protected void preHead(Page.HTML<_> html) { + commonPreHead(html); + //override the nav config from commonPReHead + set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}"); + } + + /** + * The content of this page is the attempts block + * @return AttemptsBlock.class + */ + @Override protected Class content() { + info("History Server"). + _("BuildVersion", VersionInfo.getBuildVersion()); + return InfoBlock.class; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java index fab300853b..a0e36cf90b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java @@ -21,41 +21,123 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.v2.app.webapp.App; import org.apache.hadoop.mapreduce.v2.app.webapp.AppController; +import org.apache.hadoop.yarn.webapp.View; import com.google.inject.Inject; +/** + * This class renders the various pages that the History Server WebApp supports + */ public class HsController extends AppController { @Inject HsController(App app, Configuration conf, RequestContext ctx) { super(app, conf, ctx, "History"); } + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#index() + */ @Override public void index() { - // TODO Auto-generated method stub setTitle("JobHistory"); } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#jobPage() + */ + @Override + protected Class jobPage() { + return HsJobPage.class; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#countersPage() + */ + @Override + protected Class countersPage() { + return HsCountersPage.class; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#tasksPage() + */ + @Override + protected Class tasksPage() { + return HsTasksPage.class; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#taskPage() + */ + @Override + protected Class taskPage() { + return HsTaskPage.class; + } // Need all of these methods here also as Guice doesn't look into parent // classes. + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#job() + */ + @Override public void job() { super.job(); } + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#jobCounters() + */ + @Override public void jobCounters() { super.jobCounters(); } + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#tasks() + */ + @Override public void tasks() { super.tasks(); } + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#task() + */ + @Override public void task() { super.task(); } + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#attempts() + */ @Override public void attempts() { super.attempts(); } + + /** + * @return the page about the current server. + */ + protected Class aboutPage() { + return HsAboutPage.class; + } + + /** + * Render a page about the current server. + */ + public void about() { + render(aboutPage()); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java new file mode 100644 index 0000000000..0840f91357 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java @@ -0,0 +1,63 @@ +/** +* 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.v2.hs.webapp; + +import org.apache.hadoop.mapreduce.v2.app.webapp.CountersBlock; +import org.apache.hadoop.yarn.webapp.SubView; + +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*; + +/** + * Render the counters page + */ +public class HsCountersPage extends HsView { + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ + @Override protected void preHead(Page.HTML<_> html) { + commonPreHead(html); + set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}"); + set(DATATABLES_SELECTOR, "#counters .dt-counters"); + set(initSelector(DATATABLES), + "{bJQueryUI:true, sDom:'t', iDisplayLength:-1}"); + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.yarn.webapp.view.TwoColumnLayout#postHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ + @Override protected void postHead(Page.HTML<_> html) { + html. + style("#counters, .dt-counters { table-layout: fixed }", + "#counters th { overflow: hidden; vertical-align: center }", + "#counters .dataTables_wrapper { min-height: 1em }", + "#counters .group { width: 10em }", + "#counters .name { width: 30em }"); + } + + /** + * The content of this page is the CountersBlock now. + * @return CountersBlock.class + */ + @Override protected Class content() { + return CountersBlock.class; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java new file mode 100644 index 0000000000..07cd073693 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java @@ -0,0 +1,266 @@ +/** +* 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.v2.hs.webapp; + +import com.google.inject.Inject; +import java.util.Date; +import java.util.Map; + +import org.apache.hadoop.mapreduce.v2.api.records.JobId; +import org.apache.hadoop.mapreduce.v2.api.records.JobReport; +import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; +import org.apache.hadoop.mapreduce.v2.api.records.TaskId; +import org.apache.hadoop.mapreduce.v2.app.AppContext; +import org.apache.hadoop.mapreduce.v2.app.job.Job; +import org.apache.hadoop.mapreduce.v2.app.job.Task; +import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt; +import org.apache.hadoop.mapreduce.v2.util.MRApps; +import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.util.Times; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; +import org.apache.hadoop.yarn.webapp.view.InfoBlock; +import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*; +import static org.apache.hadoop.yarn.util.StringHelper.*; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*; + +/** + * Render a block of HTML for a give job. + */ +public class HsJobBlock extends HtmlBlock { + final AppContext appContext; + + int runningMapTasks = 0; + int pendingMapTasks = 0; + int runningReduceTasks = 0; + int pendingReduceTasks = 0; + + int newMapAttempts = 0; + int runningMapAttempts = 0; + int killedMapAttempts = 0; + int failedMapAttempts = 0; + int successfulMapAttempts = 0; + int newReduceAttempts = 0; + int runningReduceAttempts = 0; + int killedReduceAttempts = 0; + int failedReduceAttempts = 0; + int successfulReduceAttempts = 0; + + @Inject HsJobBlock(AppContext appctx) { + appContext = appctx; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block) + */ + @Override protected void render(Block html) { + String jid = $(JOB_ID); + if (jid.isEmpty()) { + html. + p()._("Sorry, can't do anything without a JobID.")._(); + return; + } + JobId jobID = MRApps.toJobID(jid); + Job job = appContext.getJob(jobID); + if (job == null) { + html. + p()._("Sorry, ", jid, " not found.")._(); + return; + } + JobReport jobReport = job.getReport(); + String mapPct = percent(jobReport.getMapProgress()); + String reducePct = percent(jobReport.getReduceProgress()); + int mapTasks = job.getTotalMaps(); + int mapTasksComplete = job.getCompletedMaps(); + int reduceTasks = job.getTotalReduces(); + int reducesTasksComplete = job.getCompletedReduces(); + long startTime = jobReport.getStartTime(); + long finishTime = jobReport.getFinishTime(); + countTasksAndAttempts(job); + info("Job Overview"). + _("Job Name:", job.getName()). + _("State:", job.getState()). + _("Uberized:", job.isUber()). + _("Started:", new Date(startTime)). + _("Elapsed:", StringUtils.formatTime( + Times.elapsed(startTime, finishTime))); + html. + _(InfoBlock.class). + div(_INFO_WRAP). + + // Tasks table + table("#job"). + tr(). + th(_TH, "Task Type"). + th(_TH, "Progress"). + th(_TH, "Total"). + th(_TH, "Pending"). + th(_TH, "Running"). + th(_TH, "Complete")._(). + tr(_ODD). + th(). + a(url("tasks", jid, "m"), "Map")._(). + td(). + div(_PROGRESSBAR). + $title(join(mapPct, '%')). // tooltip + div(_PROGRESSBAR_VALUE). + $style(join("width:", mapPct, '%'))._()._()._(). + td(String.valueOf(mapTasks)). + td(String.valueOf(pendingMapTasks)). + td(String.valueOf(runningMapTasks)). + td(String.valueOf(mapTasksComplete))._(). + tr(_EVEN). + th(). + a(url("tasks", jid, "r"), "Reduce")._(). + td(). + div(_PROGRESSBAR). + $title(join(reducePct, '%')). // tooltip + div(_PROGRESSBAR_VALUE). + $style(join("width:", reducePct, '%'))._()._()._(). + td(String.valueOf(reduceTasks)). + td(String.valueOf(pendingReduceTasks)). + td(String.valueOf(runningReduceTasks)). + td(String.valueOf(reducesTasksComplete))._() + ._(). + + // Attempts table + table("#job"). + tr(). + th(_TH, "Attempt Type"). + th(_TH, "New"). + th(_TH, "Running"). + th(_TH, "Failed"). + th(_TH, "Killed"). + th(_TH, "Successful")._(). + tr(_ODD). + th("Maps"). + td().a(url("attempts", jid, "m", + TaskAttemptStateUI.NEW.toString()), + String.valueOf(newMapAttempts))._(). + td().a(url("attempts", jid, "m", + TaskAttemptStateUI.RUNNING.toString()), + String.valueOf(runningMapAttempts))._(). + td().a(url("attempts", jid, "m", + TaskAttemptStateUI.FAILED.toString()), + String.valueOf(failedMapAttempts))._(). + td().a(url("attempts", jid, "m", + TaskAttemptStateUI.KILLED.toString()), + String.valueOf(killedMapAttempts))._(). + td().a(url("attempts", jid, "m", + TaskAttemptStateUI.SUCCESSFUL.toString()), + String.valueOf(successfulMapAttempts))._(). + _(). + tr(_EVEN). + th("Reduces"). + td().a(url("attempts", jid, "r", + TaskAttemptStateUI.NEW.toString()), + String.valueOf(newReduceAttempts))._(). + td().a(url("attempts", jid, "r", + TaskAttemptStateUI.RUNNING.toString()), + String.valueOf(runningReduceAttempts))._(). + td().a(url("attempts", jid, "r", + TaskAttemptStateUI.FAILED.toString()), + String.valueOf(failedReduceAttempts))._(). + td().a(url("attempts", jid, "r", + TaskAttemptStateUI.KILLED.toString()), + String.valueOf(killedReduceAttempts))._(). + td().a(url("attempts", jid, "r", + TaskAttemptStateUI.SUCCESSFUL.toString()), + String.valueOf(successfulReduceAttempts))._(). + _(). + _(). + _(); + } + + /** + * Go through a job and update the member variables with counts for + * information to output in the page. + * @param job the job to get counts for. + */ + private void countTasksAndAttempts(Job job) { + Map tasks = job.getTasks(); + for (Task task : tasks.values()) { + switch (task.getType()) { + case MAP: + // Task counts + switch (task.getState()) { + case RUNNING: + ++runningMapTasks; + break; + case SCHEDULED: + ++pendingMapTasks; + break; + } + break; + case REDUCE: + // Task counts + switch (task.getState()) { + case RUNNING: + ++runningReduceTasks; + break; + case SCHEDULED: + ++pendingReduceTasks; + break; + } + break; + } + + // Attempts counts + Map attempts = task.getAttempts(); + for (TaskAttempt attempt : attempts.values()) { + + int newAttempts = 0, running = 0, successful = 0, failed = 0, killed =0; + + if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) { + ++newAttempts; + } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt + .getState())) { + ++running; + } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt + .getState())) { + ++successful; + } else if (TaskAttemptStateUI.FAILED + .correspondsTo(attempt.getState())) { + ++failed; + } else if (TaskAttemptStateUI.KILLED + .correspondsTo(attempt.getState())) { + ++killed; + } + + switch (task.getType()) { + case MAP: + newMapAttempts += newAttempts; + runningMapAttempts += running; + successfulMapAttempts += successful; + failedMapAttempts += failed; + killedMapAttempts += killed; + break; + case REDUCE: + newReduceAttempts += newAttempts; + runningReduceAttempts += running; + successfulReduceAttempts += successful; + failedReduceAttempts += failed; + killedReduceAttempts += killed; + break; + } + } + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.java new file mode 100644 index 0000000000..4c81a13989 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.java @@ -0,0 +1,53 @@ +/** +* 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.v2.hs.webapp; + +import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.JOB_ID; +import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; + +import org.apache.hadoop.yarn.webapp.SubView; + +/** + * Render a page that describes a specific job. + */ +public class HsJobPage extends HsView { + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ + @Override protected void preHead(Page.HTML<_> html) { + String jobID = $(JOB_ID); + set(TITLE, jobID.isEmpty() ? "Bad request: missing job ID" + : join("MapReduce Job ", $(JOB_ID))); + commonPreHead(html); + //Override the nav config from the commonPreHead + set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}"); + } + + /** + * The content of this page is the JobBlock + * @return HsJobBlock.class + */ + @Override protected Class content() { + return HsJobBlock.class; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java new file mode 100644 index 0000000000..6ef7bf0f37 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java @@ -0,0 +1,109 @@ +/** +* 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.v2.hs.webapp; + +import java.text.SimpleDateFormat; +import java.util.Date; + +import org.apache.hadoop.mapreduce.v2.api.records.JobReport; +import org.apache.hadoop.mapreduce.v2.app.AppContext; +import org.apache.hadoop.mapreduce.v2.app.job.Job; +import org.apache.hadoop.mapreduce.v2.util.MRApps; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY; +import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; + +import com.google.inject.Inject; + +/** + * Render all of the jobs that the history server is aware of. + */ +public class HsJobsBlock extends HtmlBlock { + final AppContext appContext; + static final SimpleDateFormat dateFormat = + new SimpleDateFormat("yyyy.MM.dd HH:mm:ss z"); + + @Inject HsJobsBlock(AppContext appCtx) { + appContext = appCtx; + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block) + */ + @Override protected void render(Block html) { + TBODY> tbody = html. + h2("Retired Jobs"). + table("#jobs"). + thead(). + tr(). + th("Start Time"). + th("Finish Time"). + th(".id", "Job ID"). + th(".name", "Name"). + th("User"). + th(".state", "State"). + th("Maps Total"). + th("Maps Completed"). + th("Reduces Total"). + th("Reduces Completed")._()._(). + tbody(); + LOG.info("Getting list of all Jobs."); + for (Job job : appContext.getAllJobs().values()) { + String jobID = MRApps.toString(job.getID()); + JobReport report = job.getReport(); + String mapsTotal = String.valueOf(job.getTotalMaps()); + String mapsCompleted = String.valueOf(job.getCompletedMaps()); + String reduceTotal = String.valueOf(job.getTotalReduces()); + String reduceCompleted = String.valueOf(job.getCompletedReduces()); + long startTime = report.getStartTime(); + long finishTime = report.getFinishTime(); + tbody. + tr(). + td(dateFormat.format(new Date(startTime))). + td(dateFormat.format(new Date(finishTime))). + td().a(url("job", jobID), jobID)._(). + td(job.getName().toString()). + td(job.getUserName()). + td(job.getState().toString()). + td(mapsTotal). + td(mapsCompleted). + td(reduceTotal). + td(reduceCompleted)._(); + } + tbody._(). + tfoot(). + tr(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("Start Time")._()._(). + th().input("search_init").$type(InputType.text).$name("finish_time").$value("Finish Time")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("Job ID")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("Name")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("User")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("State")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("Maps Total")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("Maps Completed")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("Reduces Total")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("Reduces Completed")._()._(). + _(). + _(). + _(); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java new file mode 100644 index 0000000000..3e32294df1 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java @@ -0,0 +1,66 @@ +/** +* 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.v2.hs.webapp; + +import org.apache.hadoop.mapreduce.v2.app.webapp.App; +import org.apache.hadoop.mapreduce.v2.util.MRApps; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; + +import com.google.inject.Inject; + +/** + * The navigation block for the history server + */ +public class HsNavBlock extends HtmlBlock { + final App app; + + @Inject HsNavBlock(App app) { this.app = app; } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block) + */ + @Override protected void render(Block html) { + DIV nav = html. + div("#nav"). + h3("Application"). + ul(). + li().a(url("about"), "About")._(). + li().a(url("app"), "Jobs")._()._(); + if (app.getJob() != null) { + String jobid = MRApps.toString(app.getJob().getID()); + nav. + h3("Job"). + ul(). + li().a(url("job", jobid), "Overview")._(). + li().a(url("jobcounters", jobid), "Counters")._(). + li().a(url("tasks", jobid, "m"), "Map tasks")._(). + li().a(url("tasks", jobid, "r"), "Reduce tasks")._()._(); + } + nav. + h3("Tools"). + ul(). + li().a("/conf", "Configuration")._(). + li().a("/stacks", "Server stacks")._(). + li().a("/metrics", "Server metrics")._()._()._(). + div("#themeswitcher")._(); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java new file mode 100644 index 0000000000..d9202b3986 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java @@ -0,0 +1,159 @@ +/** +* 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.v2.hs.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.percent; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit; + +import java.util.Collection; + +import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt; +import org.apache.hadoop.mapreduce.v2.app.webapp.App; +import org.apache.hadoop.mapreduce.v2.util.MRApps; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.Times; +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TD; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; + +import com.google.common.base.Joiner; +import com.google.inject.Inject; + +/** + * A Page the shows the status of a given task + */ +public class HsTaskPage extends HsView { + + /** + * A Block of HTML that will render a given task attempt. + */ + static class AttemptsBlock extends HtmlBlock { + final App app; + + @Inject + AttemptsBlock(App ctx) { + app = ctx; + } + + @Override + protected void render(Block html) { + if (!isValidRequest()) { + html. + h2($(TITLE)); + return; + } + TBODY> tbody = html. + table("#attempts"). + thead(). + tr(). + th(".id", "Attempt"). + th(".progress", "Progress"). + th(".state", "State"). + th(".node", "node"). + th(".tsh", "Started"). + th(".tsh", "Finished"). + th(".tsh", "Elapsed"). + th(".note", "Note")._()._(). + tbody(); + for (TaskAttempt ta : getTaskAttempts()) { + String taid = MRApps.toString(ta.getID()); + String progress = percent(ta.getProgress()); + ContainerId containerId = ta.getAssignedContainerID(); + + String nodeHttpAddr = ta.getNodeHttpAddress(); + long startTime = ta.getLaunchTime(); + long finishTime = ta.getFinishTime(); + long elapsed = Times.elapsed(startTime, finishTime); + TD>>> nodeTd = tbody. + tr(). + td(".id", taid). + td(".progress", progress). + td(".state", ta.getState().toString()). + td(). + a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr); + if (containerId != null) { + String containerIdStr = ConverterUtils.toString(containerId); + nodeTd._(" "). + a(".logslink", url("http://", nodeHttpAddr, "yarn", "containerlogs", + containerIdStr), "logs"); + } + nodeTd._(). + td(".ts", Times.format(startTime)). + td(".ts", Times.format(finishTime)). + td(".dt", StringUtils.formatTime(elapsed)). + td(".note", Joiner.on('\n').join(ta.getDiagnostics()))._(); + } + tbody._()._(); + } + + /** + * @return true if this is a valid request else false. + */ + protected boolean isValidRequest() { + return app.getTask() != null; + } + + /** + * @return all of the attempts to render. + */ + protected Collection getTaskAttempts() { + return app.getTask().getAttempts().values(); + } + } + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ + @Override protected void preHead(Page.HTML<_> html) { + commonPreHead(html); + //override the nav config from commonPReHead + set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}"); + //Set up the java script and CSS for the attempts table + set(DATATABLES_ID, "attempts"); + set(initID(DATATABLES, "attempts"), attemptsTableInit()); + setTableStyles(html, "attempts"); + } + + /** + * The content of this page is the attempts block + * @return AttemptsBlock.class + */ + @Override protected Class content() { + return AttemptsBlock.class; + } + + /** + * @return The end of the JS map that is the jquery datatable config for the + * attempts table. + */ + private String attemptsTableInit() { + return tableInit().append("}").toString(); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java new file mode 100644 index 0000000000..c3ddf05b30 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java @@ -0,0 +1,65 @@ +/** +* 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.v2.hs.webapp; + +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit; + +import org.apache.hadoop.mapreduce.v2.app.webapp.TasksBlock; +import org.apache.hadoop.yarn.webapp.SubView; + +/** + * A page showing the tasks for a given application. + */ +public class HsTasksPage extends HsView { + + /* + * (non-Javadoc) + * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ + @Override protected void preHead(Page.HTML<_> html) { + commonPreHead(html); + set(DATATABLES_ID, "tasks"); + set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}"); + set(initID(DATATABLES, "tasks"), tasksTableInit()); + setTableStyles(html, "tasks"); + } + + /** + * The content of this page is the TasksBlock + * @return TasksBlock.class + */ + @Override protected Class content() { + return TasksBlock.class; + } + + /** + * @return the end of the JS map that is the jquery datatable configuration + * for the tasks table. + */ + private String tasksTableInit() { + return tableInit(). + append(",aoColumns:[{sType:'title-numeric'},{sType:'title-numeric',"). + append("bSearchable:false},null,{sType:'title-numeric'},"). + append("{sType:'title-numeric'},{sType:'title-numeric'}]}").toString(); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java index 194571f209..ddb904da3d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java @@ -18,24 +18,42 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp; -import org.apache.hadoop.mapreduce.v2.app.webapp.JobsBlock; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.THEMESWITCHER_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.postInitID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit; + import org.apache.hadoop.yarn.webapp.SubView; import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout; -import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*; +/** + * A view that should be used as the base class for all history server pages. + */ public class HsView extends TwoColumnLayout { + /* + * (non-Javadoc) + * @see org.apache.hadoop.yarn.webapp.view.TwoColumnLayout#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ @Override protected void preHead(Page.HTML<_> html) { commonPreHead(html); set(DATATABLES_ID, "jobs"); set(initID(DATATABLES, "jobs"), jobsTableInit()); + set(postInitID(DATATABLES, "jobs"), jobsPostTableInit()); setTableStyles(html, "jobs"); } + /** + * The prehead that should be common to all subclasses. + * @param html used to render. + */ protected void commonPreHead(Page.HTML<_> html) { - //html.meta_http("refresh", "10"); set(ACCORDION_ID, "nav"); - set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}"); + set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}"); set(THEMESWITCHER_ID, "themeswitcher"); } @@ -43,22 +61,63 @@ protected void commonPreHead(Page.HTML<_> html) { * (non-Javadoc) * @see org.apache.hadoop.yarn.webapp.view.TwoColumnLayout#nav() */ - @Override protected Class nav() { - return org.apache.hadoop.mapreduce.v2.app.webapp.NavBlock.class; + return HsNavBlock.class; } + /* + * (non-Javadoc) + * @see org.apache.hadoop.yarn.webapp.view.TwoColumnLayout#content() + */ @Override protected Class content() { - return JobsBlock.class; + return HsJobsBlock.class; } - + + //TODO We need a way to move all of the javascript/CSS that is for a subview + // into that subview. + /** + * @return The end of a javascript map that is the jquery datatable + * configuration for the jobs table. the Jobs table is assumed to be + * rendered by the class returned from {@link #content()} + */ private String jobsTableInit() { return tableInit(). - append(",aoColumns:[{sType:'title-numeric'},"). - append("null,null,{sType:'title-numeric', bSearchable:false},null,"). - append("null,{sType:'title-numeric',bSearchable:false}, null, null]}"). + append(",aoColumnDefs:["). + append("{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 6 ] }"). + append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 7 ] }"). + append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 8 ] }"). + append(",{'sType':'numeric', 'bSearchable': false, 'aTargets': [ 9 ] }"). + append("]}"). toString(); } + + /** + * @return javascript to add into the jquery block after the table has + * been initialized. This code adds in per field filtering. + */ + private String jobsPostTableInit() { + return "var asInitVals = new Array();\n" + + "$('tfoot input').keyup( function () \n{"+ + " jobsDataTable.fnFilter( this.value, $('tfoot input').index(this) );\n"+ + "} );\n"+ + "$('tfoot input').each( function (i) {\n"+ + " asInitVals[i] = this.value;\n"+ + "} );\n"+ + "$('tfoot input').focus( function () {\n"+ + " if ( this.className == 'search_init' )\n"+ + " {\n"+ + " this.className = '';\n"+ + " this.value = '';\n"+ + " }\n"+ + "} );\n"+ + "$('tfoot input').blur( function (i) {\n"+ + " if ( this.value == '' )\n"+ + " {\n"+ + " this.className = 'search_init';\n"+ + " this.value = asInitVals[$('tfoot input').index(this)];\n"+ + " }\n"+ + "} );\n"; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HSWebApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java similarity index 91% rename from hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HSWebApp.java rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java index 83f5c4e36a..5273a70d22 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HSWebApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java @@ -25,11 +25,11 @@ import org.apache.hadoop.mapreduce.v2.hs.HistoryContext; import org.apache.hadoop.yarn.webapp.WebApp; -public class HSWebApp extends WebApp implements AMParams { +public class HsWebApp extends WebApp implements AMParams { private HistoryContext history; - public HSWebApp(HistoryContext history) { + public HsWebApp(HistoryContext history) { this.history = history; } @@ -44,6 +44,7 @@ public void setup() { route(pajoin("/attempts", JOB_ID, TASK_TYPE, ATTEMPT_STATE), HsController.class, "attempts"); route(pajoin("/task", TASK_ID), HsController.class, "task"); + route("/about", HsController.class, "about"); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java index e96f198748..8d25549517 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java @@ -1,27 +1,126 @@ /** - * 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. - */ +* 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.v2.hs.webapp; -import org.apache.hadoop.yarn.webapp.WebApps; +import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID; +import static org.junit.Assert.assertEquals; + +import java.util.Map; + +import org.apache.hadoop.mapreduce.v2.api.records.JobId; +import org.apache.hadoop.mapreduce.v2.app.AppContext; +import org.apache.hadoop.mapreduce.v2.app.MockJobs; +import org.apache.hadoop.mapreduce.v2.app.job.Job; +import org.apache.hadoop.yarn.Clock; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.util.Apps; +import org.apache.hadoop.yarn.webapp.test.WebAppTests; +import org.junit.Test; + +import com.google.inject.Injector; public class TestHSWebApp { - public static void main(String[] args) { - WebApps.$for("yarn").at(19888).start().joinThread(); + + static class TestAppContext implements AppContext { + final ApplicationAttemptId appAttemptID; + final ApplicationId appID; + final String user = MockJobs.newUserName(); + final Map jobs; + final long startTime = System.currentTimeMillis(); + + TestAppContext(int appid, int numJobs, int numTasks, int numAttempts) { + appID = MockJobs.newAppID(appid); + appAttemptID = MockJobs.newAppAttemptID(appID, 0); + jobs = MockJobs.newJobs(appID, numJobs, numTasks, numAttempts); + } + + TestAppContext() { + this(0, 1, 1, 1); + } + + @Override + public ApplicationAttemptId getApplicationAttemptId() { + return appAttemptID; + } + + @Override + public ApplicationId getApplicationID() { + return appID; + } + + @Override + public CharSequence getUser() { + return user; + } + + @Override + public Job getJob(JobId jobID) { + return jobs.get(jobID); + } + + @Override + public Map getAllJobs() { + return jobs; // OK + } + + @Override + public EventHandler getEventHandler() { + return null; + } + + @Override + public Clock getClock() { + return null; + } + + @Override + public String getApplicationName() { + return "TestApp"; + } + + @Override + public long getStartTime() { + return startTime; + } + } + + @Test public void testAppControllerIndex() { + TestAppContext ctx = new TestAppContext(); + Injector injector = WebAppTests.createMockInjector(AppContext.class, ctx); + HsController controller = injector.getInstance(HsController.class); + controller.index(); + assertEquals(ctx.appID.toString(), controller.get(APP_ID,"")); + } + + @Test public void testJobView() { + WebAppTests.testPage(HsJobPage.class, AppContext.class, new TestAppContext()); + } + + @Test public void testTasksView() { + WebAppTests.testPage(HsTasksPage.class, AppContext.class, + new TestAppContext()); + } + + @Test public void testTaskView() { + WebAppTests.testPage(HsTaskPage.class, AppContext.class, + new TestAppContext()); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java index 048c511b9a..bc73b89256 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java @@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; +import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.yarn.YarnException; @@ -72,8 +72,8 @@ synchronized ClientServiceDelegate getClient(JobID jobId) { private MRClientProtocol instantiateHistoryProxy() throws IOException { - String serviceAddr = conf.get(JHConfig.HS_BIND_ADDRESS, - JHConfig.DEFAULT_HS_BIND_ADDRESS); + String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS, + JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS); LOG.info("Connecting to HistoryServer at: " + serviceAddr); Configuration myConf = new Configuration(conf); //TODO This should ideally be using it's own class (instead of ClientRMSecurityInfo) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java index 4e865a39a4..f0ee71504d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java @@ -19,6 +19,7 @@ package org.apache.hadoop.mapred; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.security.PrivilegedAction; import java.util.HashMap; @@ -149,8 +150,7 @@ private MRClientProtocol getProxy() throws YarnRemoteException { LOG.info("Connecting to " + serviceAddr); instantiateAMProxy(serviceAddr); return realProxy; - } catch (Exception e) { - //possibly + } catch (IOException e) { //possibly the AM has crashed //there may be some time before AM is restarted //keep retrying by getting the address from RM @@ -159,8 +159,13 @@ private MRClientProtocol getProxy() throws YarnRemoteException { try { Thread.sleep(2000); } catch (InterruptedException e1) { + LOG.warn("getProxy() call interruped", e1); + throw new YarnException(e1); } application = rm.getApplicationReport(appId); + } catch (InterruptedException e) { + LOG.warn("getProxy() call interruped", e); + throw new YarnException(e); } } @@ -193,7 +198,7 @@ private MRClientProtocol getProxy() throws YarnRemoteException { //succeeded. if (application.getState() == ApplicationState.SUCCEEDED) { LOG.info("Application state is completed. " + - "Redirecting to job history server " + serviceAddr); + "Redirecting to job history server"); realProxy = historyServerProxy; } return realProxy; @@ -234,8 +239,14 @@ private synchronized Object invoke(String method, Class argClass, LOG.warn("Exception thrown by remote end."); LOG.warn(RPCUtil.toString(yre)); throw yre; + } catch (InvocationTargetException e) { + //TODO Finite # of errors before giving up? + LOG.info("Failed to contact AM/History for job " + jobId + + " Will retry..", e.getTargetException()); + forceRefresh = true; } catch (Exception e) { - LOG.info("Failed to contact AM for job " + jobId + " Will retry.."); + LOG.info("Failed to contact AM/History for job " + jobId + + " Will retry..", e); LOG.debug("Failing to contact application master", e); forceRefresh = true; } @@ -302,10 +313,13 @@ JobStatus getJobStatus(JobID oldJobID) throws YarnRemoteException { return TypeConverter.fromYarn(report, jobFile, ""); } - org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(JobID jobID, TaskType taskType) + org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(JobID oldJobID, TaskType taskType) throws YarnRemoteException, YarnRemoteException { - org.apache.hadoop.mapreduce.v2.api.records.JobId nJobID = TypeConverter.toYarn(jobID); + org.apache.hadoop.mapreduce.v2.api.records.JobId jobId = + TypeConverter.toYarn(oldJobID); GetTaskReportsRequest request = recordFactory.newRecordInstance(GetTaskReportsRequest.class); + request.setJobId(jobId); + request.setTaskType(TypeConverter.toYarn(taskType)); List taskReports = ((GetTaskReportsResponse) invoke("getTaskReports", GetTaskReportsRequest.class, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java index ac606c0330..0e3f37ff2d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java @@ -26,7 +26,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; @@ -59,7 +58,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationMaster; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; @@ -86,8 +84,8 @@ public ResourceMgrDelegate(Configuration conf) { YarnRPC rpc = YarnRPC.create(conf); InetSocketAddress rmAddress = NetUtils.createSocketAddr(conf.get( - YarnConfiguration.APPSMANAGER_ADDRESS, - YarnConfiguration.DEFAULT_APPSMANAGER_BIND_ADDRESS)); + YarnConfiguration.RM_ADDRESS, + YarnConfiguration.DEFAULT_RM_ADDRESS)); LOG.info("Connecting to ResourceManager at " + rmAddress); Configuration appsManagerServerConf = new Configuration(this.conf); appsManagerServerConf.setClass( diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java index 57fcd86df1..fa167a0acf 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java @@ -39,7 +39,7 @@ import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.ProtocolSignature; -import org.apache.hadoop.ipc.VersionedProtocol; +import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus; import org.apache.hadoop.mapreduce.ClusterMetrics; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.JobContext; @@ -55,11 +55,9 @@ import org.apache.hadoop.mapreduce.TaskTrackerInfo; import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.TypeConverter; -import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus; import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier; -import org.apache.hadoop.mapreduce.v2.ClientConstants; import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; import org.apache.hadoop.mapreduce.v2.util.MRApps; @@ -82,7 +80,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -93,10 +90,6 @@ public class YARNRunner implements ClientProtocol { private static final Log LOG = LogFactory.getLog(YARNRunner.class); - public static final String YARN_AM_VMEM_MB = - "yarn.am.mapreduce.resource.mb"; - private static final int DEFAULT_YARN_AM_VMEM_MB = 2048; - private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); private ResourceMgrDelegate resMgrDelegate; private ClientCache clientCache; @@ -108,10 +101,20 @@ public class YARNRunner implements ClientProtocol { * yarn * @param conf the configuration object for the client */ - public YARNRunner(Configuration conf) { - this.conf = new YarnConfiguration(conf); + public YARNRunner(YarnConfiguration conf) { + this(conf, new ResourceMgrDelegate(conf)); + } + + /** + * Similar to {@link #YARNRunner(YarnConfiguration)} but allowing injecting + * {@link ResourceMgrDelegate}. Enables mocking and testing. + * @param conf the configuration object for the client + * @param resMgrDelegate the resourcemanager client handle. + */ + public YARNRunner(YarnConfiguration conf, ResourceMgrDelegate resMgrDelegate) { + this.conf = conf; try { - this.resMgrDelegate = new ResourceMgrDelegate(this.conf); + this.resMgrDelegate = resMgrDelegate; this.clientCache = new ClientCache(this.conf, resMgrDelegate); this.defaultFileContext = FileContext.getFileContext(this.conf); @@ -119,7 +122,7 @@ public YARNRunner(Configuration conf) { throw new RuntimeException("Error in instantiating YarnClient", ufe); } } - + @Override public void cancelDelegationToken(Token arg0) throws IOException, InterruptedException { @@ -245,9 +248,11 @@ public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts) ApplicationReport appMaster = resMgrDelegate .getApplicationReport(applicationId); + String diagnostics = (appMaster == null ? "application report is null" : appMaster.getDiagnostics()); if (appMaster == null || appMaster.getState() == ApplicationState.FAILED || appMaster.getState() == ApplicationState.KILLED) { - throw RPCUtil.getRemoteException("failed to run job"); + throw new IOException("Failed to run job : " + + diagnostics); } return clientCache.getClient(jobId).getJobStatus(jobId); } @@ -265,7 +270,7 @@ private LocalResource createApplicationResource(FileContext fs, Path p) return rsrc; } - private ApplicationSubmissionContext createApplicationSubmissionContext( + public ApplicationSubmissionContext createApplicationSubmissionContext( Configuration jobConf, String jobSubmitDir, Credentials ts) throws IOException { ApplicationSubmissionContext appContext = @@ -273,7 +278,8 @@ private ApplicationSubmissionContext createApplicationSubmissionContext( ApplicationId applicationId = resMgrDelegate.getApplicationId(); appContext.setApplicationId(applicationId); Resource capability = recordFactory.newRecordInstance(Resource.class); - capability.setMemory(conf.getInt(YARN_AM_VMEM_MB, DEFAULT_YARN_AM_VMEM_MB)); + capability.setMemory(conf.getInt(MRJobConfig.MR_AM_VMEM_MB, + MRJobConfig.DEFAULT_MR_AM_VMEM_MB)); LOG.info("AppMaster capability = " + capability); appContext.setMasterCapability(capability); @@ -334,15 +340,14 @@ private ApplicationSubmissionContext createApplicationSubmissionContext( Vector vargs = new Vector(8); vargs.add(javaHome + "/bin/java"); vargs.add("-Dhadoop.root.logger=" - + conf.get(ClientConstants.MR_APPMASTER_LOG_OPTS, - ClientConstants.DEFAULT_MR_APPMASTER_LOG_OPTS) + ",console"); + + conf.get(MRJobConfig.MR_AM_LOG_OPTS, + MRJobConfig.DEFAULT_MR_AM_LOG_OPTS) + ",console"); - vargs.add(conf.get(ClientConstants.MR_APPMASTER_COMMAND_OPTS, - ClientConstants.DEFAULT_MR_APPMASTER_COMMAND_OPTS)); + vargs.add(conf.get(MRJobConfig.MR_AM_COMMAND_OPTS, + MRJobConfig.DEFAULT_MR_AM_COMMAND_OPTS)); // Add { job jar, MR app jar } to classpath. Map environment = new HashMap(); -// appContext.environment = new HashMap(); MRApps.setInitialClasspath(environment); MRApps.addToClassPath(environment, MRConstants.JOB_JAR); MRApps.addToClassPath(environment, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientProtocolProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientProtocolProvider.java index 431b49fb20..74bd303d06 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientProtocolProvider.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientProtocolProvider.java @@ -25,13 +25,14 @@ import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider; +import org.apache.hadoop.yarn.conf.YarnConfiguration; public class YarnClientProtocolProvider extends ClientProtocolProvider { @Override public ClientProtocol create(Configuration conf) throws IOException { if ("yarn".equals(conf.get(MRConfig.FRAMEWORK_NAME))) { - return new YARNRunner(conf); + return new YARNRunner(new YarnConfiguration(conf)); } return null; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapreduce/v2/ClientConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapreduce/v2/ClientConstants.java deleted file mode 100644 index 7cab156dbb..0000000000 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapreduce/v2/ClientConstants.java +++ /dev/null @@ -1,31 +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.mapreduce.v2; - -public interface ClientConstants { - - public static final String MR_APPMASTER_COMMAND_OPTS = - "yarn.appMaster.commandOpts"; - - public static final String DEFAULT_MR_APPMASTER_COMMAND_OPTS = "-Xmx1536m"; - - public static final String MR_APPMASTER_LOG_OPTS = "yarn.appMaster.logOpts"; - - public static final String DEFAULT_MR_APPMASTER_LOG_OPTS = "INFO"; -} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java index 24df9be59b..492ecc8780 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java @@ -21,7 +21,6 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; -import java.util.ArrayList; import java.util.Iterator; import junit.framework.Assert; @@ -64,8 +63,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.JobReport; import org.apache.hadoop.mapreduce.v2.api.records.JobState; -import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig; +import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.yarn.YarnException; @@ -122,8 +120,8 @@ public void testRedirect() throws Exception { Configuration conf = new YarnConfiguration(); conf.set(MRConfig.FRAMEWORK_NAME, "yarn"); - conf.set(YarnConfiguration.APPSMANAGER_ADDRESS, RMADDRESS); - conf.set(JHConfig.HS_BIND_ADDRESS, HSHOSTADDRESS); + conf.set(YarnConfiguration.RM_ADDRESS, RMADDRESS); + conf.set(JHAdminConfig.MR_HISTORY_ADDRESS, HSHOSTADDRESS); RMService rmService = new RMService("test"); rmService.init(conf); rmService.start(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java index efe8c3acb7..5af3e7775f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java @@ -29,11 +29,10 @@ import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.service.Service; @@ -62,19 +61,21 @@ public MiniMRYarnCluster(String testName) { public void init(Configuration conf) { conf.set(MRConfig.FRAMEWORK_NAME, "yarn"); conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name")); - conf.set(MRConstants.APPS_STAGING_DIR_KEY, new File(getTestWorkDir(), + conf.set(MRJobConfig.MR_AM_STAGING_DIR, new File(getTestWorkDir(), "apps_staging_dir/${user.name}/").getAbsolutePath()); conf.set(MRConfig.MASTER_ADDRESS, "test"); // The default is local because of // which shuffle doesn't happen //configure the shuffle service in NM - conf.setStrings(AuxServices.AUX_SERVICES, + conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID }); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID), ShuffleHandler.class, Service.class); + // Non-standard shuffle port conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 8083); - conf.setClass(NMConfig.NM_CONTAINER_EXECUTOR_CLASS, + + conf.setClass(YarnConfiguration.NM_CONTAINER_EXECUTOR, DefaultContainerExecutor.class, ContainerExecutor.class); // TestMRJobs is for testing non-uberized operation only; see TestUberAM diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java index c0747b0d53..ff1dbc52c3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java @@ -68,9 +68,6 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.YarnServerConfig; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -335,11 +332,11 @@ public void testSleepJobWithSecurityOn() throws IOException, mrCluster.getConfig().set( CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - mrCluster.getConfig().set(RMConfig.RM_KEYTAB, "/etc/krb5.keytab"); - mrCluster.getConfig().set(NMConfig.NM_KEYTAB, "/etc/krb5.keytab"); - mrCluster.getConfig().set(YarnConfiguration.RM_SERVER_PRINCIPAL_KEY, + mrCluster.getConfig().set(YarnConfiguration.RM_KEYTAB, "/etc/krb5.keytab"); + mrCluster.getConfig().set(YarnConfiguration.NM_KEYTAB, "/etc/krb5.keytab"); + mrCluster.getConfig().set(YarnConfiguration.RM_PRINCIPAL, "rm/sightbusy-lx@LOCALHOST"); - mrCluster.getConfig().set(YarnServerConfig.NM_SERVER_PRINCIPAL_KEY, + mrCluster.getConfig().set(YarnConfiguration.NM_PRINCIPAL, "nm/sightbusy-lx@LOCALHOST"); UserGroupInformation.setConfiguration(mrCluster.getConfig()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestYARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestYARNRunner.java new file mode 100644 index 0000000000..bc0dfe5fa4 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestYARNRunner.java @@ -0,0 +1,125 @@ +/** + * 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.v2; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +import junit.framework.TestCase; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.ResourceMgrDelegate; +import org.apache.hadoop.mapred.YARNRunner; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.TypeConverter; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationState; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.junit.Before; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * Test if the jobclient shows enough diagnostics + * on a job failure. + * + */ +public class TestYARNRunner extends TestCase { + private static final Log LOG = LogFactory.getLog(TestYARNRunner.class); + private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + + private YARNRunner yarnRunner; + private ResourceMgrDelegate resourceMgrDelegate; + private YarnConfiguration conf; + private ApplicationId appId; + private JobID jobId; + private File testWorkDir = + new File("target", TestYARNRunner.class.getName()); + private ApplicationSubmissionContext submissionContext; + private static final String failString = "Rejected job"; + + @Before + public void setUp() throws Exception { + resourceMgrDelegate = mock(ResourceMgrDelegate.class); + conf = new YarnConfiguration(); + yarnRunner = new YARNRunner(conf, resourceMgrDelegate); + yarnRunner = spy(yarnRunner); + submissionContext = mock(ApplicationSubmissionContext.class); + doAnswer( + new Answer() { + @Override + public ApplicationSubmissionContext answer(InvocationOnMock invocation) + throws Throwable { + return submissionContext; + } + } + ).when(yarnRunner).createApplicationSubmissionContext(any(Configuration.class), + any(String.class), any(Credentials.class)); + + appId = recordFactory.newRecordInstance(ApplicationId.class); + appId.setClusterTimestamp(System.currentTimeMillis()); + appId.setId(1); + jobId = TypeConverter.fromYarn(appId); + if (testWorkDir.exists()) { + FileContext.getLocalFSFileContext().delete(new Path(testWorkDir.toString()), true); + } + testWorkDir.mkdirs(); + } + + + @Test + public void testJobSubmissionFailure() throws Exception { + when(resourceMgrDelegate.submitApplication(any(ApplicationSubmissionContext.class))). + thenReturn(appId); + ApplicationReport report = mock(ApplicationReport.class); + when(report.getApplicationId()).thenReturn(appId); + when(report.getDiagnostics()).thenReturn(failString); + when(report.getState()).thenReturn(ApplicationState.FAILED); + when(resourceMgrDelegate.getApplicationReport(appId)).thenReturn(report); + Credentials credentials = new Credentials(); + File jobxml = new File(testWorkDir, "job.xml"); + OutputStream out = new FileOutputStream(jobxml); + conf.writeXml(out); + out.close(); + try { + yarnRunner.submitJob(jobId, testWorkDir.getAbsolutePath().toString(), credentials); + } catch(IOException io) { + LOG.info("Logging exception:", io); + assertTrue(io.getLocalizedMessage().contains(failString)); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java index d63b8ca924..0ef8d95aa0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java @@ -67,7 +67,7 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeInt; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer; import org.apache.hadoop.yarn.service.AbstractService; @@ -263,10 +263,6 @@ public synchronized ByteBuffer getMeta() { } } - Shuffle createShuffle() { - return new Shuffle(getConfig()); - } - class HttpPipelineFactory implements ChannelPipelineFactory { final Shuffle SHUFFLE; @@ -295,11 +291,13 @@ class Shuffle extends SimpleChannelUpstreamHandler { private final Configuration conf; private final IndexCache indexCache; private final LocalDirAllocator lDirAlloc = - new LocalDirAllocator(NMConfig.NM_LOCAL_DIR); + new LocalDirAllocator(YarnConfiguration.NM_LOCAL_DIRS); + private final int port; public Shuffle(Configuration conf) { this.conf = conf; indexCache = new IndexCache(new JobConf(conf)); + this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); } private List splitMaps(List mapq) { @@ -362,7 +360,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); try { verifyRequest(jobId, ctx, request, response, - new URL("http", "", port, reqUri)); + new URL("http", "", this.port, reqUri)); } catch (IOException e) { LOG.warn("Shuffle failure ", e); sendError(ctx, e.getMessage(), UNAUTHORIZED); diff --git a/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml index 03853ad989..219fd1eb57 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -65,6 +65,11 @@ + + + + + diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerResponsePBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerResponsePBImpl.java index 4fbdf97c7c..a01b11bac1 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerResponsePBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerResponsePBImpl.java @@ -48,20 +48,20 @@ public StartContainerResponsePBImpl(StartContainerResponseProto proto) { viaProto = true; } - public StartContainerResponseProto getProto() { + public synchronized StartContainerResponseProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; return proto; } - private void mergeLocalToBuilder() { + private synchronized void mergeLocalToBuilder() { if (this.serviceResponse != null) { addServiceResponseToProto(); } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) { maybeInitBuilder(); } @@ -70,7 +70,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = StartContainerResponseProto.newBuilder(proto); } @@ -79,17 +79,17 @@ private void maybeInitBuilder() { @Override - public Map getAllServiceResponse() { + public synchronized Map getAllServiceResponse() { initServiceResponse(); return this.serviceResponse; } @Override - public ByteBuffer getServiceResponse(String key) { + public synchronized ByteBuffer getServiceResponse(String key) { initServiceResponse(); return this.serviceResponse.get(key); } - private void initServiceResponse() { + private synchronized void initServiceResponse() { if (this.serviceResponse != null) { return; } @@ -103,14 +103,14 @@ private void initServiceResponse() { } @Override - public void addAllServiceResponse(final Map serviceResponse) { + public synchronized void addAllServiceResponse(final Map serviceResponse) { if (serviceResponse == null) return; initServiceResponse(); this.serviceResponse.putAll(serviceResponse); } - private void addServiceResponseToProto() { + private synchronized void addServiceResponseToProto() { maybeInitBuilder(); builder.clearServiceResponse(); if (serviceResponse == null) @@ -118,24 +118,24 @@ private void addServiceResponseToProto() { Iterable iterable = new Iterable() { @Override - public Iterator iterator() { + public synchronized Iterator iterator() { return new Iterator() { Iterator keyIter = serviceResponse.keySet().iterator(); @Override - public void remove() { + public synchronized void remove() { throw new UnsupportedOperationException(); } @Override - public StringBytesMapProto next() { + public synchronized StringBytesMapProto next() { String key = keyIter.next(); return StringBytesMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(serviceResponse.get(key))).build(); } @Override - public boolean hasNext() { + public synchronized boolean hasNext() { return keyIter.hasNext(); } }; @@ -144,17 +144,17 @@ public boolean hasNext() { builder.addAllServiceResponse(iterable); } @Override - public void setServiceResponse(String key, ByteBuffer val) { + public synchronized void setServiceResponse(String key, ByteBuffer val) { initServiceResponse(); this.serviceResponse.put(key, val); } @Override - public void removeServiceResponse(String key) { + public synchronized void removeServiceResponse(String key) { initServiceResponse(); this.serviceResponse.remove(key); } @Override - public void clearServiceResponse() { + public synchronized void clearServiceResponse() { initServiceResponse(); this.serviceResponse.clear(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMResponse.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMResponse.java index 54b0055133..e009d2eb17 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMResponse.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMResponse.java @@ -86,32 +86,17 @@ public interface AMResponse { */ @Public @Stable - public List getNewContainerList(); + public List getAllocatedContainers(); - @Private - @Unstable - public Container getNewContainer(int index); + /** + * Set the list of newly allocated Container by the + * ResourceManager. + * @param containers list of newly allocated Container + */ + @Public + @Stable + public void setAllocatedContainers(List containers); - @Private - @Unstable - public int getNewContainerCount(); - - @Private - @Unstable - public void addAllNewContainers(List containers); - - @Private - @Unstable - public void addNewContainer(Container container); - - @Private - @Unstable - public void removeNewContainer(int index); - - @Private - @Unstable - public void clearNewContainers(); - /** * Get the available headroom for resources in the cluster for the * application. @@ -127,35 +112,14 @@ public interface AMResponse { public void setAvailableResources(Resource limit); /** - * Get the list of completed containers. - * @return the list of completed containers + * Get the list of completed containers' statuses. + * @return the list of completed containers' statuses */ @Public @Stable - public List getFinishedContainerList(); + public List getCompletedContainersStatuses(); @Private @Unstable - public Container getFinishedContainer(int index); - - @Private - @Unstable - public int getFinishedContainerCount(); - - - @Private - @Unstable - public void addAllFinishedContainers(List containers); - - @Private - @Unstable - public void addFinishedContainer(Container container); - - @Private - @Unstable - public void removeFinishedContainer(int index); - - @Private - @Unstable - public void clearFinishedContainers(); + public void setCompletedContainersStatuses(List containers); } \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java index cc958c357b..854afa1a71 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java @@ -18,11 +18,105 @@ package org.apache.hadoop.yarn.api.records; -public interface ApplicationAttemptId extends Comparable{ +import java.text.NumberFormat; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + *

ApplicationAttemptId denotes the particular attempt + * of an ApplicationMaster for a given {@link ApplicationId}.

+ * + *

Multiple attempts might be needed to run an application to completion due + * to temporal failures of the ApplicationMaster such as hardware + * failures, connectivity issues etc. on the node on which it was scheduled.

+ */ +@Public +@Stable +public abstract class ApplicationAttemptId implements + Comparable { + + /** + * Get the ApplicationId of the ApplicationAttempId. + * @return ApplicationId of the ApplicationAttempId + */ + @Public + @Stable public abstract ApplicationId getApplicationId(); + + @Private + @Unstable + public abstract void setApplicationId(ApplicationId appID); + + /** + * Get the attempt id of the Application. + * @return attempt id of the Application + */ public abstract int getAttemptId(); - public abstract void setApplicationId(ApplicationId appID); + @Private + @Unstable public abstract void setAttemptId(int attemptId); + + + protected static final NumberFormat idFormat = NumberFormat.getInstance(); + static { + idFormat.setGroupingUsed(false); + idFormat.setMinimumIntegerDigits(4); + } + + protected static final NumberFormat counterFormat = NumberFormat + .getInstance(); + static { + counterFormat.setGroupingUsed(false); + counterFormat.setMinimumIntegerDigits(6); + } + + @Override + public int hashCode() { + // Generated by eclipse. + final int prime = 31; + int result = 1; + ApplicationId appId = getApplicationId(); + result = prime * result + ((appId == null) ? 0 : appId.hashCode()); + result = prime * result + getAttemptId(); + return result; + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + ApplicationAttemptId otherAttemptId = (ApplicationAttemptId) other; + if (this.getApplicationId().equals(otherAttemptId.getApplicationId())) { + return this.getAttemptId() == otherAttemptId.getAttemptId(); + } + } + return false; + } + + @Override + public int compareTo(ApplicationAttemptId other) { + int compareAppIds = this.getApplicationId().compareTo( + other.getApplicationId()); + if (compareAppIds == 0) { + return this.getAttemptId() - other.getAttemptId(); + } else { + return compareAppIds; + } + } + + @Override + public String toString() { + String id = + (this.getApplicationId() != null) ? this.getApplicationId() + .getClusterTimestamp() + + "_" + + idFormat.format(this.getApplicationId().getId()) : "none"; + return "appattempt_" + id + "_" + counterFormat.format(getAttemptId()); + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java index 1f6bfc8ce1..f98bf99c87 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java @@ -18,13 +18,87 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; - -public interface ApplicationId extends Comparable { +/** + *

ApplicationId represents the globally unique + * identifier for an application.

+ * + *

The globally unique nature of the identifier is achieved by using the + * cluster timestamp i.e. start-time of the + * ResourceManager along with a monotonically increasing counter + * for the application.

+ */ +@Public +@Stable +public abstract class ApplicationId implements Comparable { + + /** + * Get the short integer identifier of the ApplicationId + * which is unique for all applications started by a particular instance + * of the ResourceManager. + * @return short integer identifier of the ApplicationId + */ + @Public + @Stable public abstract int getId(); + + @Private + @Unstable + public abstract void setId(int id); + + /** + * Get the start time of the ResourceManager which is + * used to generate globally unique ApplicationId. + * @return start time of the ResourceManager + */ public abstract long getClusterTimestamp(); - public abstract void setId(int id); + @Private + @Unstable public abstract void setClusterTimestamp(long clusterTimestamp); + + @Override + public int compareTo(ApplicationId other) { + if (this.getClusterTimestamp() - other.getClusterTimestamp() == 0) { + return this.getId() - other.getId(); + } else { + return this.getClusterTimestamp() > other.getClusterTimestamp() ? 1 : + this.getClusterTimestamp() < other.getClusterTimestamp() ? -1 : 0; + } + } + + @Override + public String toString() { + return "application_" + this.getClusterTimestamp() + "_" + this.getId(); + } + + @Override + public int hashCode() { + // Generated by eclipse. + final int prime = 31; + int result = 1; + long clusterTimestamp = getClusterTimestamp(); + result = prime * result + + (int) (clusterTimestamp ^ (clusterTimestamp >>> 32)); + result = prime * result + getId(); + return result; + } + + @Override + public boolean equals(Object other) { + if (other == null) return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + ApplicationId otherAppId = (ApplicationId)other; + if (this.getClusterTimestamp() == otherAppId.getClusterTimestamp() && + this.getId() == otherAppId.getId()) { + return true; + } + } + return false; + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationMaster.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationMaster.java index 725820ccc2..3137009f48 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationMaster.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationMaster.java @@ -18,26 +18,43 @@ package org.apache.hadoop.yarn.api.records; -//TODO: Split separate object for register, deregister and in-RM use. +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * For internal use only... + */ +@Private +@Unstable public interface ApplicationMaster { ApplicationId getApplicationId(); - String getHost(); - int getRpcPort(); - String getTrackingUrl(); - ApplicationStatus getStatus(); - ApplicationState getState(); - String getClientToken(); - int getAMFailCount(); - int getContainerCount(); - String getDiagnostics(); void setApplicationId(ApplicationId appId); + + String getHost(); void setHost(String host); + + int getRpcPort(); void setRpcPort(int rpcPort); + + String getTrackingUrl(); void setTrackingUrl(String url); + + ApplicationStatus getStatus(); void setStatus(ApplicationStatus status); + + ApplicationState getState(); void setState(ApplicationState state); + + String getClientToken(); void setClientToken(String clientToken); + + int getAMFailCount(); void setAMFailCount(int amFailCount); + + int getContainerCount(); void setContainerCount(int containerCount); + + String getDiagnostics(); void setDiagnostics(String diagnostics); } + diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationState.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationState.java index 9faafd93cb..6fcdea5cac 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationState.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationState.java @@ -18,6 +18,30 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; + +/** + * Ennumeration of various states of an Application. + */ +@Public +@Stable public enum ApplicationState { - NEW, SUBMITTED, RUNNING, SUCCEEDED, FAILED, KILLED + /** Application which was just created. */ + NEW, + + /** Application which has been submitted. */ + SUBMITTED, + + /** Application which is currently running. */ + RUNNING, + + /** Application which completed successfully. */ + SUCCEEDED, + + /** Application which failed. */ + FAILED, + + /** Application which was terminated by a user or admin. */ + KILLED } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationStatus.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationStatus.java index 6748c5369a..a71a1652fe 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationStatus.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationStatus.java @@ -18,12 +18,21 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * For internal use only... + */ +@Private +@Unstable public interface ApplicationStatus { ApplicationAttemptId getApplicationAttemptId(); - int getResponseId(); - float getProgress(); - void setApplicationAttemptId(ApplicationAttemptId applicationAttemptId); + + int getResponseId(); void setResponseId(int id); + + float getProgress(); void setProgress(float progress); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java index 523400b0ea..97c84e4d10 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java @@ -18,21 +18,133 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ContainerManager; +/** + *

Container represents an allocated resource in the cluster. + *

+ * + *

The ResourceManager is the sole authority to allocate any + * Container to applications. The allocated Container + * is always on a single node and has a unique {@link ContainerId}. It has + * a specific amount of {@link Resource} allocated.

+ * + *

It includes details such as: + *

    + *
  • {@link ContainerId} for the container, which is globally unique.
  • + *
  • + * {@link NodeId} of the node on which identifies the node on which it + * is allocated. + *
  • + *
  • HTTP uri of the node.
  • + *
  • {@link Resource} allocated to the container.
  • + *
  • {@link ContainerState} of the container.
  • + *
  • + * {@link ContainerToken} of the container, used to securely verify + * authenticity of the allocation. + *
  • + *
  • {@link ContainerStatus} of the container.
  • + *
+ *

+ * + *

Typically, an ApplicationMaster receives the + * Container from the ResourceManager during + * resource-negotiation and then talks to the NodManager to + * start/stop containers.

+ * + * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) + * @see ContainerManager#stopContainer(org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest) + */ +@Public +@Stable public interface Container extends Comparable { + /** + * Get the globally unique identifier for the container. + * @return globally unique identifier for the container + */ + @Public + @Stable ContainerId getId(); + + @Private + @Unstable + void setId(ContainerId id); + + /** + * Get the identifier of the node on which the container is allocated. + * @return identifier of the node on which the container is allocated + */ + @Public + @Stable NodeId getNodeId(); + + @Private + @Unstable + void setNodeId(NodeId nodeId); + + /** + * Get the http uri of the node on which the container is allocated. + * @return http uri of the node on which the container is allocated + */ + @Public + @Stable String getNodeHttpAddress(); + + @Private + @Unstable + void setNodeHttpAddress(String nodeHttpAddress); + + /** + * Get the Resource allocated to the container. + * @return Resource allocated to the container + */ + @Public + @Stable Resource getResource(); + + @Private + @Unstable + void setResource(Resource resource); + + /** + * Get the current ContainerState of the container. + * @return current ContainerState of the container + */ + @Public + @Stable ContainerState getState(); + + @Private + @Unstable + void setState(ContainerState state); + + /** + * Get the ContainerToken for the container. + * @return ContainerToken for the container + */ + @Public + @Stable ContainerToken getContainerToken(); + + @Private + @Unstable + void setContainerToken(ContainerToken containerToken); + + /** + * Get the ContainerStatus of the container. + * @return ContainerStatus of the container + */ + @Public + @Stable ContainerStatus getContainerStatus(); - void setId(ContainerId id); - void setNodeId(NodeId nodeId); - void setNodeHttpAddress(String nodeHttpAddress); - void setResource(Resource resource); - void setState(ContainerState state); - void setContainerToken(ContainerToken containerToken); + @Private + @Unstable void setContainerStatus(ContainerStatus containerStatus); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java index 57f4a21834..7e43f164dd 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java @@ -18,13 +18,131 @@ package org.apache.hadoop.yarn.api.records; -public interface ContainerId extends Comparable{ - public abstract ApplicationAttemptId getAppAttemptId(); - public abstract ApplicationId getAppId(); +import java.text.NumberFormat; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + *

ContainerId represents a globally unique identifier + * for a {@link Container} in the cluster.

+ */ +@Public +@Stable +public abstract class ContainerId implements Comparable{ + /** + * Get the ApplicationAttemptId of the application to which + * the Container was assigned. + * @return ApplicationAttemptId of the application to which + * the Container was assigned + */ + @Public + @Stable + public abstract ApplicationAttemptId getApplicationAttemptId(); + + @Private + @Unstable + public abstract void setApplicationAttemptId(ApplicationAttemptId atId); + + /** + * Get the identifier of the ContainerId. + * @return identifier of the ContainerId + */ + @Public + @Stable public abstract int getId(); - - public abstract void setAppAttemptId(ApplicationAttemptId atId); - public abstract void setAppId(ApplicationId appID); + + @Private + @Unstable public abstract void setId(int id); + + // TODO: Why thread local? + // ^ NumberFormat instances are not threadsafe + private static final ThreadLocal appIdFormat = + new ThreadLocal() { + @Override + public NumberFormat initialValue() { + NumberFormat fmt = NumberFormat.getInstance(); + fmt.setGroupingUsed(false); + fmt.setMinimumIntegerDigits(4); + return fmt; + } + }; + + // TODO: fail the app submission if attempts are more than 10 or something + private static final ThreadLocal appAttemptIdFormat = + new ThreadLocal() { + @Override + public NumberFormat initialValue() { + NumberFormat fmt = NumberFormat.getInstance(); + fmt.setGroupingUsed(false); + fmt.setMinimumIntegerDigits(2); + return fmt; + } + }; + // TODO: Why thread local? + // ^ NumberFormat instances are not threadsafe + private static final ThreadLocal containerIdFormat = + new ThreadLocal() { + @Override + public NumberFormat initialValue() { + NumberFormat fmt = NumberFormat.getInstance(); + fmt.setGroupingUsed(false); + fmt.setMinimumIntegerDigits(6); + return fmt; + } + }; + + @Override + public int hashCode() { + // Generated by eclipse. + final int prime = 31; + int result = 1; + result = prime * result + getId(); + result = prime * result + + ((getApplicationAttemptId() == null) ? 0 : getApplicationAttemptId().hashCode()); + return result; + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + ContainerId otherCId = (ContainerId)other; + if (this.getApplicationAttemptId().equals( + otherCId.getApplicationAttemptId())) { + return this.getId() == otherCId.getId(); + } + } + return false; + } + + @Override + public int compareTo(ContainerId other) { + if (this.getApplicationAttemptId().compareTo( + other.getApplicationAttemptId()) == 0) { + return this.getId() - other.getId(); + } else { + return this.getApplicationAttemptId().compareTo( + other.getApplicationAttemptId()); + } + + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + ApplicationId appId = getApplicationAttemptId().getApplicationId(); + sb.append("container_").append(appId.getClusterTimestamp()).append("_"); + sb.append(appIdFormat.get().format(appId.getId())).append("_"); + sb.append(appAttemptIdFormat.get().format(getApplicationAttemptId(). + getAttemptId())).append("_"); + sb.append(containerIdFormat.get().format(getId())); + return sb.toString(); + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java index cf5c9d0d4e..0339df9af1 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java @@ -22,10 +22,8 @@ import java.util.List; import java.util.Map; -import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ContainerManager; /** @@ -121,100 +119,52 @@ public interface ContainerLaunchContext { void setContainerTokens(ByteBuffer containerToken); /** - * Get all LocalResource required by the container. + * Get LocalResource required by the container. * @return all LocalResource required by the container */ @Public @Stable - Map getAllLocalResources(); + Map getLocalResources(); - @Private - @Unstable - LocalResource getLocalResource(String key); - /** - * Add all LocalResource required by the container. + * Set LocalResource required by the container. * @param localResources LocalResource required by the container */ @Public @Stable - void addAllLocalResources(Map localResources); - - @Private - @Unstable - void setLocalResource(String key, LocalResource value); - - @Private - @Unstable - void removeLocalResource(String key); - - @Private - @Unstable - void clearLocalResources(); + void setLocalResources(Map localResources); /** - * Get application-specific binary service data. - * @return application-specific binary service data + * Get application-specific binary service data. + * @return application-specific binary service data */ @Public @Stable - Map getAllServiceData(); - - @Private - @Unstable - ByteBuffer getServiceData(String key); - - /** - * Add add application-specific binary service data. - * @param serviceData application-specific binary service data - */ - @Public - @Stable - void addAllServiceData(Map serviceData); - - @Private - @Unstable - void setServiceData(String key, ByteBuffer value); - - @Private - @Unstable - void removeServiceData(String key); - - @Private - @Unstable - void clearServiceData(); - - /** - * Get environment variables for the launched container. - * @return environment variables for the launched container - */ - @Public - @Stable - Map getAllEnv(); - - @Private - @Unstable - String getEnv(String key); + Map getServiceData(); /** - * Add environment variables for the launched container. - * @param env environment variables for the launched container + * Set application-specific binary service data. + * @param serviceData application-specific binary service data */ @Public @Stable - void addAllEnv(Map env); + void setServiceData(Map serviceData); - @Private - @Unstable - void setEnv(String key, String value); - - @Private - @Unstable - void removeEnv(String key); - - @Private - @Unstable - void clearEnv(); + /** + * Get environment variables for the container. + * @return environment variables for the container + */ + @Public + @Stable + Map getEnv(); + + /** + * Add environment variables for the container. + * @param environment environment variables for the container + */ + @Public + @Stable + void setEnv(Map environment); /** * Get the list of commands for launching the container. @@ -222,15 +172,7 @@ public interface ContainerLaunchContext { */ @Public @Stable - List getCommandList(); - - @Private - @Unstable - String getCommand(int index); - - @Private - @Unstable - int getCommandCount(); + List getCommands(); /** * Add the list of commands for launching the container. @@ -238,17 +180,6 @@ public interface ContainerLaunchContext { */ @Public @Stable - void addAllCommands(List commands); + void setCommands(List commands); - @Private - @Unstable - void addCommand(String command); - - @Private - @Unstable - void removeCommand(int index); - - @Private - @Unstable - void clearCommands(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java index 8a149d01fa..b7ff5371fc 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java @@ -18,6 +18,16 @@ package org.apache.hadoop.yarn.api.records; +/** + *

State of a Container.

+ */ public enum ContainerState { - NEW, RUNNING, COMPLETE + /** New container */ + NEW, + + /** Running container */ + RUNNING, + + /** Completed container */ + COMPLETE } \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java index c52a47c548..e2dfc82c4b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java @@ -18,14 +18,81 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + *

ContainerStatus represents the current status of a + * Container.

+ * + *

It provides details such as: + *

    + *
  • ContainerId of the container.
  • + *
  • ContainerState of the container.
  • + *
  • Exit status of a completed container.
  • + *
  • Diagnostic message for a failed container.
  • + *
+ *

+ */ +@Public +@Stable public interface ContainerStatus { + /** + * Get the ContainerId of the container. + * @return ContainerId of the container + */ + @Public + @Stable ContainerId getContainerId(); + + @Private + @Unstable + void setContainerId(ContainerId containerId); + + /** + * Get the ContainerState of the container. + * @return ContainerState of the container + */ + @Public + @Stable ContainerState getState(); - String getExitStatus(); + + @Private + @Unstable + void setState(ContainerState state); + + /** + *

Get the exit status for the container.

+ * + *

Note: This is valid only for completed containers i.e. containers + * with state {@link ContainerState#COMPLETE}. + * Otherwise, it returns an invalid exit code equal to {@literal -1000};

+ * + *

Container killed by the framework, either due to being released by + * the application or being 'lost' due to node failures etc. have a special + * exit code of {@literal -100}.

+ * + * @return exit status for the container + */ + @Public + @Stable + int getExitStatus(); + + @Private + @Unstable + void setExitStatus(int exitStatus); + + /** + * Get diagnostic messages for failed containers. + * @return diagnostic messages for failed containers + */ + @Public + @Stable String getDiagnostics(); - void setContainerId(ContainerId containerId); - void setState(ContainerState state); - void setExitStatus(String exitStatus); + @Private + @Unstable void setDiagnostics(String diagnostics); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerToken.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerToken.java index 52290dbd47..da34f71f92 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerToken.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerToken.java @@ -20,15 +20,76 @@ import java.nio.ByteBuffer; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.ContainerManager; + +/** + *

ContainerToken is the security token used by the framework + * to verify authenticity of any Container.

+ * + *

The ResourceManager, on container allocation provides a + * secure token which is verified by the NodeManager on + * container launch.

+ * + *

Applications do not need to care about ContainerToken, they + * are transparently handled by the framework - the allocated + * Container includes the ContainerToken.

+ * + * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest) + */ +@Public +@Stable public interface ContainerToken { + /** + * Get the token identifier. + * @return token identifier + */ + @Public + @Stable public abstract ByteBuffer getIdentifier(); - public abstract ByteBuffer getPassword(); - public abstract String getKind(); - public abstract String getService(); + @Private + @Stable public abstract void setIdentifier(ByteBuffer identifier); + + /** + * Get the token password + * @return token password + */ + @Public + @Stable + public abstract ByteBuffer getPassword(); + + @Private + @Stable public abstract void setPassword(ByteBuffer password); + + /** + * Get the token kind. + * @return token kind + */ + @Public + @Stable + public abstract String getKind(); + + @Private + @Stable public abstract void setKind(String kind); + + /** + * Get the service to which the token is allocated. + * @return service to which the token is allocated + */ + @Public + @Stable + public abstract String getService(); + + @Private + @Stable public abstract void setService(String service); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java index a7cec29e1b..b06af127ac 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java @@ -18,11 +18,43 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + *

NodeId is the unique identifier for a node.

+ * + *

It includes the hostname and port to uniquely + * identify the node. Thus, it is unique across restarts of any + * NodeManager.

+ */ +@Public +@Stable public interface NodeId extends Comparable { + /** + * Get the hostname of the node. + * @return hostname of the node + */ + @Public + @Stable String getHost(); + + @Private + @Unstable void setHost(String host); + /** + * Get the port for communicating with the node. + * @return port for communicating with the node + */ + @Public + @Stable int getPort(); + + @Private + @Unstable void setPort(int port); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ProtoBase.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ProtoBase.java index c0c438b564..c40e233869 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ProtoBase.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ProtoBase.java @@ -20,11 +20,15 @@ import java.nio.ByteBuffer; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.util.ProtoUtils; import com.google.protobuf.ByteString; import com.google.protobuf.Message; +@Private +@Unstable public abstract class ProtoBase { public abstract T getProto(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java index eb5b681ede..497c65ebe2 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java @@ -18,10 +18,40 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.yarn.api.AMRMProtocol; + +/** + *

Resource models a set of computer resources in the + * cluster.

+ * + *

Currrently it only models memory.

+ * + *

Typically, applications request Resource of suitable + * capability to run their component tasks.

+ * + * @see ResourceRequest + * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + */ +@Public +@Stable public interface Resource extends Comparable { + + /** + * Get memory of the resource. + * @return memory of the resource + */ + @Public + @Stable public abstract int getMemory(); + /** + * Set memory of the resource. + * @param memory memory of the resource + */ + @Public + @Stable public abstract void setMemory(int memory); - } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java index 79e29b91e8..4072da1b61 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java @@ -18,16 +18,107 @@ package org.apache.hadoop.yarn.api.records; -public interface ResourceRequest extends Comparable{ +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.yarn.api.AMRMProtocol; + +/** + *

ResourceRequest represents the request made by an + * application to the ResourceManager to obtain various + * Container allocations.

+ * + *

It includes: + *

    + *
  • {@link Priority} of the request.
  • + *
  • + * The name of the machine or rack on which the allocation is + * desired. A special value of * signifies that + * any host/rack is acceptable to the application. + *
  • + *
  • {@link Resource} required for each request.
  • + *
  • + * Number of containers of such specifications which are required + * by the application. + *
  • + *
+ *

+ * + * @see Resource + * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + */ +@Public +@Stable +public interface ResourceRequest extends Comparable { + /** + * Get the Priority of the request. + * @return Priority of the request + */ + @Public + @Stable public abstract Priority getPriority(); + + /** + * Set the Priority of the request + * @param priority Priority of the request + */ + @Public + @Stable + public abstract void setPriority(Priority priority); + + /** + * Get the host/rack on which the allocation is desired. + * + * A special value of * signifies that any host/rack is + * acceptable. + * + * @return host/rack on which the allocation is desired + */ + @Public + @Stable public abstract String getHostName(); + + /** + * Set host/rack on which the allocation is desired. + * + * A special value of * signifies that any host/rack is + * acceptable. + * + * @param hostName host/rack on which the allocation is desired + */ + @Public + @Stable + public abstract void setHostName(String hostName); + + /** + * Get the Resource capability of the request. + * @return Resource capability of the request + */ + @Public + @Stable public abstract Resource getCapability(); + + /** + * Set the Resource capability of the request + * @param capability Resource capability of the request + */ + @Public + @Stable + public abstract void setCapability(Resource capability); + + /** + * Get the number of containers required with the given specifications. + * @return number of containers required with the given specifications + */ + @Public + @Stable public abstract int getNumContainers(); - public abstract void setPriority(Priority priority); - public abstract void setHostName(String hostName); - public abstract void setCapability(Resource capability); + /** + * Set the number of containers required with the given specifications + * @param numContainers number of containers required with the given + * specifications + */ + @Public + @Stable public abstract void setNumContainers(int numContainers); - - } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java index bdf4a7d285..973302fff6 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java @@ -18,14 +18,77 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; + +/** + *

URL represents a serializable {@link java.net.URL}.

+ */ +@Public +@Evolving public interface URL { + + /** + * Get the scheme of the URL. + * @return scheme of the URL + */ + @Public + @Evolving public abstract String getScheme(); + + /** + * Set the scheme of the URL + * @param scheme scheme of the URL + */ + @Public + @Evolving + public abstract void setScheme(String scheme); + + /** + * Get the host of the URL. + * @return host of the URL + */ + @Public + @Evolving public abstract String getHost(); + + /** + * Set the host of the URL. + * @param host host of the URL + */ + @Public + @Evolving + public abstract void setHost(String host); + + /** + * Get the port of the URL. + * @return port of the URL + */ + @Public + @Evolving public abstract int getPort(); + + /** + * Set the port of the URL + * @param port port of the URL + */ + @Public + @Evolving + public abstract void setPort(int port); + + /** + * Get the file of the URL. + * @return file of the URL + */ + @Public + @Evolving public abstract String getFile(); - public abstract void setScheme(String scheme); - public abstract void setHost(String host); - public abstract void setPort(int port); + /** + * Set the file of the URL. + * @param file file of the URL + */ + @Public + @Evolving public abstract void setFile(String file); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnClusterMetrics.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnClusterMetrics.java index 963cf068c4..8ecf60c0ed 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnClusterMetrics.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnClusterMetrics.java @@ -18,10 +18,30 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + *

YarnClusterMetrics represents cluster metrics.

+ * + *

Currently only number of NodeManagers is provided.

+ */ +@Public +@Stable public interface YarnClusterMetrics { + /** + * Get the number of NodeManagers in the cluster. + * @return number of NodeManagers in the cluster + */ + @Public + @Stable public abstract int getNumNodeManagers(); - + + @Private + @Unstable public abstract void setNumNodeManagers(int numNodeManagers); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMResponsePBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMResponsePBImpl.java index 7ae3e86569..11fde9470b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMResponsePBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMResponsePBImpl.java @@ -25,11 +25,13 @@ import org.apache.hadoop.yarn.api.records.AMResponse; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ProtoBase; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.proto.YarnProtos.AMResponseProto; import org.apache.hadoop.yarn.proto.YarnProtos.AMResponseProtoOrBuilder; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; @@ -41,8 +43,8 @@ public class AMResponsePBImpl extends ProtoBase implements AMRe Resource limit; - private List newContainersList = null; - private List finishedContainersList = null; + private List allocatedContainers = null; + private List completedContainersStatuses = null; // private boolean hasLocalContainerList = false; @@ -63,15 +65,17 @@ public synchronized AMResponseProto getProto() { } private synchronized void mergeLocalToBuilder() { - if (this.newContainersList != null) { - builder.clearNewContainers(); - Iterable iterable = getProtoIterable(this.newContainersList); - builder.addAllNewContainers(iterable); + if (this.allocatedContainers != null) { + builder.clearAllocatedContainers(); + Iterable iterable = + getProtoIterable(this.allocatedContainers); + builder.addAllAllocatedContainers(iterable); } - if (this.finishedContainersList != null) { - builder.clearFinishedContainers(); - Iterable iterable = getProtoIterable(this.finishedContainersList); - builder.addAllFinishedContainers(iterable); + if (this.completedContainersStatuses != null) { + builder.clearCompletedContainerStatuses(); + Iterable iterable = + getContainerStatusProtoIterable(this.completedContainersStatuses); + builder.addAllCompletedContainerStatuses(iterable); } if (this.limit != null) { builder.setLimit(convertToProtoFormat(this.limit)); @@ -139,42 +143,31 @@ public synchronized void setAvailableResources(Resource limit) { } @Override - public synchronized List getNewContainerList() { + public synchronized List getAllocatedContainers() { initLocalNewContainerList(); - return this.newContainersList; - } - - @Override - public synchronized Container getNewContainer(int index) { - initLocalNewContainerList(); - return this.newContainersList.get(index); - } - @Override - public synchronized int getNewContainerCount() { - initLocalNewContainerList(); - return this.newContainersList.size(); + return this.allocatedContainers; } //Once this is called. containerList will never be null - untill a getProto is called. private synchronized void initLocalNewContainerList() { - if (this.newContainersList != null) { + if (this.allocatedContainers != null) { return; } AMResponseProtoOrBuilder p = viaProto ? proto : builder; - List list = p.getNewContainersList(); - newContainersList = new ArrayList(); + List list = p.getAllocatedContainersList(); + allocatedContainers = new ArrayList(); for (ContainerProto c : list) { - newContainersList.add(convertFromProtoFormat(c)); + allocatedContainers.add(convertFromProtoFormat(c)); } } @Override - public synchronized void addAllNewContainers(final List containers) { + public synchronized void setAllocatedContainers(final List containers) { if (containers == null) return; initLocalNewContainerList(); - newContainersList.addAll(containers); + allocatedContainers.addAll(containers); } private synchronized Iterable getProtoIterable( @@ -207,86 +200,71 @@ public synchronized void remove() { } }; } - - @Override - public synchronized void addNewContainer(Container containers) { - initLocalNewContainerList(); - if (containers == null) - return; - this.newContainersList.add(containers); - } - - @Override - public synchronized void removeNewContainer(int index) { - initLocalNewContainerList(); - this.newContainersList.remove(index); - } - @Override - public synchronized void clearNewContainers() { - initLocalNewContainerList(); - this.newContainersList.clear(); + + private synchronized Iterable + getContainerStatusProtoIterable( + final List newContainersList) { + maybeInitBuilder(); + return new Iterable() { + @Override + public synchronized Iterator iterator() { + return new Iterator() { + + Iterator iter = newContainersList.iterator(); + + @Override + public synchronized boolean hasNext() { + return iter.hasNext(); + } + + @Override + public synchronized ContainerStatusProto next() { + return convertToProtoFormat(iter.next()); + } + + @Override + public synchronized void remove() { + throw new UnsupportedOperationException(); + + } + }; + + } + }; } //// Finished containers @Override - public synchronized List getFinishedContainerList() { + public synchronized List getCompletedContainersStatuses() { initLocalFinishedContainerList(); - return this.finishedContainersList; - } - - @Override - public synchronized Container getFinishedContainer(int index) { - initLocalFinishedContainerList(); - return this.finishedContainersList.get(index); - } - @Override - public synchronized int getFinishedContainerCount() { - initLocalFinishedContainerList(); - return this.finishedContainersList.size(); + return this.completedContainersStatuses; } //Once this is called. containerList will never be null - untill a getProto is called. private synchronized void initLocalFinishedContainerList() { - if (this.finishedContainersList != null) { + if (this.completedContainersStatuses != null) { return; } AMResponseProtoOrBuilder p = viaProto ? proto : builder; - List list = p.getFinishedContainersList(); - finishedContainersList = new ArrayList(); + List list = p.getCompletedContainerStatusesList(); + completedContainersStatuses = new ArrayList(); - for (ContainerProto c : list) { - finishedContainersList.add(convertFromProtoFormat(c)); + for (ContainerStatusProto c : list) { + completedContainersStatuses.add(convertFromProtoFormat(c)); } } @Override - public synchronized void addAllFinishedContainers(final List containers) { + public synchronized void setCompletedContainersStatuses( + final List containers) { if (containers == null) return; initLocalFinishedContainerList(); - finishedContainersList.addAll(containers); + completedContainersStatuses.addAll(containers); } - @Override - public synchronized void addFinishedContainer(Container containers) { - initLocalFinishedContainerList(); - if (containers == null) - return; - this.finishedContainersList.add(containers); - } - - @Override - public synchronized void removeFinishedContainer(int index) { - initLocalFinishedContainerList(); - this.finishedContainersList.remove(index); - } - @Override - public synchronized void clearFinishedContainers() { - initLocalFinishedContainerList(); - this.finishedContainersList.clear(); - } - - private synchronized ContainerPBImpl convertFromProtoFormat(ContainerProto p) { + private synchronized ContainerPBImpl convertFromProtoFormat( + ContainerProto p) { return new ContainerPBImpl(p); } @@ -294,6 +272,15 @@ private synchronized ContainerProto convertToProtoFormat(Container t) { return ((ContainerPBImpl)t).getProto(); } + private synchronized ContainerStatusPBImpl convertFromProtoFormat( + ContainerStatusProto p) { + return new ContainerStatusPBImpl(p); + } + + private synchronized ContainerStatusProto convertToProtoFormat(ContainerStatus t) { + return ((ContainerStatusPBImpl)t).getProto(); + } + private synchronized ResourcePBImpl convertFromProtoFormat(ResourceProto p) { return new ResourcePBImpl(p); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java index 7f6e9d98b5..8ad70764fb 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java @@ -18,35 +18,19 @@ package org.apache.hadoop.yarn.api.records.impl.pb; - -import java.text.NumberFormat; - import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ProtoBase; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProtoOrBuilder; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; -public class ApplicationAttemptIdPBImpl extends ProtoBase implements ApplicationAttemptId { +public class ApplicationAttemptIdPBImpl extends ApplicationAttemptId { ApplicationAttemptIdProto proto = ApplicationAttemptIdProto.getDefaultInstance(); ApplicationAttemptIdProto.Builder builder = null; boolean viaProto = false; private ApplicationId applicationId = null; - protected static final NumberFormat idFormat = NumberFormat.getInstance(); - static { - idFormat.setGroupingUsed(false); - idFormat.setMinimumIntegerDigits(4); - } - - protected static final NumberFormat counterFormat = NumberFormat.getInstance(); - static { - counterFormat.setGroupingUsed(false); - counterFormat.setMinimumIntegerDigits(6); - } - - + public ApplicationAttemptIdPBImpl() { builder = ApplicationAttemptIdProto.newBuilder(); } @@ -117,44 +101,11 @@ public synchronized void setApplicationId(ApplicationId appId) { this.applicationId = appId; } - private synchronized ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) { + private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) { return new ApplicationIdPBImpl(p); } - private synchronized ApplicationIdProto convertToProtoFormat(ApplicationId t) { + private ApplicationIdProto convertToProtoFormat(ApplicationId t) { return ((ApplicationIdPBImpl)t).getProto(); } - - @Override - public synchronized int hashCode() { - return getProto().hashCode(); - } - - @Override - public synchronized boolean equals(Object other) { - if (other == null) return false; - if (other.getClass().isAssignableFrom(this.getClass())) { - return this.getProto().equals(this.getClass().cast(other).getProto()); - } - return false; - } - - @Override - public synchronized int compareTo(ApplicationAttemptId other) { - int compareAppIds = this.getApplicationId().compareTo( - other.getApplicationId()); - if (compareAppIds == 0) { - return this.getAttemptId() - other.getAttemptId(); - } else { - return compareAppIds; - } - - } - - @Override - public synchronized String toString() { - String id = (this.getApplicationId() != null) ? this.getApplicationId().getClusterTimestamp() + "_" + - idFormat.format(this.getApplicationId().getId()): "none"; - return "appattempt_" + id + "_" + counterFormat.format(getAttemptId()); - } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java index 915da41413..ad5c778b4e 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java @@ -20,13 +20,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ProtoBase; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProtoOrBuilder; -public class ApplicationIdPBImpl extends ProtoBase implements ApplicationId { +public class ApplicationIdPBImpl extends ApplicationId { ApplicationIdProto proto = ApplicationIdProto.getDefaultInstance(); ApplicationIdProto.Builder builder = null; boolean viaProto = false; @@ -40,13 +39,13 @@ public ApplicationIdPBImpl(ApplicationIdProto proto) { viaProto = true; } - public ApplicationIdProto getProto() { + public synchronized ApplicationIdProto getProto() { proto = viaProto ? proto : builder.build(); viaProto = true; return proto; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = ApplicationIdProto.newBuilder(proto); } @@ -55,40 +54,25 @@ private void maybeInitBuilder() { @Override - public int getId() { + public synchronized int getId() { ApplicationIdProtoOrBuilder p = viaProto ? proto : builder; return (p.getId()); } @Override - public void setId(int id) { + public synchronized void setId(int id) { maybeInitBuilder(); builder.setId((id)); } @Override - public long getClusterTimestamp() { + public synchronized long getClusterTimestamp() { ApplicationIdProtoOrBuilder p = viaProto ? proto : builder; return (p.getClusterTimestamp()); } @Override - public void setClusterTimestamp(long clusterTimestamp) { + public synchronized void setClusterTimestamp(long clusterTimestamp) { maybeInitBuilder(); builder.setClusterTimestamp((clusterTimestamp)); } - - @Override - public int compareTo(ApplicationId other) { - if (this.getId() - other.getId() == 0) { - return this.getClusterTimestamp() > other.getClusterTimestamp() ? 1 : - this.getClusterTimestamp() < other.getClusterTimestamp() ? -1 : 0; - } else { - return this.getId() - other.getId(); - } - } - - @Override - public String toString() { - return "application_" + this.getClusterTimestamp() + "_" + this.getId(); - } -} +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java index a4e2d49d96..3227ce8a90 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java @@ -18,72 +18,20 @@ package org.apache.hadoop.yarn.api.records.impl.pb; - -import java.text.NumberFormat; - import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ProtoBase; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProtoOrBuilder; -public class ContainerIdPBImpl extends ProtoBase implements ContainerId { +public class ContainerIdPBImpl extends ContainerId { ContainerIdProto proto = ContainerIdProto.getDefaultInstance(); ContainerIdProto.Builder builder = null; boolean viaProto = false; - private ApplicationId applicationId = null; - private ApplicationAttemptId appAttemptId = null; - protected static final NumberFormat idFormat = NumberFormat.getInstance(); - static { - idFormat.setGroupingUsed(false); - idFormat.setMinimumIntegerDigits(4); - } - - protected static final NumberFormat counterFormat = NumberFormat.getInstance(); - static { - counterFormat.setGroupingUsed(false); - counterFormat.setMinimumIntegerDigits(6); - } - - // TODO: Why thread local? - // ^ NumberFormat instances are not threadsafe - private static final ThreadLocal appIdFormat = new ThreadLocal() { - @Override - public NumberFormat initialValue() { - NumberFormat fmt = NumberFormat.getInstance(); - fmt.setGroupingUsed(false); - fmt.setMinimumIntegerDigits(4); - return fmt; - } - }; + private ApplicationAttemptId applicationAttemptId = null; - // TODO: fail the app submission if attempts are more than 10 or something - private static final ThreadLocal appAttemptIdFormat = new ThreadLocal() { - @Override - public NumberFormat initialValue() { - NumberFormat fmt = NumberFormat.getInstance(); - fmt.setGroupingUsed(false); - fmt.setMinimumIntegerDigits(2); - return fmt; - } - }; - // TODO: Why thread local? - // ^ NumberFormat instances are not threadsafe - private static final ThreadLocal containerIdFormat = new ThreadLocal() { - @Override - public NumberFormat initialValue() { - NumberFormat fmt = NumberFormat.getInstance(); - fmt.setGroupingUsed(false); - fmt.setMinimumIntegerDigits(6); - return fmt; - } - }; - public ContainerIdPBImpl() { builder = ContainerIdProto.newBuilder(); } @@ -93,23 +41,22 @@ public ContainerIdPBImpl(ContainerIdProto proto) { viaProto = true; } - public ContainerIdProto getProto() { + public synchronized ContainerIdProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; return proto; } - private void mergeLocalToBuilder() { - if (this.applicationId != null && !((ApplicationIdPBImpl)applicationId).getProto().equals(builder.getAppId())) { - builder.setAppId(convertToProtoFormat(this.applicationId)); - } - if (this.appAttemptId != null && !((ApplicationAttemptIdPBImpl)appAttemptId).getProto().equals(builder.getAppAttemptId())) { - builder.setAppAttemptId(convertToProtoFormat(this.appAttemptId)); + private synchronized void mergeLocalToBuilder() { + if (this.applicationAttemptId != null && ! + ((ApplicationAttemptIdPBImpl)applicationAttemptId).getProto().equals( + builder.getAppAttemptId())) { + builder.setAppAttemptId(convertToProtoFormat(this.applicationAttemptId)); } } - private void mergeLocalToProto() { + private synchronized void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); mergeLocalToBuilder(); @@ -117,7 +64,7 @@ private void mergeLocalToProto() { viaProto = true; } - private void maybeInitBuilder() { + private synchronized void maybeInitBuilder() { if (viaProto || builder == null) { builder = ContainerIdProto.newBuilder(proto); } @@ -126,109 +73,46 @@ private void maybeInitBuilder() { @Override - public int getId() { + public synchronized int getId() { ContainerIdProtoOrBuilder p = viaProto ? proto : builder; return (p.getId()); } @Override - public void setId(int id) { + public synchronized void setId(int id) { maybeInitBuilder(); builder.setId((id)); } - @Override - public ApplicationId getAppId() { - ContainerIdProtoOrBuilder p = viaProto ? proto : builder; - if (this.applicationId != null) { - return this.applicationId; - } - if (!p.hasAppId()) { - return null; - } - this.applicationId = convertFromProtoFormat(p.getAppId()); - return this.applicationId; - } + @Override - public ApplicationAttemptId getAppAttemptId() { + public synchronized ApplicationAttemptId getApplicationAttemptId() { ContainerIdProtoOrBuilder p = viaProto ? proto : builder; - if (this.appAttemptId != null) { - return this.appAttemptId; + if (this.applicationAttemptId != null) { + return this.applicationAttemptId; } if (!p.hasAppAttemptId()) { return null; } - this.appAttemptId = convertFromProtoFormat(p.getAppAttemptId()); - return this.appAttemptId; + this.applicationAttemptId = convertFromProtoFormat(p.getAppAttemptId()); + return this.applicationAttemptId; } @Override - public void setAppId(ApplicationId appId) { - maybeInitBuilder(); - if (appId == null) - builder.clearAppId(); - this.applicationId = appId; - } - - @Override - public void setAppAttemptId(ApplicationAttemptId atId) { + public synchronized void setApplicationAttemptId(ApplicationAttemptId atId) { maybeInitBuilder(); if (atId == null) builder.clearAppAttemptId(); - this.appAttemptId = atId; + this.applicationAttemptId = atId; } - private ApplicationAttemptIdPBImpl convertFromProtoFormat(ApplicationAttemptIdProto p) { + private ApplicationAttemptIdPBImpl convertFromProtoFormat( + ApplicationAttemptIdProto p) { return new ApplicationAttemptIdPBImpl(p); } - private ApplicationAttemptIdProto convertToProtoFormat(ApplicationAttemptId t) { + private ApplicationAttemptIdProto convertToProtoFormat( + ApplicationAttemptId t) { return ((ApplicationAttemptIdPBImpl)t).getProto(); } - - private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) { - return new ApplicationIdPBImpl(p); - } - - private ApplicationIdProto convertToProtoFormat(ApplicationId t) { - return ((ApplicationIdPBImpl)t).getProto(); - } - - @Override - public int hashCode() { - return getProto().hashCode(); - } - - @Override - public boolean equals(Object other) { - if (other == null) { - return false; - } - if (other.getClass().isAssignableFrom(this.getClass())) { - return this.getProto().equals(this.getClass().cast(other).getProto()); - } - return false; - } - - @Override - public int compareTo(ContainerId other) { - if (this.getAppAttemptId().compareTo(other.getAppAttemptId()) == 0) { - return this.getId() - other.getId(); - } else { - return this.getAppAttemptId().compareTo(other.getAppAttemptId()); - } - - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - ApplicationId appId = getAppId(); - sb.append("container_").append(appId.getClusterTimestamp()).append("_"); - sb.append(appIdFormat.get().format(appId.getId())).append("_"); - sb.append(appAttemptIdFormat.get().format(getAppAttemptId(). - getAttemptId())).append("_"); - sb.append(containerIdFormat.get().format(getId())); - return sb.toString(); - } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java index 07d1705aa1..0696d8327b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java @@ -41,8 +41,11 @@ -public class ContainerLaunchContextPBImpl extends ProtoBase implements ContainerLaunchContext { - ContainerLaunchContextProto proto = ContainerLaunchContextProto.getDefaultInstance(); +public class ContainerLaunchContextPBImpl +extends ProtoBase +implements ContainerLaunchContext { + ContainerLaunchContextProto proto = + ContainerLaunchContextProto.getDefaultInstance(); ContainerLaunchContextProto.Builder builder = null; boolean viaProto = false; @@ -72,10 +75,14 @@ public ContainerLaunchContextProto getProto() { } private void mergeLocalToBuilder() { - if (this.containerId != null && !((ContainerIdPBImpl)containerId).getProto().equals(builder.getContainerId())) { + if (this.containerId != null && + !((ContainerIdPBImpl)containerId).getProto().equals( + builder.getContainerId())) { builder.setContainerId(convertToProtoFormat(this.containerId)); } - if (this.resource != null && !((ResourcePBImpl)this.resource).getProto().equals(builder.getResource())) { + if (this.resource != null && + !((ResourcePBImpl)this.resource).getProto().equals( + builder.getResource())) { builder.setResource(convertToProtoFormat(this.resource)); } if (this.localResources != null) { @@ -131,22 +138,13 @@ public void setResource(Resource resource) { builder.clearResource(); this.resource = resource; } + @Override - public List getCommandList() { + public List getCommands() { initCommands(); return this.commands; } - @Override - public String getCommand(int index) { - initCommands(); - return this.commands.get(index); - } - @Override - public int getCommandCount() { - initCommands(); - return this.commands.size(); - } - + private void initCommands() { if (this.commands != null) { return; @@ -161,11 +159,12 @@ private void initCommands() { } @Override - public void addAllCommands(final List command) { - if (command == null) + public void setCommands(final List commands) { + if (commands == null) return; initCommands(); - this.commands.addAll(command); + this.commands.clear(); + this.commands.addAll(commands); } private void addCommandsToProto() { @@ -175,21 +174,7 @@ private void addCommandsToProto() { return; builder.addAllCommand(this.commands); } - @Override - public void addCommand(String command) { - initCommands(); - this.commands.add(command); - } - @Override - public void removeCommand(int index) { - initCommands(); - this.commands.remove(index); - } - @Override - public void clearCommands() { - initCommands(); - this.commands.clear(); - } + @Override public String getUser() { ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder; @@ -228,17 +213,13 @@ public void setContainerId(ContainerId containerId) { builder.clearContainerId(); this.containerId = containerId; } + @Override - public Map getAllLocalResources() { + public Map getLocalResources() { initLocalResources(); return this.localResources; } - @Override - public LocalResource getLocalResource(String key) { - initLocalResources(); - return this.localResources.get(key); - } - + private void initLocalResources() { if (this.localResources != null) { return; @@ -253,10 +234,12 @@ private void initLocalResources() { } @Override - public void addAllLocalResources(final Map localResources) { + public void setLocalResources( + final Map localResources) { if (localResources == null) return; initLocalResources(); + this.localResources.clear(); this.localResources.putAll(localResources); } @@ -265,7 +248,8 @@ private void addLocalResourcesToProto() { builder.clearLocalResources(); if (localResources == null) return; - Iterable iterable = new Iterable() { + Iterable iterable = + new Iterable() { @Override public Iterator iterator() { @@ -281,7 +265,8 @@ public void remove() { @Override public StringLocalResourceMapProto next() { String key = keyIter.next(); - return StringLocalResourceMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(localResources.get(key))).build(); + return StringLocalResourceMapProto.newBuilder().setKey(key). + setValue(convertToProtoFormat(localResources.get(key))).build(); } @Override @@ -293,21 +278,7 @@ public boolean hasNext() { }; builder.addAllLocalResources(iterable); } - @Override - public void setLocalResource(String key, LocalResource val) { - initLocalResources(); - this.localResources.put(key, val); - } - @Override - public void removeLocalResource(String key) { - initLocalResources(); - this.localResources.remove(key); - } - @Override - public void clearLocalResources() { - initLocalResources(); - this.localResources.clear(); - } + @Override public ByteBuffer getContainerTokens() { ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder; @@ -328,16 +299,12 @@ public void setContainerTokens(ByteBuffer containerTokens) { builder.clearContainerTokens(); this.containerTokens = containerTokens; } + @Override - public Map getAllServiceData() { + public Map getServiceData() { initServiceData(); return this.serviceData; } - @Override - public ByteBuffer getServiceData(String key) { - initServiceData(); - return this.serviceData.get(key); - } private void initServiceData() { if (this.serviceData != null) { @@ -353,7 +320,7 @@ private void initServiceData() { } @Override - public void addAllServiceData(final Map serviceData) { + public void setServiceData(final Map serviceData) { if (serviceData == null) return; initServiceData(); @@ -365,7 +332,8 @@ private void addServiceDataToProto() { builder.clearServiceData(); if (serviceData == null) return; - Iterable iterable = new Iterable() { + Iterable iterable = + new Iterable() { @Override public Iterator iterator() { @@ -381,7 +349,8 @@ public void remove() { @Override public StringBytesMapProto next() { String key = keyIter.next(); - return StringBytesMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(serviceData.get(key))).build(); + return StringBytesMapProto.newBuilder().setKey(key).setValue( + convertToProtoFormat(serviceData.get(key))).build(); } @Override @@ -393,31 +362,12 @@ public boolean hasNext() { }; builder.addAllServiceData(iterable); } + @Override - public void setServiceData(String key, ByteBuffer val) { - initServiceData(); - this.serviceData.put(key, val); - } - @Override - public void removeServiceData(String key) { - initServiceData(); - this.serviceData.remove(key); - } - @Override - public void clearServiceData() { - initServiceData(); - this.serviceData.clear(); - } - @Override - public Map getAllEnv() { + public Map getEnv() { initEnv(); return this.env; } - @Override - public String getEnv(String key) { - initEnv(); - return this.env.get(key); - } private void initEnv() { if (this.env != null) { @@ -433,10 +383,11 @@ private void initEnv() { } @Override - public void addAllEnv(final Map env) { + public void setEnv(final Map env) { if (env == null) return; initEnv(); + this.env.clear(); this.env.putAll(env); } @@ -445,7 +396,8 @@ private void addEnvToProto() { builder.clearEnv(); if (env == null) return; - Iterable iterable = new Iterable() { + Iterable iterable = + new Iterable() { @Override public Iterator iterator() { @@ -461,7 +413,8 @@ public void remove() { @Override public StringStringMapProto next() { String key = keyIter.next(); - return StringStringMapProto.newBuilder().setKey(key).setValue((env.get(key))).build(); + return StringStringMapProto.newBuilder().setKey(key).setValue( + (env.get(key))).build(); } @Override @@ -473,21 +426,6 @@ public boolean hasNext() { }; builder.addAllEnv(iterable); } - @Override - public void setEnv(String key, String val) { - initEnv(); - this.env.put(key, val); - } - @Override - public void removeEnv(String key) { - initEnv(); - this.env.remove(key); - } - @Override - public void clearEnv() { - initEnv(); - this.env.clear(); - } private ResourcePBImpl convertFromProtoFormat(ResourceProto p) { return new ResourcePBImpl(p); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java index 8011b9d7a3..bfde5dd2d8 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java @@ -31,7 +31,8 @@ -public class ContainerStatusPBImpl extends ProtoBase implements ContainerStatus { +public class ContainerStatusPBImpl extends ProtoBase +implements ContainerStatus { ContainerStatusProto proto = ContainerStatusProto.getDefaultInstance(); ContainerStatusProto.Builder builder = null; boolean viaProto = false; @@ -116,13 +117,13 @@ public void setContainerId(ContainerId containerId) { this.containerId = containerId; } @Override - public String getExitStatus() { + public int getExitStatus() { ContainerStatusProtoOrBuilder p = viaProto ? proto : builder; - return (p.getExitStatus()); + return p.getExitStatus(); } @Override - public void setExitStatus(String exitStatus) { + public void setExitStatus(int exitStatus) { maybeInitBuilder(); builder.setExitStatus(exitStatus); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto index b91449c3df..61e3d1f5b9 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto @@ -177,8 +177,8 @@ message ResourceRequestProto { message AMResponseProto { optional bool reboot = 1; optional int32 response_id = 2; - repeated ContainerProto new_containers = 3; - repeated ContainerProto finished_containers = 4; + repeated ContainerProto allocated_containers = 3; + repeated ContainerStatusProto completed_container_statuses = 4; optional ResourceProto limit = 5; } @@ -250,7 +250,7 @@ message ContainerStatusProto { optional ContainerIdProto container_id = 1; optional ContainerStateProto state = 2; optional string diagnostics = 3 [default = "N/A"]; - optional string exit_status = 4 [default = "N/A"]; + optional int32 exit_status = 4 [default = -1000]; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 2c80adfb93..2169ee3e90 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -27,50 +27,333 @@ public class YarnConfiguration extends Configuration { private static final Splitter ADDR_SPLITTER = Splitter.on(':').trimResults(); private static final Joiner JOINER = Joiner.on(""); - public static final String RM_PREFIX = "yarn.server.resourcemanager."; - - public static final String SCHEDULER_ADDRESS = RM_PREFIX - + "scheduler.address"; - - public static final String AM_EXPIRY_INTERVAL = RM_PREFIX - + "application.expiry.interval"; - - public static final String DEFAULT_SCHEDULER_BIND_ADDRESS = "0.0.0.0:8030"; - - public static final String APPSMANAGER_ADDRESS = RM_PREFIX - + "appsManager.address"; - - public static final String YARN_SECURITY_INFO = - "yarn.security.info.class.name"; - - public static final String DEFAULT_APPSMANAGER_BIND_ADDRESS = - "0.0.0.0:8040"; - private static final String YARN_DEFAULT_XML_FILE = "yarn-default.xml"; private static final String YARN_SITE_XML_FILE = "yarn-site.xml"; - public static final String APPLICATION_MANAGER_PRINCIPAL = - "yarn.jobmanager.user-name"; - - public static final String RM_WEBAPP_BIND_ADDRESS = RM_PREFIX - + "webapp.address"; - - public static final String DEFAULT_RM_WEBAPP_BIND_ADDRESS = "0.0.0.0:8088"; - static { Configuration.addDefaultResource(YARN_DEFAULT_XML_FILE); Configuration.addDefaultResource(YARN_SITE_XML_FILE); } - public static final String RM_SERVER_PRINCIPAL_KEY = - "yarn.resourcemanager.principal"; - + //Configurations + + /** ACL of who can view this application.*/ public static final String APPLICATION_ACL_VIEW_APP = - "application.acl-view-job"; - + "yarn.app.acl.view-job"; + + /** ACL of who can modify this application.*/ public static final String APPLICATION_ACL_MODIFY_APP = - "application.acl-modify-job"; + "yarn.app.acl.modify-job"; + + /** + * Security info class This is an internal config set and + * read by YARN itself. + */ + public static final String YARN_SECURITY_INFO = + "yarn.security.info.class"; + + /** Delay before deleting resource to ease debugging of NM issues */ + public static final String DEBUG_NM_DELETE_DELAY_SEC = + YarnConfiguration.NM_PREFIX + "delete.debug-delay-sec"; + + //////////////////////////////// + // IPC Configs + //////////////////////////////// + public static final String IPC_PREFIX = "yarn.ipc."; + /** Factory to create client IPC classes.*/ + public static final String IPC_CLIENT_FACTORY = + IPC_PREFIX + "client.factory.class"; + + /** Type of serialization to use.*/ + public static final String IPC_SERIALIZER_TYPE = + IPC_PREFIX + "serializer.type"; + public static final String DEFAULT_IPC_SERIALIZER_TYPE = "protocolbuffers"; + + /** Factory to create server IPC classes.*/ + public static final String IPC_SERVER_FACTORY = + IPC_PREFIX + "server.factory.class"; + + /** Factory to create IPC exceptions.*/ + public static final String IPC_EXCEPTION_FACTORY = + IPC_PREFIX + "exception.factory.class"; + + /** Factory to create serializeable records.*/ + public static final String IPC_RECORD_FACTORY = + IPC_PREFIX + "record.factory.class"; + + /** RPC class implementation*/ + public static final String IPC_RPC_IMPL = + IPC_PREFIX + "rpc.class"; + public static final String DEFAULT_IPC_RPC_IMPL = + "org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC"; + + //////////////////////////////// + // Resource Manager Configs + //////////////////////////////// + public static final String RM_PREFIX = "yarn.resourcemanager."; + + /** The address of the applications manager interface in the RM.*/ + public static final String RM_ADDRESS = + RM_PREFIX + "address"; + public static final String DEFAULT_RM_ADDRESS = + "0.0.0.0:8040"; + + /** The number of threads used to handle applications manager requests.*/ + public static final String RM_CLIENT_THREAD_COUNT = + RM_PREFIX + "client.thread-count"; + public static final int DEFAULT_RM_CLIENT_THREAD_COUNT = 10; + + /** The expiry interval for application master reporting.*/ + public static final String RM_AM_EXPIRY_INTERVAL_MS = + RM_PREFIX + "am.liveness-monitor.expiry-interval-ms"; + public static final int DEFAULT_RM_AM_EXPIRY_INTERVAL_MS = 600000; + + /** The Kerberos principal for the resource manager.*/ + public static final String RM_PRINCIPAL = + RM_PREFIX + "principal"; + + /** The address of the scheduler interface.*/ + public static final String RM_SCHEDULER_ADDRESS = + RM_PREFIX + "scheduler.address"; + public static final String DEFAULT_RM_SCHEDULER_ADDRESS = "0.0.0.0:8030"; + + /** Number of threads to handle scheduler interface.*/ + public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT = + RM_PREFIX + "scheduler.client.thread-count"; + public static final int DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT = 10; + + /** The address of the RM web application.*/ + public static final String RM_WEBAPP_ADDRESS = + RM_PREFIX + "webapp.address"; + public static final String DEFAULT_RM_WEBAPP_ADDRESS = "0.0.0.0:8088"; + + public static final String RM_RESOURCE_TRACKER_ADDRESS = + RM_PREFIX + "resource-tracker.address"; + public static final String DEFAULT_RM_RESOURCE_TRACKER_ADDRESS = + "0.0.0.0:8025"; + + /** Are RM acls enabled.*/ + public static final String RM_ACL_ENABLE = + RM_PREFIX + "acl.enable"; + public static final boolean DEFAULT_RM_ACL_ENABLE = false; + + /** ACL of who can be admin of RM.*/ + public static final String RM_ADMIN_ACL = + RM_PREFIX + "admin.acl"; + public static final String DEFAULT_RM_ADMIN_ACL = "*"; + + /** The address of the RM admin interface.*/ + public static final String RM_ADMIN_ADDRESS = + RM_PREFIX + "admin.address"; + public static final String DEFAULT_RM_ADMIN_ADDRESS = "0.0.0.0:8141"; + + /**Number of threads used to handle RM admin interface.*/ + public static final String RM_ADMIN_CLIENT_THREAD_COUNT = + RM_PREFIX + "admin.client.thread-count"; + public static final int DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT = 1; + + /** How often should the RM check that the AM is still alive.*/ + public static final String RM_AM_LIVENESS_MONITOR_INTERVAL_MS = + RM_PREFIX + "amliveliness-monitor.interval-ms"; + public static final int DEFAULT_RM_AM_LIVENESS_MONITOR_INTERVAL_MS = 1000; + + /** The maximum number of application master retries.*/ + public static final String RM_AM_MAX_RETRIES = + RM_PREFIX + "am.max-retries"; + public static final int DEFAULT_RM_AM_MAX_RETRIES = 1; + + /** How often to check that containers are still alive. */ + public static final String RM_CONTAINER_LIVENESS_MONITOR_INTERVAL_MS = + RM_PREFIX + "container.liveness-monitor.interval-ms"; + public static final int DEFAULT_RM_CONTAINER_LIVENESS_MONITOR_INTERVAL_MS = + 600000; + + /** The keytab for the resource manager.*/ + public static final String RM_KEYTAB = + RM_PREFIX + "keytab"; + + /** How long to wait until a node manager is considered dead.*/ + public static final String RM_NM_EXPIRY_INTERVAL_MS = + RM_PREFIX + "nm.liveness-monitor.expiry-interval-ms"; + public static final int DEFAULT_RM_NM_EXPIRY_INTERVAL_MS = 600000; + + /** How often to check that node managers are still alive.*/ + public static final String RM_NM_LIVENESS_MONITOR_INTERVAL_MS = + RM_PREFIX + "nm.liveness-monitor.interval-ms"; + public static final int DEFAULT_RM_NM_LIVENESS_MONITOR_INTERVAL_MS = 1000; + + /** Path to file with nodes to include.*/ + public static final String RM_NODES_INCLUDE_FILE_PATH = + RM_PREFIX + "nodes.include-path"; + public static final String DEFAULT_RM_NODES_INCLUDE_FILE_PATH = ""; + + /** Path to file with nodes to exclude.*/ + public static final String RM_NODES_EXCLUDE_FILE_PATH = + RM_PREFIX + "nodes.exclude-path"; + public static final String DEFAULT_RM_NODES_EXCLUDE_FILE_PATH = ""; + + /** Number of threads to handle resource tracker calls.*/ + public static final String RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT = + RM_PREFIX + "resource-tracker.client.thread-count"; + public static final int DEFAULT_RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT = 10; + + /** The class to use as the resource scheduler.*/ + public static final String RM_SCHEDULER = + RM_PREFIX + "scheduler.class"; + + /** The class to use as the persistent store.*/ + public static final String RM_STORE = RM_PREFIX + "store.class"; + + /** The address of the zookeeper instance to use with ZK store.*/ + public static final String RM_ZK_STORE_ADDRESS = + RM_PREFIX + "zookeeper-store.address"; + + /** The zookeeper session timeout for the zookeeper store.*/ + public static final String RM_ZK_STORE_TIMEOUT_MS = + RM_PREFIX + "zookeeper-store.session.timeout-ms"; + public static final int DEFAULT_RM_ZK_STORE_TIMEOUT_MS = 60000; + + /** The maximum number of completed applications RM keeps. */ + public static final String RM_MAX_COMPLETED_APPLICATIONS = + RM_PREFIX + "max-completed-applications"; + public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 10000; + + //////////////////////////////// + // Node Manager Configs + //////////////////////////////// + + /** Prefix for all node manager configs.*/ + public static final String NM_PREFIX = "yarn.nodemanager."; + + /** address of node manager IPC.*/ + public static final String NM_ADDRESS = NM_PREFIX + "address"; + public static final String DEFAULT_NM_ADDRESS = "0.0.0.0:45454"; + + /** who will execute(launch) the containers.*/ + public static final String NM_CONTAINER_EXECUTOR = + NM_PREFIX + "container-executor.class"; + + /** Number of threads container manager uses.*/ + public static final String NM_CONTAINER_MGR_THREAD_COUNT = + NM_PREFIX + "container-manager.thread-count"; + public static final int DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT = 5; + + /** Number of threads used in cleanup.*/ + public static final String NM_DELETE_THREAD_COUNT = + NM_PREFIX + "delete.thread-count"; + public static final int DEFAULT_NM_DELETE_THREAD_COUNT = 4; + + // TODO: Should this instead be dictated by RM? + /** Heartbeat interval to RM*/ + public static final String NM_TO_RM_HEARTBEAT_INTERVAL_MS = + NM_PREFIX + "heartbeat.interval-ms"; + public static final int DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS = 1000; + + /** Keytab for NM.*/ + public static final String NM_KEYTAB = NM_PREFIX + "keytab"; + + /**List of directories to store localized files in.*/ + public static final String NM_LOCAL_DIRS = NM_PREFIX + "local-dirs"; + public static final String DEFAULT_NM_LOCAL_DIRS = "/tmp/nm-local-dir"; + + /** Address where the localizer IPC is.*/ + public static final String NM_LOCALIZER_ADDRESS = + NM_PREFIX + "localizer.address"; + public static final String DEFAULT_NM_LOCALIZER_ADDRESS = "0.0.0.0:4344"; + + /** Interval in between cache cleanups.*/ + public static final String NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = + NM_PREFIX + "localizer.cache.cleanup.interval-ms"; + public static final long DEFAULT_NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = + 10 * 60 * 1000; + + /** Target size of localizer cache in MB, per local directory.*/ + public static final String NM_LOCALIZER_CACHE_TARGET_SIZE_MB = + NM_PREFIX + "localizer.cache.target-size-mb"; + public static final long DEFAULT_NM_LOCALIZER_CACHE_TARGET_SIZE_MB = 10 * 1024; + + /** Number of threads to handle localization requests.*/ + public static final String NM_LOCALIZER_CLIENT_THREAD_COUNT = + NM_PREFIX + "localizer.client.thread-count"; + public static final int DEFAULT_NM_LOCALIZER_CLIENT_THREAD_COUNT = 5; + + /** Number of threads to use for localization fetching.*/ + public static final String NM_LOCALIZER_FETCH_THREAD_COUNT = + NM_PREFIX + "localizer.fetch.thread-count"; + public static final int DEFAULT_NM_LOCALIZER_FETCH_THREAD_COUNT = 4; + + /** Where to store container logs.*/ + public static final String NM_LOG_DIRS = NM_PREFIX + "log-dirs"; + public static final String DEFAULT_NM_LOG_DIRS = "/tmp/logs"; + + /** Where to aggregate logs to.*/ + public static final String NM_REMOTE_APP_LOG_DIR = + NM_PREFIX + "remote-app-log-dir"; + public static final String DEFAULT_NM_REMOTE_APP_LOG_DIR = "/tmp/logs"; + + /** Amount of memory in GB that can be allocated for containers.*/ + public static final String NM_VMEM_GB = NM_PREFIX + "resource.memory-gb"; + public static final int DEFAULT_NM_VMEM_GB = 8; + + /** NM Webapp address.**/ + public static final String NM_WEBAPP_ADDRESS = NM_PREFIX + "webapp.address"; + public static final String DEFAULT_NM_WEBAPP_ADDRESS = "0.0.0.0:9999"; + + /** How often to monitor containers.*/ + public final static String NM_CONTAINER_MON_INTERVAL_MS = + NM_PREFIX + "container-monitor.interval-ms"; + public final static int DEFAULT_NM_CONTAINER_MON_INTERVAL_MS = 3000; + + /** Class that calculates containers current resource utilization.*/ + public static final String NM_CONTAINER_MON_RESOURCE_CALCULATOR = + NM_PREFIX + "container-monitor.resource-calculator.class"; + + /** Amount of physical ram to reserve for other applications, -1 disables.*/ + public static final String NM_RESERVED_MEMORY_MB = + NM_PREFIX + "reserved.memory-mb"; + + /** Frequency of running node health script.*/ + public static final String NM_HEALTH_CHECK_INTERVAL_MS = + NM_PREFIX + "health-checker.interval-ms"; + public static final long DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS = 10 * 60 * 1000; + + /** Script time out period.*/ + public static final String NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS = + NM_PREFIX + "health-checker.script.timeout-ms"; + public static final long DEFAULT_NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS = + 2 * DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS; + + /** The health check script to run.*/ + public static final String NM_HEALTH_CHECK_SCRIPT_PATH = + NM_PREFIX + "health-checker.script.path"; + + /** The arguments to pass to the health check script.*/ + public static final String NM_HEALTH_CHECK_SCRIPT_OPTS = + NM_PREFIX + "health-checker.script.opts"; + + /** The path to the Linux container executor.*/ + public static final String NM_LINUX_CONTAINER_EXECUTOR_PATH = + NM_PREFIX + "linux-container-executor.path"; + + /** T-file compression types used to compress aggregated logs.*/ + public static final String NM_LOG_AGG_COMPRESSION_TYPE = + NM_PREFIX + "log-aggregation.compression-type"; + public static final String DEFAULT_NM_LOG_AGG_COMPRESSION_TYPE = "none"; + + /** The kerberos principal for the node manager.*/ + public static final String NM_PRINCIPAL = + NM_PREFIX + "principal"; + + public static final String NM_AUX_SERVICES = + NM_PREFIX + "aux-services"; + + public static final String NM_AUX_SERVICE_FMT = + NM_PREFIX + "aux-services.%s.class"; + + public static final int INVALID_CONTAINER_EXIT_STATUS = -1000; + public static final int ABORTED_CONTAINER_EXIT_STATUS = -100; + public YarnConfiguration() { super(); } @@ -83,13 +366,13 @@ public YarnConfiguration(Configuration conf) { } public static String getRMWebAppURL(Configuration conf) { - String addr = conf.get(RM_WEBAPP_BIND_ADDRESS, - DEFAULT_RM_WEBAPP_BIND_ADDRESS); + String addr = conf.get(YarnConfiguration.RM_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS); Iterator it = ADDR_SPLITTER.split(addr).iterator(); it.next(); // ignore the bind host String port = it.next(); // Use apps manager address to figure out the host for webapp - addr = conf.get(APPSMANAGER_ADDRESS, DEFAULT_APPSMANAGER_BIND_ADDRESS); + addr = conf.get(YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS); String host = ADDR_SPLITTER.split(addr).iterator().next(); return JOINER.join("http://", host, ":", port, "/"); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java index 253c3b1c28..c94ff737b0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java @@ -23,16 +23,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.impl.pb.RecordFactoryPBImpl; public class RecordFactoryProvider { - - public static final String RPC_SERIALIZER_KEY = "org.apache.yarn.ipc.rpc.serializer.property"; - public static final String RPC_SERIALIZER_DEFAULT = "protocolbuffers"; - - public static final String RECORD_FACTORY_CLASS_KEY = "org.apache.yarn.ipc.record.factory.class"; - private static Configuration defaultConf; static { @@ -48,13 +43,13 @@ public static RecordFactory getRecordFactory(Configuration conf) { //Users can specify a particular factory by providing a configuration. conf = defaultConf; } - String recordFactoryClassName = conf.get(RECORD_FACTORY_CLASS_KEY); + String recordFactoryClassName = conf.get(YarnConfiguration.IPC_RECORD_FACTORY); if (recordFactoryClassName == null) { - String serializer = conf.get(RPC_SERIALIZER_KEY, RPC_SERIALIZER_DEFAULT); - if (serializer.equals(RPC_SERIALIZER_DEFAULT)) { + String serializer = conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE, YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE); + if (serializer.equals(YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE)) { return RecordFactoryPBImpl.get(); } else { - throw new YarnException("Unknown serializer: [" + conf.get(RPC_SERIALIZER_KEY) + "]. Use keys: [" + RECORD_FACTORY_CLASS_KEY + "] to specify Record factory"); + throw new YarnException("Unknown serializer: [" + conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE) + "]. Use keys: [" + YarnConfiguration.IPC_RECORD_FACTORY + "] to specify Record factory"); } } else { return (RecordFactory) getFactoryClassInstance(recordFactoryClassName); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java index d06ba72363..d5c5ce1a89 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/RpcFactoryProvider.java @@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RpcClientFactory; import org.apache.hadoop.yarn.factories.RpcServerFactory; import org.apache.hadoop.yarn.factories.impl.pb.RpcClientFactoryPBImpl; @@ -35,13 +36,7 @@ */ public class RpcFactoryProvider { private static final Log LOG = LogFactory.getLog(RpcFactoryProvider.class); - //TODO Move these keys to CommonConfigurationKeys - public static final String RPC_SERIALIZER_KEY = "org.apache.yarn.ipc.rpc.serializer.property"; - public static final String RPC_SERIALIZER_DEFAULT = "protocolbuffers"; - public static final String RPC_CLIENT_FACTORY_CLASS_KEY = "org.apache.yarn.ipc.client.factory.class"; - public static final String RPC_SERVER_FACTORY_CLASS_KEY = "org.apache.yarn.ipc.server.factory.class"; - private RpcFactoryProvider() { } @@ -51,12 +46,12 @@ public static RpcServerFactory getServerFactory(Configuration conf) { if (conf == null) { conf = new Configuration(); } - String serverFactoryClassName = conf.get(RPC_SERVER_FACTORY_CLASS_KEY); + String serverFactoryClassName = conf.get(YarnConfiguration.IPC_SERVER_FACTORY); if (serverFactoryClassName == null) { - if (conf.get(RPC_SERIALIZER_KEY, RPC_SERIALIZER_DEFAULT).equals(RPC_SERIALIZER_DEFAULT)) { + if (conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE, YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE).equals(YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE)) { return RpcServerFactoryPBImpl.get(); } else { - throw new YarnException("Unknown serializer: [" + conf.get(RPC_SERIALIZER_KEY) + "]. Use keys: [" + RPC_CLIENT_FACTORY_CLASS_KEY + "][" + RPC_SERVER_FACTORY_CLASS_KEY + "] to specify factories"); + throw new YarnException("Unknown serializer: [" + conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE) + "]. Use keys: [" + YarnConfiguration.IPC_CLIENT_FACTORY + "][" + YarnConfiguration.IPC_SERVER_FACTORY + "] to specify factories"); } } else { return (RpcServerFactory) getFactoryClassInstance(serverFactoryClassName); @@ -64,12 +59,12 @@ public static RpcServerFactory getServerFactory(Configuration conf) { } public static RpcClientFactory getClientFactory(Configuration conf) { - String clientFactoryClassName = conf.get(RPC_CLIENT_FACTORY_CLASS_KEY); + String clientFactoryClassName = conf.get(YarnConfiguration.IPC_CLIENT_FACTORY); if (clientFactoryClassName == null) { - if (conf.get(RPC_SERIALIZER_KEY, RPC_SERIALIZER_DEFAULT).equals(RPC_SERIALIZER_DEFAULT)) { + if (conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE, YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE).equals(YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE)) { return RpcClientFactoryPBImpl.get(); } else { - throw new YarnException("Unknown serializer: [" + conf.get(RPC_SERIALIZER_KEY) + "]. Use keys: [" + RPC_CLIENT_FACTORY_CLASS_KEY + "][" + RPC_SERVER_FACTORY_CLASS_KEY + "] to specify factories"); + throw new YarnException("Unknown serializer: [" + conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE) + "]. Use keys: [" + YarnConfiguration.IPC_CLIENT_FACTORY + "][" + YarnConfiguration.IPC_SERVER_FACTORY + "] to specify factories"); } } else { return(RpcClientFactory) getFactoryClassInstance(clientFactoryClassName); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/YarnRemoteExceptionFactoryProvider.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/YarnRemoteExceptionFactoryProvider.java index b0d17e4134..2cd9f12d92 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/YarnRemoteExceptionFactoryProvider.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/YarnRemoteExceptionFactoryProvider.java @@ -23,16 +23,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.YarnRemoteExceptionFactory; import org.apache.hadoop.yarn.factories.impl.pb.YarnRemoteExceptionFactoryPBImpl; public class YarnRemoteExceptionFactoryProvider { - public static final String RPC_SERIALIZER_KEY = "org.apache.yarn.ipc.rpc.serializer.property"; - public static final String RPC_SERIALIZER_DEFAULT = "protocolbuffers"; - - public static final String EXCEPTION_FACTORY_CLASS_KEY = "org.apache.yarn.ipc.exception.factory.class"; - private YarnRemoteExceptionFactoryProvider() { } @@ -40,13 +36,13 @@ public static YarnRemoteExceptionFactory getYarnRemoteExceptionFactory(Configura if (conf == null) { conf = new Configuration(); } - String recordFactoryClassName = conf.get(EXCEPTION_FACTORY_CLASS_KEY); + String recordFactoryClassName = conf.get(YarnConfiguration.IPC_EXCEPTION_FACTORY); if (recordFactoryClassName == null) { - String serializer = conf.get(RPC_SERIALIZER_KEY, RPC_SERIALIZER_DEFAULT); - if (serializer.equals(RPC_SERIALIZER_DEFAULT)) { + String serializer = conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE, YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE); + if (serializer.equals(YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE)) { return YarnRemoteExceptionFactoryPBImpl.get(); } else { - throw new YarnException("Unknown serializer: [" + conf.get(RPC_SERIALIZER_KEY) + "]. Use keys: [" + EXCEPTION_FACTORY_CLASS_KEY + "] to specify Exception factory"); + throw new YarnException("Unknown serializer: [" + conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE) + "]. Use keys: [" + YarnConfiguration.IPC_EXCEPTION_FACTORY + "] to specify Exception factory"); } } else { return (YarnRemoteExceptionFactory) getFactoryClassInstance(recordFactoryClassName); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java index b6f96597e4..9a623a1a8a 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java @@ -73,15 +73,6 @@ public ProtocolProxy getProxy(Class protocol, long clientVersion, addr, ticket, conf, factory, rpcTimeout)), false); } - @Override - public void stopProxy(Object proxy) { - try { - ((Invoker) Proxy.getInvocationHandler(proxy)).close(); - } catch (IOException e) { - LOG.warn("Error while stopping " + proxy, e); - } - } - private static class Invoker implements InvocationHandler, Closeable { private Map returnTypes = new ConcurrentHashMap(); private boolean isClosed = false; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java index 8b476d8b7b..bbd02a0c70 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java @@ -27,6 +27,7 @@ import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; /** * Abstraction to get the RPC implementation for Yarn. @@ -34,13 +35,6 @@ public abstract class YarnRPC { private static final Log LOG = LogFactory.getLog(YarnRPC.class); - public static final String RPC_CLASSNAME - = "org.apache.hadoop.yarn.ipc.YarnRPC.classname"; - - //use the default as Hadoop RPC - public static final String DEFAULT_RPC - = "org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC"; - public abstract Object getProxy(Class protocol, InetSocketAddress addr, Configuration conf); @@ -50,10 +44,10 @@ public abstract Server getServer(Class protocol, Object instance, int numHandlers); public static YarnRPC create(Configuration conf) { - LOG.info("Creating YarnRPC for " + conf.get(RPC_CLASSNAME)); - String clazzName = conf.get(RPC_CLASSNAME); + LOG.info("Creating YarnRPC for " + conf.get(YarnConfiguration.IPC_RPC_IMPL)); + String clazzName = conf.get(YarnConfiguration.IPC_RPC_IMPL); if (clazzName == null) { - clazzName = DEFAULT_RPC; + clazzName = YarnConfiguration.DEFAULT_IPC_RPC_IMPL; } try { return (YarnRPC) Class.forName(clazzName).newInstance(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java index 521ca27c77..c2510bca3d 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java @@ -68,28 +68,42 @@ public Resource getResource() { @Override public void write(DataOutput out) throws IOException { LOG.debug("Writing ContainerTokenIdentifier to RPC layer"); - out.writeInt(this.containerId.getAppId().getId()); - out.writeInt(this.containerId.getAppAttemptId().getAttemptId()); + ApplicationAttemptId applicationAttemptId = + containerId.getApplicationAttemptId(); + ApplicationId applicationId = applicationAttemptId.getApplicationId(); + out.writeLong(applicationId.getClusterTimestamp()); + out.writeInt(applicationId.getId()); + out.writeInt(applicationAttemptId.getAttemptId()); out.writeInt(this.containerId.getId()); - // TODO: Cluster time-stamp? out.writeUTF(this.nmHostName); - out.writeInt(this.resource.getMemory()); // TODO: more resources. + out.writeInt(this.resource.getMemory()); } @Override public void readFields(DataInput in) throws IOException { - this.containerId = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ContainerId.class); - this.containerId.setAppId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class)); - this.containerId.setAppAttemptId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationAttemptId.class)); - this.containerId.getAppId().setId(in.readInt()); - this.containerId.getAppAttemptId().setApplicationId(this.containerId.getAppId()); - this.containerId.getAppAttemptId().setAttemptId(in.readInt()); + this.containerId = + RecordFactoryProvider.getRecordFactory(null).newRecordInstance( + ContainerId.class); + ApplicationAttemptId applicationAttemptId = + RecordFactoryProvider.getRecordFactory(null).newRecordInstance( + ApplicationAttemptId.class); + ApplicationId applicationId = + RecordFactoryProvider.getRecordFactory(null).newRecordInstance( + ApplicationId.class); + applicationId.setClusterTimestamp(in.readLong()); + applicationId.setId(in.readInt()); + applicationAttemptId.setApplicationId(applicationId); + applicationAttemptId.setAttemptId(in.readInt()); + this.containerId.setApplicationAttemptId(applicationAttemptId); this.containerId.setId(in.readInt()); this.nmHostName = in.readUTF(); - this.resource = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Resource.class); - this.resource.setMemory(in.readInt()); // TODO: more resources. + this.resource = + RecordFactoryProvider.getRecordFactory(null).newRecordInstance( + Resource.class); + this.resource.setMemory(in.readInt()); } + @SuppressWarnings("static-access") @Override public Text getKind() { return this.KIND; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientRMSecurityInfo.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientRMSecurityInfo.java index 90e0855a24..ef26a3e177 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientRMSecurityInfo.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientRMSecurityInfo.java @@ -44,7 +44,7 @@ public Class annotationType() { @Override public String serverPrincipal() { - return YarnConfiguration.RM_SERVER_PRINCIPAL_KEY; + return YarnConfiguration.RM_PRINCIPAL; } @Override diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java index fcb3f956e0..90e181576c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java @@ -33,10 +33,6 @@ public class Apps { public static final String APP = "app"; public static final String ID = "ID"; - public static String toString(ApplicationId id) { - return _join("app", id.getClusterTimestamp(), id.getId()); - } - public static ApplicationId toAppID(String aid) { Iterator it = _split(aid).iterator(); return toAppID(APP, aid, it); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java index 1afe464061..4eb63c0447 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java @@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; @@ -86,12 +87,11 @@ public int compare(org.apache.hadoop.yarn.api.records.ResourceRequest r1, } } - public static LocalResource newLocalResource(RecordFactory recordFactory, - URI uri, LocalResourceType type, LocalResourceVisibility visibility, - long size, long timestamp) { + public static LocalResource newLocalResource(URL url, LocalResourceType type, + LocalResourceVisibility visibility, long size, long timestamp) { LocalResource resource = - recordFactory.newRecordInstance(LocalResource.class); - resource.setResource(ConverterUtils.getYarnUrlFromURI(uri)); + recordFactory.newRecordInstance(LocalResource.class); + resource.setResource(url); resource.setType(type); resource.setVisibility(visibility); resource.setSize(size); @@ -99,6 +99,13 @@ public static LocalResource newLocalResource(RecordFactory recordFactory, return resource; } + public static LocalResource newLocalResource(URI uri, + LocalResourceType type, LocalResourceVisibility visibility, long size, + long timestamp) { + return newLocalResource(ConverterUtils.getYarnUrlFromURI(uri), type, + visibility, size, timestamp); + } + public static ApplicationId newApplicationId(RecordFactory recordFactory, long clustertimestamp, CharSequence id) { ApplicationId applicationId = @@ -125,6 +132,15 @@ public static ApplicationId newApplicationId(long clusterTimeStamp, int id) { return applicationId; } + public static ApplicationAttemptId newApplicationAttemptId( + ApplicationId appId, int attemptId) { + ApplicationAttemptId appAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); + appAttemptId.setApplicationId(appId); + appAttemptId.setAttemptId(attemptId); + return appAttemptId; + } + public static ApplicationId convert(long clustertimestamp, CharSequence id) { ApplicationId applicationId = recordFactory.newRecordInstance(ApplicationId.class); @@ -133,13 +149,29 @@ public static ApplicationId convert(long clustertimestamp, CharSequence id) { return applicationId; } + public static ContainerId newContainerId(ApplicationAttemptId appAttemptId, + int containerId) { + ContainerId id = recordFactory.newRecordInstance(ContainerId.class); + id.setId(containerId); + id.setApplicationAttemptId(appAttemptId); + return id; + } + + public static ContainerId newContainerId(int appId, int appAttemptId, + long timestamp, int id) { + ApplicationId applicationId = newApplicationId(timestamp, appId); + ApplicationAttemptId applicationAttemptId = newApplicationAttemptId( + applicationId, appAttemptId); + ContainerId cId = newContainerId(applicationAttemptId, id); + return cId; + } + public static ContainerId newContainerId(RecordFactory recordFactory, ApplicationId appId, ApplicationAttemptId appAttemptId, int containerId) { ContainerId id = recordFactory.newRecordInstance(ContainerId.class); - id.setAppId(appId); id.setId(containerId); - id.setAppAttemptId(appAttemptId); + id.setApplicationAttemptId(appAttemptId); return id; } @@ -147,8 +179,7 @@ public static ContainerId newContainerId(RecordFactory recordFactory, ApplicationAttemptId appAttemptId, int containerId) { ContainerId id = recordFactory.newRecordInstance(ContainerId.class); - id.setAppAttemptId(appAttemptId); - id.setAppId(appAttemptId.getApplicationId()); + id.setApplicationAttemptId(appAttemptId); id.setId(containerId); return id; } @@ -227,4 +258,20 @@ public static ApplicationReport newApplicationReport( report.setStartTime(startTime); return report; } + + public static Resource newResource(int memory) { + Resource resource = recordFactory.newRecordInstance(Resource.class); + resource.setMemory(memory); + return resource; + } + + public static URL newURL(String scheme, String host, int port, String file) { + URL url = recordFactory.newRecordInstance(URL.class); + url.setScheme(scheme); + url.setHost(host); + url.setPort(port); + url.setFile(file); + return url; + } + } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java index 0d02cb96b4..ab6bd7395d 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java @@ -29,6 +29,7 @@ import java.util.Map.Entry; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.URL; @@ -130,6 +131,20 @@ private static ApplicationId toApplicationId(RecordFactory recordFactory, return appId; } + private static ApplicationAttemptId toApplicationAttemptId( + RecordFactory recordFactory, + Iterator it) { + ApplicationId appId = + recordFactory.newRecordInstance(ApplicationId.class); + appId.setClusterTimestamp(Long.parseLong(it.next())); + appId.setId(Integer.parseInt(it.next())); + ApplicationAttemptId appAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); + appAttemptId.setApplicationId(appId); + appAttemptId.setAttemptId(Integer.parseInt(it.next())); + return appAttemptId; + } + public static String toString(ContainerId cId) { return cId.toString(); } @@ -138,10 +153,11 @@ public static ContainerId toContainerId(RecordFactory recordFactory, String containerIdStr) { Iterator it = _split(containerIdStr).iterator(); it.next(); // prefix. TODO: Validate container prefix - ApplicationId appID = toApplicationId(recordFactory, it); + ApplicationAttemptId appAttemptID = + toApplicationAttemptId(recordFactory, it); ContainerId containerId = recordFactory.newRecordInstance(ContainerId.class); - containerId.setAppId(appID); + containerId.setApplicationAttemptId(appAttemptID); containerId.setId(Integer.parseInt(it.next())); return containerId; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java index b588395ad4..255dcb6036 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java @@ -97,7 +97,7 @@ public abstract class ResourceCalculatorPlugin extends Configured { @InterfaceStability.Unstable public abstract ProcResourceValues getProcResourceValues(); - public class ProcResourceValues { + public static class ProcResourceValues { private final long cumulativeCpuTime; private final long physicalMemorySize; private final long virtualMemorySize; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java index 55f9010627..85b88d16cc 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java @@ -67,7 +67,6 @@ public static class Builder { boolean findPort = false; Configuration conf; boolean devMode = false; - Module[] modules; Builder(String name, Class api, T application) { this.name = name; @@ -99,11 +98,6 @@ public Builder with(Configuration conf) { return this; } - public Builder with(Module... modules) { - this.modules = modules; // OK - return this; - } - public Builder inDevMode() { devMode = true; return this; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java index c33fa97a73..4ff504ce93 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java @@ -26,6 +26,11 @@ import org.apache.hadoop.yarn.webapp.WebAppException; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +/** + * The parent class of all HTML pages. Override + * {@link #render(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)} + * to actually render the page. + */ public abstract class HtmlPage extends TextView { public static class _ implements Hamlet._ { @@ -79,6 +84,10 @@ public void render() { } } + /** + * Render the the HTML page. + * @param html the page to render data to. + */ protected abstract void render(Page.HTML<_> html); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java index 7069ef3cef..bd5ec4885d 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java @@ -133,8 +133,12 @@ protected void initDataTables(List list) { if (init.isEmpty()) { init = defaultInit; } - list.add(join(" $('#", id, "').dataTable(", init, + list.add(join(id,"DataTable = $('#", id, "').dataTable(", init, ").fnSetFilteringDelay(188);")); + String postInit = $(postInitID(DATATABLES, id)); + if(!postInit.isEmpty()) { + list.add(postInit); + } } } String selector = $(DATATABLES_SELECTOR); @@ -210,6 +214,10 @@ protected String getTheme() { public static String initID(String name, String id) { return djoin(name, id, "init"); } + + public static String postInitID(String name, String id) { + return djoin(name, id, "postinit"); + } public static String initSelector(String name) { return djoin(name, "selector.init"); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java index c79e7de0c2..f32ab38f05 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java @@ -18,21 +18,25 @@ package org.apache.hadoop.yarn.webapp.view; -import com.google.common.collect.Lists; -import com.google.inject.Inject; -import java.util.List; +import static org.apache.hadoop.yarn.util.StringHelper.join; -import static org.apache.hadoop.yarn.util.StringHelper.*; -import static org.apache.hadoop.yarn.webapp.Params.*; +import java.util.List; import org.apache.hadoop.yarn.webapp.SubView; +import com.google.common.collect.Lists; + /** - * A simpler two column layout implementation. Works with resizable themes. + * A simpler two column layout implementation with a header, a navigation bar + * on the left, content on the right, and a footer. Works with resizable themes. * @see TwoColumnCssLayout */ public class TwoColumnLayout extends HtmlPage { + /* + * (non-Javadoc) + * @see org.apache.hadoop.yarn.webapp.view.HtmlPage#render(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ @Override protected void render(Page.HTML<_> html) { preHead(html); html. @@ -65,28 +69,55 @@ public class TwoColumnLayout extends HtmlPage { _(content())._()._()._()._()._(); } + /** + * Do what needs to be done before the header is rendered. This usually + * involves setting page variables for Javascript and CSS rendering. + * @param html the html to use to render. + */ protected void preHead(Page.HTML<_> html) { } + /** + * Do what needs to be done after the header is rendered. + * @param html the html to use to render. + */ protected void postHead(Page.HTML<_> html) { } + /** + * @return the class that will render the header of the page. + */ protected Class header() { return HeaderBlock.class; } + /** + * @return the class that will render the content of the page. + */ protected Class content() { return LipsumBlock.class; } + /** + * @return the class that will render the navigation bar. + */ protected Class nav() { return NavBlock.class; } + /** + * @return the class that will render the footer. + */ protected Class footer() { return FooterBlock.class; } + /** + * Sets up a table to be a consistent style. + * @param html the HTML to use to render. + * @param tableId the ID of the table to set styles on. + * @param innerStyles any other styles to add to the table. + */ protected void setTableStyles(Page.HTML<_> html, String tableId, String... innerStyles) { List styles = Lists.newArrayList(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java index 28ed4cfcb0..58efcc4230 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java @@ -32,12 +32,14 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; @@ -70,7 +72,7 @@ public void testHadoopProtoRPC() throws Exception { private void test(String rpcClass) throws Exception { Configuration conf = new Configuration(); - conf.set(YarnRPC.RPC_CLASSNAME, rpcClass); + conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass); YarnRPC rpc = YarnRPC.create(conf); String bindAddr = "localhost:0"; InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); @@ -80,21 +82,34 @@ private void test(String rpcClass) throws Exception { ContainerManager proxy = (ContainerManager) rpc.getProxy(ContainerManager.class, NetUtils.createSocketAddr("localhost:" + server.getPort()), conf); - ContainerLaunchContext containerLaunchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class); + ContainerLaunchContext containerLaunchContext = + recordFactory.newRecordInstance(ContainerLaunchContext.class); containerLaunchContext.setUser("dummy-user"); - containerLaunchContext.setContainerId(recordFactory.newRecordInstance(ContainerId.class)); - containerLaunchContext.getContainerId().setAppId(recordFactory.newRecordInstance(ApplicationId.class)); - containerLaunchContext.getContainerId().getAppId().setId(0); - containerLaunchContext.getContainerId().setId(100); - containerLaunchContext.setResource(recordFactory.newRecordInstance(Resource.class)); + ContainerId containerId = + recordFactory.newRecordInstance(ContainerId.class); + ApplicationId applicationId = + recordFactory.newRecordInstance(ApplicationId.class); + ApplicationAttemptId applicationAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); + applicationId.setClusterTimestamp(0); + applicationId.setId(0); + applicationAttemptId.setApplicationId(applicationId); + applicationAttemptId.setAttemptId(0); + containerId.setApplicationAttemptId(applicationAttemptId); + containerId.setId(100); + containerLaunchContext.setContainerId(containerId); + containerLaunchContext.setResource( + recordFactory.newRecordInstance(Resource.class)); // containerLaunchContext.env = new HashMap(); // containerLaunchContext.command = new ArrayList(); - StartContainerRequest scRequest = recordFactory.newRecordInstance(StartContainerRequest.class); + StartContainerRequest scRequest = + recordFactory.newRecordInstance(StartContainerRequest.class); scRequest.setContainerLaunchContext(containerLaunchContext); proxy.startContainer(scRequest); - GetContainerStatusRequest gcsRequest = recordFactory.newRecordInstance(GetContainerStatusRequest.class); + GetContainerStatusRequest gcsRequest = + recordFactory.newRecordInstance(GetContainerStatusRequest.class); gcsRequest.setContainerId(containerLaunchContext.getContainerId()); GetContainerStatusResponse response = proxy.getContainerStatus(gcsRequest); ContainerStatus status = response.getStatus(); @@ -117,7 +132,7 @@ private void test(String rpcClass) throws Exception { server.close(); Assert.assertNotNull(status); - Assert.assertEquals(ContainerState.RUNNING, status.getState().RUNNING); + Assert.assertEquals(ContainerState.RUNNING, status.getState()); } public class DummyContainerManager implements ContainerManager { @@ -125,28 +140,35 @@ public class DummyContainerManager implements ContainerManager { private ContainerStatus status = null; @Override - public GetContainerStatusResponse getContainerStatus(GetContainerStatusRequest request) throws YarnRemoteException { - GetContainerStatusResponse response = recordFactory.newRecordInstance(GetContainerStatusResponse.class); + public GetContainerStatusResponse getContainerStatus( + GetContainerStatusRequest request) + throws YarnRemoteException { + GetContainerStatusResponse response = + recordFactory.newRecordInstance(GetContainerStatusResponse.class); response.setStatus(status); return response; } @Override - public StartContainerResponse startContainer(StartContainerRequest request) throws YarnRemoteException { + public StartContainerResponse startContainer(StartContainerRequest request) + throws YarnRemoteException { ContainerLaunchContext container = request.getContainerLaunchContext(); - StartContainerResponse response = recordFactory.newRecordInstance(StartContainerResponse.class); + StartContainerResponse response = + recordFactory.newRecordInstance(StartContainerResponse.class); status = recordFactory.newRecordInstance(ContainerStatus.class); status.setState(ContainerState.RUNNING); status.setContainerId(container.getContainerId()); - status.setExitStatus(String.valueOf(0)); + status.setExitStatus(0); return response; } @Override - public StopContainerResponse stopContainer(StopContainerRequest request) throws YarnRemoteException { + public StopContainerResponse stopContainer(StopContainerRequest request) + throws YarnRemoteException { Exception e = new Exception(EXCEPTION_MSG, new Exception(EXCEPTION_CAUSE)); - throw YarnRemoteExceptionFactoryProvider.getYarnRemoteExceptionFactory(null).createYarnRemoteException(e); + throw YarnRemoteExceptionFactoryProvider + .getYarnRemoteExceptionFactory(null).createYarnRemoteException(e); } } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRpcFactoryProvider.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRpcFactoryProvider.java index b167a51164..84e1968acb 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRpcFactoryProvider.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRpcFactoryProvider.java @@ -21,6 +21,7 @@ import junit.framework.Assert; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RpcClientFactory; import org.apache.hadoop.yarn.factories.RpcServerFactory; import org.apache.hadoop.yarn.factories.impl.pb.RpcClientFactoryPBImpl; @@ -42,7 +43,7 @@ public void testFactoryProvider() { Assert.assertEquals(RpcClientFactoryPBImpl.class, clientFactory.getClass()); Assert.assertEquals(RpcServerFactoryPBImpl.class, serverFactory.getClass()); - conf.set(RpcFactoryProvider.RPC_SERIALIZER_KEY, "writable"); + conf.set(YarnConfiguration.IPC_SERIALIZER_TYPE, "writable"); try { clientFactory = RpcFactoryProvider.getClientFactory(conf); Assert.fail("Expected an exception - unknown serializer"); @@ -55,8 +56,8 @@ public void testFactoryProvider() { } conf = new Configuration(); - conf.set(RpcFactoryProvider.RPC_CLIENT_FACTORY_CLASS_KEY, "NonExistantClass"); - conf.set(RpcFactoryProvider.RPC_SERVER_FACTORY_CLASS_KEY, RpcServerFactoryPBImpl.class.getName()); + conf.set(YarnConfiguration.IPC_CLIENT_FACTORY, "NonExistantClass"); + conf.set(YarnConfiguration.IPC_SERVER_FACTORY, RpcServerFactoryPBImpl.class.getName()); try { clientFactory = RpcFactoryProvider.getClientFactory(conf); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java new file mode 100644 index 0000000000..6da84a96c2 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java @@ -0,0 +1,48 @@ +package org.apache.hadoop.yarn.api; + +import junit.framework.Assert; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.util.Records; +import org.junit.Test; + +public class TestApplicationAttemptId { + + @Test + public void testApplicationAttemptId() { + ApplicationAttemptId a1 = createAppAttemptId(10l, 1, 1); + ApplicationAttemptId a2 = createAppAttemptId(10l, 1, 2); + ApplicationAttemptId a3 = createAppAttemptId(10l, 2, 1); + ApplicationAttemptId a4 = createAppAttemptId(8l, 1, 4); + ApplicationAttemptId a5 = createAppAttemptId(10l, 1, 1); + + Assert.assertTrue(a1.equals(a5)); + Assert.assertFalse(a1.equals(a2)); + Assert.assertFalse(a1.equals(a3)); + Assert.assertFalse(a1.equals(a4)); + + Assert.assertTrue(a1.compareTo(a5) == 0); + Assert.assertTrue(a1.compareTo(a2) < 0); + Assert.assertTrue(a1.compareTo(a3) < 0); + Assert.assertTrue(a1.compareTo(a4) > 0); + + Assert.assertTrue(a1.hashCode() == a5.hashCode()); + Assert.assertFalse(a1.hashCode() == a2.hashCode()); + Assert.assertFalse(a1.hashCode() == a3.hashCode()); + Assert.assertFalse(a1.hashCode() == a4.hashCode()); + + } + + private ApplicationAttemptId createAppAttemptId(long clusterTimeStamp, + int id, int attemptId) { + ApplicationAttemptId appAttemptId = + Records.newRecord(ApplicationAttemptId.class); + ApplicationId appId = Records.newRecord(ApplicationId.class); + appId.setClusterTimestamp(clusterTimeStamp); + appId.setId(id); + appAttemptId.setApplicationId(appId); + appAttemptId.setAttemptId(attemptId); + return appAttemptId; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java new file mode 100644 index 0000000000..62d1dbc5f4 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java @@ -0,0 +1,37 @@ +package org.apache.hadoop.yarn.api; + +import junit.framework.Assert; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.util.Records; +import org.junit.Test; + +public class TestApplicationId { + + @Test + public void testApplicationId() { + ApplicationId a1 = createAppId(10l, 1); + ApplicationId a2 = createAppId(10l, 2); + ApplicationId a3 = createAppId(10l, 1); + ApplicationId a4 = createAppId(8l, 3); + + Assert.assertFalse(a1.equals(a2)); + Assert.assertFalse(a1.equals(a4)); + Assert.assertTrue(a1.equals(a3)); + + Assert.assertTrue(a1.compareTo(a2) < 0); + Assert.assertTrue(a1.compareTo(a3) == 0); + Assert.assertTrue(a1.compareTo(a4) > 0); + + Assert.assertTrue(a1.hashCode() == a3.hashCode()); + Assert.assertFalse(a1.hashCode() == a2.hashCode()); + Assert.assertFalse(a2.hashCode() == a4.hashCode()); + } + + private ApplicationId createAppId(long clusterTimeStamp, int id) { + ApplicationId appId = Records.newRecord(ApplicationId.class); + appId.setClusterTimestamp(clusterTimeStamp); + appId.setId(id); + return appId; + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java new file mode 100644 index 0000000000..8dccb34c2a --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java @@ -0,0 +1,63 @@ +package org.apache.hadoop.yarn.api; + +import junit.framework.Assert; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.util.Records; +import org.junit.Test; + +public class TestContainerId { + + @Test + public void testContainerId() { + ContainerId c1 = createContainerId(10l, 1, 1, 1); + ContainerId c2 = createContainerId(10l, 1, 1, 2); + ContainerId c3 = createContainerId(10l, 1, 1, 1); + ContainerId c4 = createContainerId(10l, 1, 3, 1); + ContainerId c5 = createContainerId(8l, 1, 3, 1); + + Assert.assertTrue(c1.equals(c3)); + Assert.assertFalse(c1.equals(c2)); + Assert.assertFalse(c1.equals(c4)); + Assert.assertFalse(c1.equals(c5)); + + Assert.assertTrue(c1.compareTo(c3) == 0); + Assert.assertTrue(c1.compareTo(c2) < 0); + Assert.assertTrue(c1.compareTo(c4) < 0); + Assert.assertTrue(c1.compareTo(c5) > 0); + + Assert.assertTrue(c1.hashCode() == c3.hashCode()); + Assert.assertFalse(c1.hashCode() == c2.hashCode()); + Assert.assertFalse(c1.hashCode() == c4.hashCode()); + Assert.assertFalse(c1.hashCode() == c5.hashCode()); + } + + private ContainerId createContainerId(long clusterTimestamp, int appIdInt, + int appAttemptIdInt, int containerIdInt) { + ApplicationId appId = createAppId(clusterTimestamp, appIdInt); + ApplicationAttemptId appAttemptId = + createAppAttemptId(appId, appAttemptIdInt); + ContainerId containerId = Records.newRecord(ContainerId.class); + containerId.setApplicationAttemptId(appAttemptId); + containerId.setId(containerIdInt); + return containerId; + } + + private ApplicationId createAppId(long clusterTimeStamp, int id) { + ApplicationId appId = Records.newRecord(ApplicationId.class); + appId.setClusterTimestamp(clusterTimeStamp); + appId.setId(id); + return appId; + } + + private ApplicationAttemptId createAppAttemptId(ApplicationId appId, + int attemptId) { + ApplicationAttemptId appAttemptId = + Records.newRecord(ApplicationAttemptId.class); + appAttemptId.setApplicationId(appId); + appAttemptId.setAttemptId(attemptId); + return appAttemptId; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/NodeHealthCheckerService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/NodeHealthCheckerService.java index 908fc2118a..b02e8b13ad 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/NodeHealthCheckerService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/NodeHealthCheckerService.java @@ -32,6 +32,7 @@ import org.apache.hadoop.util.Shell.ShellCommandExecutor; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.service.AbstractService; /** @@ -62,28 +63,9 @@ public class NodeHealthCheckerService extends AbstractService { /** Pattern used for searching in the output of the node health script */ static private final String ERROR_PATTERN = "ERROR"; - /* Configuration keys */ - public static final String HEALTH_CHECK_SCRIPT_PROPERTY = - "yarn.server.nodemanager.healthchecker.script.path"; - - public static final String HEALTH_CHECK_INTERVAL_PROPERTY = - "yarn.server.nodemanager.healthchecker.interval"; - - public static final String HEALTH_CHECK_FAILURE_INTERVAL_PROPERTY = - "yarn.server.nodemanager.healthchecker.script.timeout"; - - public static final String HEALTH_CHECK_SCRIPT_ARGUMENTS_PROPERTY = - "yarn.server.nodemanager.healthchecker.script.args"; - - /* end of configuration keys */ /** Time out error message */ static final String NODE_HEALTH_SCRIPT_TIMED_OUT_MSG = "Node health script timed out"; - /** Default frequency of running node health script */ - private static final long DEFAULT_HEALTH_CHECK_INTERVAL = 10 * 60 * 1000; - /** Default script time out period */ - private static final long DEFAULT_HEALTH_SCRIPT_FAILURE_INTERVAL = 2 * DEFAULT_HEALTH_CHECK_INTERVAL; - private boolean isHealthy; private String healthReport; @@ -224,13 +206,13 @@ public NodeHealthCheckerService(Configuration conf) { public void init(Configuration conf) { this.conf = conf; this.nodeHealthScript = - conf.get(HEALTH_CHECK_SCRIPT_PROPERTY); - this.intervalTime = conf.getLong(HEALTH_CHECK_INTERVAL_PROPERTY, - DEFAULT_HEALTH_CHECK_INTERVAL); + conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH); + this.intervalTime = conf.getLong(YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS, + YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS); this.scriptTimeout = conf.getLong( - HEALTH_CHECK_FAILURE_INTERVAL_PROPERTY, - DEFAULT_HEALTH_SCRIPT_FAILURE_INTERVAL); - String[] args = conf.getStrings(HEALTH_CHECK_SCRIPT_ARGUMENTS_PROPERTY, + YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS, + YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS); + String[] args = conf.getStrings(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_OPTS, new String[] {}); timer = new NodeHealthMonitorExecutor(args); } @@ -340,7 +322,7 @@ private synchronized void setLastReportedTime(long lastReportedTime) { */ public static boolean shouldRun(Configuration conf) { String nodeHealthScript = - conf.get(HEALTH_CHECK_SCRIPT_PROPERTY); + conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH); if (nodeHealthScript == null || nodeHealthScript.trim().isEmpty()) { return false; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/RMNMSecurityInfoClass.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/RMNMSecurityInfoClass.java index 39861d471b..1b23b77322 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/RMNMSecurityInfoClass.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/RMNMSecurityInfoClass.java @@ -43,12 +43,12 @@ public Class annotationType() { @Override public String serverPrincipal() { - return YarnConfiguration.RM_SERVER_PRINCIPAL_KEY; + return YarnConfiguration.RM_PRINCIPAL; } @Override public String clientPrincipal() { - return YarnServerConfig.NM_SERVER_PRINCIPAL_KEY; + return YarnConfiguration.NM_PRINCIPAL; } }; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/YarnServerConfig.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/YarnServerConfig.java deleted file mode 100644 index 2d7b561165..0000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/YarnServerConfig.java +++ /dev/null @@ -1,30 +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; - -import org.apache.hadoop.yarn.conf.YarnConfiguration; - -public class YarnServerConfig { - public static final String NM_SERVER_PRINCIPAL_KEY = - "yarn.nodemanager.principal"; - public static final String RESOURCETRACKER_ADDRESS = - YarnConfiguration.RM_PREFIX + "resourcetracker.address"; - public static final String DEFAULT_RESOURCETRACKER_BIND_ADDRESS = - "0.0.0.0:8020"; -} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java index 7965b5cc68..7822789eb0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java @@ -18,10 +18,8 @@ package org.apache.hadoop.yarn.server.api.records; import java.util.List; -import java.util.Map; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.NodeId; @@ -31,17 +29,13 @@ public interface NodeStatus { public abstract NodeId getNodeId(); public abstract int getResponseId(); - public abstract Map> getAllContainers(); - public abstract List getContainers(ApplicationId key); + public abstract List getContainersStatuses(); + public abstract void setContainersStatuses( + List containersStatuses); NodeHealthStatus getNodeHealthStatus(); void setNodeHealthStatus(NodeHealthStatus healthStatus); public abstract void setNodeId(NodeId nodeId); public abstract void setResponseId(int responseId); - - public abstract void addAllContainers(Map> containers); - public abstract void setContainers(ApplicationId key, List containers); - public abstract void removeContainers(ApplicationId key); - public abstract void clearContainers(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java index 2bb178f0d0..03d5e8cdbd 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java @@ -20,27 +20,19 @@ import java.util.ArrayList; -import java.util.HashMap; import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.ProtoBase; -import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NodeHealthStatusPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl; -import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeHealthStatusProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; -import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.ApplicationIdContainerListMapProto; -import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.ContainerListProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProtoOrBuilder; import org.apache.hadoop.yarn.server.api.records.NodeStatus; @@ -51,7 +43,7 @@ public class NodeStatusPBImpl extends ProtoBase implements Node boolean viaProto = false; private NodeId nodeId = null; - private Map> containers = null; + private List containers = null; private NodeHealthStatus nodeHealthStatus = null; public NodeStatusPBImpl() { @@ -99,7 +91,40 @@ private void maybeInitBuilder() { viaProto = false; } + private void addContainersToProto() { + maybeInitBuilder(); + builder.clearContainersStatuses(); + if (containers == null) + return; + Iterable iterable = new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + Iterator iter = containers.iterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public ContainerStatusProto next() { + return convertToProtoFormat(iter.next()); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + + } + }; + + } + }; + builder.addAllContainersStatuses(iterable); + } + @Override public int getResponseId() { NodeStatusProtoOrBuilder p = viaProto ? proto : builder; @@ -133,24 +158,17 @@ public void setNodeId(NodeId nodeId) { } @Override - public Map> getAllContainers() { + public List getContainersStatuses() { initContainers(); - HashMap> returnMap = new HashMap>( - this.containers.size()); - for (Entry> entry : this.containers.entrySet()) { - returnMap.put(convertFromProtoFormat(entry.getKey()), entry.getValue()); - } - return returnMap; + return this.containers; } @Override - public List getContainers(ApplicationId applicationId) { - initContainers(); - ApplicationIdProto applicationIdProto = convertToProtoFormat(applicationId); - if (this.containers.get(applicationIdProto) == null) { - this.containers.put(applicationIdProto, new ArrayList()); + public void setContainersStatuses(List containers) { + if (containers == null) { + builder.clearContainersStatuses(); } - return this.containers.get(applicationIdProto); + this.containers = containers; } private void initContainers() { @@ -158,59 +176,15 @@ private void initContainers() { return; } NodeStatusProtoOrBuilder p = viaProto ? proto : builder; - List list = p.getContainersList(); - this.containers = new HashMap>(); + List list = p.getContainersStatusesList(); + this.containers = new ArrayList(); - for (ApplicationIdContainerListMapProto c : list) { - this.containers.put(c.getApplicationId(), convertFromProtoFormat(c.getValue())); + for (ContainerStatusProto c : list) { + this.containers.add(convertFromProtoFormat(c)); } } - @Override - public void addAllContainers(final Map> containers) { - if (containers == null) - return; - initContainers(); - for (Entry> entry : containers.entrySet()) { - this.containers.put(convertToProtoFormat(entry.getKey()), entry.getValue()); - } - } - - private void addContainersToProto() { - maybeInitBuilder(); - builder.clearContainers(); - viaProto = false; - Iterable iterable = new Iterable() { - - @Override - public Iterator iterator() { - return new Iterator() { - - Iterator keyIter = containers.keySet().iterator(); - @Override - public boolean hasNext() { - return keyIter.hasNext(); - } - - @Override - public ApplicationIdContainerListMapProto next() { - ApplicationIdProto applicationIdProto = keyIter.next(); - return ApplicationIdContainerListMapProto.newBuilder().setApplicationId(applicationIdProto).setValue(convertToProtoFormat(containers.get(applicationIdProto))).build(); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - - }; - } - - }; - builder.addAllContainers(iterable); - } - @Override public NodeHealthStatus getNodeHealthStatus() { NodeStatusProtoOrBuilder p = viaProto ? proto : builder; @@ -233,66 +207,6 @@ public void setNodeHealthStatus(NodeHealthStatus healthStatus) { this.nodeHealthStatus = healthStatus; } - /* - * - * @Override - public String getApplicationName() { - ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder; - if (!p.hasApplicationName()) { - return null; - } - return (p.getApplicationName()); - } - - @Override - public void setApplicationName(String applicationName) { - maybeInitBuilder(); - if (applicationName == null) { - builder.clearApplicationName(); - return; - } - builder.setApplicationName((applicationName)); - } - */ - - private ContainerListProto convertToProtoFormat(List src) { - ContainerListProto.Builder ret = ContainerListProto.newBuilder(); - for (Container c : src) { - ret.addContainer(((ContainerPBImpl)c).getProto()); - } - return ret.build(); - } - - private List convertFromProtoFormat(ContainerListProto src) { - List ret = new ArrayList(); - for (ContainerProto c : src.getContainerList()) { - ret.add(convertFromProtoFormat(c)); - } - return ret; - } - - private Container convertFromProtoFormat(ContainerProto src) { - return new ContainerPBImpl(src); - } - - @Override - public void setContainers(ApplicationId applicationId, List containers) { - initContainers(); - this.containers.put(convertToProtoFormat(applicationId), containers); - } - - @Override - public void removeContainers(ApplicationId applicationId) { - initContainers(); - this.containers.remove(convertToProtoFormat(applicationId)); - } - - @Override - public void clearContainers() { - initContainers(); - this.containers.clear(); - } - private NodeIdProto convertToProtoFormat(NodeId nodeId) { return ((NodeIdPBImpl)nodeId).getProto(); } @@ -301,14 +215,6 @@ private NodeId convertFromProtoFormat(NodeIdProto proto) { return new NodeIdPBImpl(proto); } - private ApplicationIdProto convertToProtoFormat(ApplicationId applicationId) { - return ((ApplicationIdPBImpl)applicationId).getProto(); - } - - private ApplicationId convertFromProtoFormat(ApplicationIdProto proto) { - return new ApplicationIdPBImpl(proto); - } - private NodeHealthStatusProto convertToProtoFormat( NodeHealthStatus healthStatus) { return ((NodeHealthStatusPBImpl) healthStatus).getProto(); @@ -317,4 +223,12 @@ private NodeHealthStatusProto convertToProtoFormat( private NodeHealthStatus convertFromProtoFormat(NodeHealthStatusProto proto) { return new NodeHealthStatusPBImpl(proto); } + + private ContainerStatusPBImpl convertFromProtoFormat(ContainerStatusProto c) { + return new ContainerStatusPBImpl(c); + } + + private ContainerStatusProto convertToProtoFormat(ContainerStatus c) { + return ((ContainerStatusPBImpl)c).getProto(); + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto index 8d0d2383af..5198c5743f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto @@ -26,7 +26,7 @@ import "yarn_protos.proto"; message NodeStatusProto { optional NodeIdProto node_id = 1; optional int32 response_id = 2; - repeated ApplicationIdContainerListMapProto containers = 3; + repeated ContainerStatusProto containersStatuses = 3; optional NodeHealthStatusProto nodeHealthStatus = 4; } @@ -41,12 +41,3 @@ message HeartbeatResponseProto { repeated ApplicationIdProto applications_to_cleanup = 4; } -message ContainerListProto { - repeated ContainerProto container = 1; -} - -message ApplicationIdContainerListMapProto { - optional ApplicationIdProto application_id = 1; - optional ContainerListProto value = 2; -} - diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/resources/yarn-default.xml b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/resources/yarn-default.xml index aed8639130..05e979da4c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/resources/yarn-default.xml +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/resources/yarn-default.xml @@ -1,212 +1,367 @@ + + + + Factory to create client IPC classes. + yarn.ipc.client.factory.class + - + + Type of serialization to use. + yarn.ipc.serializer.type + protocolbuffers + + + + Factory to create server IPC classes. + yarn.ipc.server.factory.class + + + + Factory to create IPC exceptions. + yarn.ipc.exception.factory.class + + + + Factory to create serializeable records. + yarn.ipc.record.factory.class + + + + RPC class implementation + yarn.ipc.rpc.class + org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC + + + + + The address of the applications manager interface in the RM. + yarn.resourcemanager.address + 0.0.0.0:8040 + + + + The number of threads used to handle applications manager requests. + yarn.resourcemanager.client.thread-count + 10 + + + + The expiry interval for application master reporting. + yarn.resourcemanager.am.liveness-monitor.expiry-interval-ms + 600000 + + + + The Kerberos principal for the resource manager. yarn.resourcemanager.principal rm/sightbusy-lx@LOCALHOST - - yarn.nodemanager.principal - nm/sightbusy-lx@LOCALHOST - - - - - - yarn.server.resourcemanager.address - 0.0.0.0:8020 - - - - yarn.server.resourcemanager.resourcetracker.address - 0.0.0.0:8025 - - - - yarn.server.resourcemanager.scheduler.address + The address of the scheduler interface. + yarn.resourcemanager.scheduler.address 0.0.0.0:8030 - - yarn.server.resourcemanager.admin.address + + Number of threads to handle scheduler interface. + yarn.resourcemanager.scheduler.client.thread-count + 10 + + + + The address of the RM web application. + yarn.resourcemanager.webapp.address + 0.0.0.0:8088 + + + + yarn.resourcemanager.resource-tracker.address + 0.0.0.0:8025 + + + + Are RM acls enabled. + yarn.resourcemanager.acl.enable + false + + + + ACL of who can be admin of RM. + yarn.resourcemanager.admin.acl + * + + + + The address of the RM admin interface. + yarn.resourcemanager.admin.address 0.0.0.0:8141 - yarn.server.resourcemanager.application.max.retries + Number of threads used to handle RM admin interface. + yarn.resourcemanager.admin.client.thread-count 1 - The number of times an application will be retried in case - of AM failure. + - yarn.server.resourcemanager.keytab + How often should the RM check that the AM is still alive. + yarn.resourcemanager.amliveliness-monitor.interval-ms + 1000 + + + + The maximum number of application master retries. + yarn.resourcemanager.am.max-retries + 1 + + + + How often to check that containers are still alive. + yarn.resourcemanager.container.liveness-monitor.interval-ms + 600000 + + + + The keytab for the resource manager. + yarn.resourcemanager.keytab /etc/krb5.keytab - yarn.server.resourcemanager.expire.applications.completed.max - 10000 - the maximum number of completed applications the RM - keeps in memory - + How long to wait until a node manager is considered dead. + yarn.resourcemanager.nm.liveness-monitor.expiry-interval-ms + 600000 - + + How often to check that node managers are still alive. + yarn.resourcemanager.nm.liveness-monitor.interval-ms + 1000 + - yarn.server.nodemanager.local-dir + Path to file with nodes to include. + yarn.resourcemanager.nodes.include-path + + + + + Path to file with nodes to exclude. + yarn.resourcemanager.nodes.exclude-path + + + + + Number of threads to handle resource tracker calls. + yarn.resourcemanager.resource-tracker.client.thread-count + 10 + + + + The class to use as the resource scheduler. + yarn.resourcemanager.scheduler.class + + + + The class to use as the persistent store. + yarn.resourcemanager.store.class + + + + The address of the zookeeper instance to use with ZK store. + yarn.resourcemanager.zookeeper-store.address + + + + The zookeeper session timeout for the zookeeper store. + yarn.resourcemanager.zookeeper-store.session.timeout-ms + 60000 + + + + The maximum number of completed applications RM keeps. + yarn.resourcemanager.max-completed-applications + 10000 + + + + + address of node manager IPC. + yarn.nodemanager.address + 0.0.0.0:45454 + + + + who will execute(launch) the containers. + yarn.nodemanager.container-executor.class + org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor + + + + + Number of threads container manager uses. + yarn.nodemanager.container-manager.thread-count + 5 + + + + Number of threads used in cleanup. + yarn.nodemanager.delete.thread-count + 4 + + + + Heartbeat interval to RM + yarn.nodemanager.heartbeat.interval-ms + 1000 + + + + Keytab for NM. + yarn.nodemanager.keytab + /etc/krb5.keytab + + + + List of directories to store localized files in. + yarn.nodemanager.local-dirs /tmp/nm-local-dir - yarn.server.nodemanager.log.dir + Address where the localizer IPC is. + yarn.nodemanager.localizer.address + 0.0.0.0:4344 + + + + Interval in between cache cleanups. + yarn.nodemanager.localizer.cache.cleanup.interval-ms + 600000 + + + + Target size of localizer cache in MB, per local directory. + yarn.nodemanager.localizer.cache.target-size-mb + 10240 + + + + Number of threads to handle localization requests. + yarn.nodemanager.localizer.client.thread-count + 5 + + + + Number of threads to use for localization fetching. + yarn.nodemanager.localizer.fetch.thread-count + 4 + + + + Where to store container logs. + yarn.nodemanager.log-dirs /tmp/logs - yarn.apps.stagingDir - /tmp/hadoop-yarn/${user.name}/staging - - - - yarn.apps.history.stagingDir - /tmp/hadoop-yarn/${user.name}/staging - - - - yarn.server.nodemanager.keytab - /etc/krb5.keytab + Where to aggregate logs to. + yarn.nodemanager.remote-app-log-dir + /tmp/logs - yarn.server.nodemanager.container-executor.class - org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor - - - - NM_HOSTS0.0.0.0:45454 - - - yarn.server.nodemanager.address - 0.0.0.0:45454 - - - - - yarn.server.nodemanager.healthchecker.script.path - - Location of the node's health-check script on the local - file-system. - + Amount of memory in GB that can be allocated for containers. + yarn.nodemanager.resource.memory-gb + 8 - yarn.server.nodemanager.healthchecker.interval - 600000 - Frequency of the health-check run by the NodeManager - + NM Webapp address. + yarn.nodemanager.webapp.address + 0.0.0.0:9999 - yarn.server.nodemanager.healthchecker.script.timeout - 1200000 - Timeout for the health-check run by the NodeManager - - - - - yarn.server.nodemanager.healthchecker.script.args - - Arguments to be passed to the health-check script run - by the NodeManager - - - - yarn.server.nodemanager.healthchecker.script.path - - Location of the node's health-check script on the local - file-system. - - - - - yarn.server.nodemanager.healthchecker.interval - 600000 - Frequency of the health-check run by the NodeManager - - - - - yarn.server.nodemanager.healthchecker.script.timeout - 1200000 - Timeout for the health-check run by the NodeManager - - - - - yarn.server.nodemanager.healthchecker.script.args - - Arguments to be passed to the health-check script run - by the NodeManager - - - - - - - yarn.server.nodemanager.containers-monitor.monitoring-interval + How often to monitor containers. + yarn.nodemanager.container-monitor.interval-ms 3000 - yarn.server.nodemanager.containers-monitor.resourcecalculatorplugin + Class that calculates containers current resource utilization. + yarn.nodemanager.container-monitor.resource-calculator.class + + + + Amount of physical ram to reserve for other applications, -1 disables. + yarn.nodemanager.reserved.memory-mb + -1 + + + + Frequency of running node health script. + yarn.nodemanager.health-checker.interval-ms + 600000 + + + + Script time out period. + yarn.nodemanager.health-checker.script.timeout-ms + 1200000 + + + + The health check script to run. + yarn.nodemanager.health-checker.script.path - - yarn.server.nodemanager.reserved-physical-memory.mb - -1 - - - - - - - yarn.server.mapreduce-appmanager.attempt-listener.bindAddress - 0.0.0.0 + The arguments to pass to the health check script. + yarn.nodemanager.health-checker.script.opts + - yarn.server.mapreduce-appmanager.client-service.bindAddress - 0.0.0.0 + The path to the Linux container executor. + yarn.nodemanager.linux-container-executor.path + + T-file compression types used to compress aggregated logs. + yarn.nodemanager.log-aggregation.compression-type + none + + + + The kerberos principal for the node manager. + yarn.nodemanager.principal + nm/sightbusy-lx@LOCALHOST + + + + yarn.nodemanager.aux-services + + + + + + + yarn.nodemanager.aux-services.mapreduce.shuffle.class + org.apache.hadoop.mapred.ShuffleHandler + mapreduce.job.jar - - + - mapreduce.job.hdfs-servers - ${fs.default.name} - - - - nodemanager.auxiluary.services - - + mapreduce.job.hdfs-servers + ${fs.default.name} - - - diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/TestNodeHealthService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/TestNodeHealthService.java index 3f37d62718..54c3033ba2 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/TestNodeHealthService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/TestNodeHealthService.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.junit.After; @@ -66,11 +67,11 @@ public void tearDown() throws Exception { private Configuration getConfForNodeHealthScript() { Configuration conf = new Configuration(); - conf.set(NodeHealthCheckerService.HEALTH_CHECK_SCRIPT_PROPERTY, + conf.set(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH, nodeHealthscriptFile.getAbsolutePath()); - conf.setLong(NodeHealthCheckerService.HEALTH_CHECK_INTERVAL_PROPERTY, 500); + conf.setLong(YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS, 500); conf.setLong( - NodeHealthCheckerService.HEALTH_CHECK_FAILURE_INTERVAL_PROPERTY, 1000); + YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS, 1000); return conf; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java index 46e5f9672a..a7e82a2d41 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java @@ -33,6 +33,7 @@ import org.apache.hadoop.util.Shell.ExitCodeException; import org.apache.hadoop.util.Shell.ShellCommandExecutor; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent; @@ -96,9 +97,11 @@ public int launchContainer(Container container, // create container dirs on all disks String containerIdStr = ConverterUtils.toString(containerId); String appIdStr = - ConverterUtils.toString(container.getContainerID().getAppId()); + ConverterUtils.toString( + container.getContainerID().getApplicationAttemptId(). + getApplicationId()); String[] sLocalDirs = - getConf().getStrings(NMConfig.NM_LOCAL_DIR, NMConfig.DEFAULT_NM_LOCAL_DIR); + getConf().getStrings(YarnConfiguration.NM_LOCAL_DIRS, YarnConfiguration.DEFAULT_NM_LOCAL_DIRS); for (String sLocalDir : sLocalDirs) { Path usersdir = new Path(sLocalDir, ContainerLocalizer.USERCACHE); Path userdir = new Path(usersdir, userName); @@ -358,7 +361,7 @@ private void createAppLogDirs(String appId) throws IOException { String[] rootLogDirs = getConf() - .getStrings(NMConfig.NM_LOG_DIR, NMConfig.DEFAULT_NM_LOG_DIR); + .getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS); boolean appLogDirStatus = false; FsPermission appLogDirPerms = new FsPermission(LOGDIR_PERM); @@ -386,7 +389,7 @@ private void createContainerLogDirs(String appId, String containerId) throws IOException { String[] rootLogDirs = getConf() - .getStrings(NMConfig.NM_LOG_DIR, NMConfig.DEFAULT_NM_LOG_DIR); + .getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS); boolean containerLogDirStatus = false; FsPermission containerLogDirPerms = new FsPermission(LOGDIR_PERM); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java index 60206e0d1b..346e79e7a7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java @@ -28,21 +28,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.UnsupportedFileSystemException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.service.AbstractService; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.*; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; public class DeletionService extends AbstractService { - static final Log LOG = LogFactory.getLog(DeletionService.class); - /** Delay before deleting resource to ease debugging of NM issues */ - static final String DEBUG_DELAY_SEC = - NMConfig.NM_PREFIX + "debug.delete.delay"; - private int debugDelay; private final ContainerExecutor exec; private ScheduledThreadPoolExecutor sched; @@ -79,10 +73,10 @@ public void delete(String user, Path subDir, Path... baseDirs) { public void init(Configuration conf) { if (conf != null) { sched = new ScheduledThreadPoolExecutor( - conf.getInt(NM_MAX_DELETE_THREADS, DEFAULT_MAX_DELETE_THREADS)); - debugDelay = conf.getInt(DEBUG_DELAY_SEC, 0); + conf.getInt(YarnConfiguration.NM_DELETE_THREAD_COUNT, YarnConfiguration.DEFAULT_NM_DELETE_THREAD_COUNT)); + debugDelay = conf.getInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 0); } else { - sched = new ScheduledThreadPoolExecutor(DEFAULT_MAX_DELETE_THREADS); + sched = new ScheduledThreadPoolExecutor(YarnConfiguration.DEFAULT_NM_DELETE_THREAD_COUNT); } sched.setKeepAliveTime(60L, SECONDS); super.init(conf); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java index 56361cbaa5..97721f72a3 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java @@ -33,6 +33,7 @@ import org.apache.hadoop.util.Shell.ShellCommandExecutor; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer; @@ -44,8 +45,6 @@ public class LinuxContainerExecutor extends ContainerExecutor { .getLog(LinuxContainerExecutor.class); private String containerExecutorExe; - protected static final String CONTAINER_EXECUTOR_EXEC_KEY = - NMConfig.NM_PREFIX + "linux-container-executor.path"; @Override public void setConf(Configuration conf) { @@ -98,7 +97,7 @@ protected String getContainerExecutorExecutablePath(Configuration conf) { new File(hadoopBin, "container-executor").getAbsolutePath(); return null == conf ? defaultPath - : conf.get(CONTAINER_EXECUTOR_EXEC_KEY, defaultPath); + : conf.get(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH, defaultPath); } @Override diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NMConfig.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NMConfig.java deleted file mode 100644 index 77bb1a8733..0000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NMConfig.java +++ /dev/null @@ -1,103 +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.nodemanager; - -/** this class stores all the configuration constant keys - * for the nodemanager. All the configuration key variables - * that are going to be used in the nodemanager should be - * stored here. This allows us to see all the configuration - * parameters at one place. - */ -public class NMConfig { - public static final String NM_PREFIX = "yarn.server.nodemanager."; - - public static final String DEFAULT_NM_BIND_ADDRESS = "0.0.0.0:45454"; - - /** host:port address to which to bind to **/ - public static final String NM_BIND_ADDRESS = NM_PREFIX + "address"; - - public static final String DEFAULT_NM_HTTP_BIND_ADDRESS = "0.0.0.0:9999"; - - /** host:port address to which webserver has to bind to **/ - public static final String NM_HTTP_BIND_ADDRESS = NM_PREFIX + "http-address"; - - public static final String DEFAULT_NM_LOCALIZER_BIND_ADDRESS = "0.0.0.0:4344"; - - public static final String NM_LOCALIZER_BIND_ADDRESS = - NM_PREFIX + "localizer.address"; - - public static final String NM_KEYTAB = NM_PREFIX + "keytab"; - - public static final String NM_CONTAINER_EXECUTOR_CLASS = NM_PREFIX - + "container-executor.class"; - - public static final String NM_LOCAL_DIR = NM_PREFIX + "local-dir"; - - public static final String DEFAULT_NM_LOCAL_DIR = "/tmp/nm-local-dir"; - - public static final String NM_LOG_DIR = NM_PREFIX + "log.dir"; // TODO: Rename - - public static final String DEFAULT_NM_LOG_DIR = "/tmp/logs"; - - public static final String REMOTE_USER_LOG_DIR = NM_PREFIX - + "remote-app-log-dir"; - - public static final String DEFAULT_REMOTE_APP_LOG_DIR = "/tmp/logs"; - - public static final int DEFAULT_NM_VMEM_GB = 8; - - public static final String NM_VMEM_GB = NM_PREFIX + "resource.memory.gb"; - - // TODO: Should this instead be dictated by RM? - public static final String HEARTBEAT_INTERVAL = NM_PREFIX - + "heartbeat-interval"; - - public static final int DEFAULT_HEARTBEAT_INTERVAL = 1000; - - public static final String NM_MAX_DELETE_THREADS = NM_PREFIX + - "max.delete.threads"; - - public static final int DEFAULT_MAX_DELETE_THREADS = 4; - - public static final String NM_MAX_PUBLIC_FETCH_THREADS = NM_PREFIX + - "max.public.fetch.threads"; - - public static final int DEFAULT_MAX_PUBLIC_FETCH_THREADS = 4; - - public static final String NM_LOCALIZATION_THREADS = - NM_PREFIX + "localiation.threads"; - - public static final int DEFAULT_NM_LOCALIZATION_THREADS = 5; - - public static final String NM_CONTAINER_MGR_THREADS = - NM_PREFIX + "container.manager.threads"; - - public static final int DEFAULT_NM_CONTAINER_MGR_THREADS = 5; - - public static final String NM_TARGET_CACHE_MB = - NM_PREFIX + "target.cache.size"; - - public static final long DEFAULT_NM_TARGET_CACHE_MB = 10 * 1024; - - public static final String NM_CACHE_CLEANUP_MS = - NM_PREFIX + "target.cache.cleanup.period.ms"; - - public static final long DEFAULT_NM_CACHE_CLEANUP_MS = 10 * 60 * 1000; - -} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java index b826042278..b5249440c7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java @@ -18,9 +18,6 @@ package org.apache.hadoop.yarn.server.nodemanager; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_CONTAINER_EXECUTOR_CLASS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_KEYTAB; - import java.io.IOException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -32,6 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.YarnException; @@ -42,27 +40,29 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.server.YarnServerConfig; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.service.CompositeService; import org.apache.hadoop.yarn.service.Service; public class NodeManager extends CompositeService { private static final Log LOG = LogFactory.getLog(NodeManager.class); protected final NodeManagerMetrics metrics = NodeManagerMetrics.create(); + protected ContainerTokenSecretManager containerTokenSecretManager; public NodeManager() { super(NodeManager.class.getName()); } protected NodeStatusUpdater createNodeStatusUpdater(Context context, - Dispatcher dispatcher, NodeHealthCheckerService healthChecker) { + Dispatcher dispatcher, NodeHealthCheckerService healthChecker, + ContainerTokenSecretManager containerTokenSecretManager) { return new NodeStatusUpdaterImpl(context, dispatcher, healthChecker, - metrics); + metrics, containerTokenSecretManager); } protected NodeResourceMonitor createNodeResourceMonitor() { @@ -71,9 +71,10 @@ protected NodeResourceMonitor createNodeResourceMonitor() { protected ContainerManagerImpl createContainerManager(Context context, ContainerExecutor exec, DeletionService del, - NodeStatusUpdater nodeStatusUpdater) { + NodeStatusUpdater nodeStatusUpdater, ContainerTokenSecretManager + containerTokenSecretManager) { return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater, - metrics); + metrics, containerTokenSecretManager); } protected WebServer createWebServer(Context nmContext, @@ -82,8 +83,8 @@ protected WebServer createWebServer(Context nmContext, } protected void doSecureLogin() throws IOException { - SecurityUtil.login(getConfig(), NM_KEYTAB, - YarnServerConfig.NM_SERVER_PRINCIPAL_KEY); + SecurityUtil.login(getConfig(), YarnConfiguration.NM_KEYTAB, + YarnConfiguration.NM_PRINCIPAL); } @Override @@ -91,8 +92,15 @@ public void init(Configuration conf) { Context context = new NMContext(); + // Create the secretManager if need be. + if (UserGroupInformation.isSecurityEnabled()) { + LOG.info("Security is enabled on NodeManager. " + + "Creating ContainerTokenSecretManager"); + this.containerTokenSecretManager = new ContainerTokenSecretManager(); + } + ContainerExecutor exec = ReflectionUtils.newInstance( - conf.getClass(NM_CONTAINER_EXECUTOR_CLASS, + conf.getClass(YarnConfiguration.NM_CONTAINER_EXECUTOR, DefaultContainerExecutor.class, ContainerExecutor.class), conf); DeletionService del = new DeletionService(exec); addService(del); @@ -106,18 +114,16 @@ public void init(Configuration conf) { addService(healthChecker); } - // StatusUpdater should be added first so that it can start first. Once it - // contacts RM, does registration and gets tokens, then only - // ContainerManager can start. NodeStatusUpdater nodeStatusUpdater = - createNodeStatusUpdater(context, dispatcher, healthChecker); - addService(nodeStatusUpdater); + createNodeStatusUpdater(context, dispatcher, healthChecker, + this.containerTokenSecretManager); NodeResourceMonitor nodeResourceMonitor = createNodeResourceMonitor(); addService(nodeResourceMonitor); ContainerManagerImpl containerManager = - createContainerManager(context, exec, del, nodeStatusUpdater); + createContainerManager(context, exec, del, nodeStatusUpdater, + this.containerTokenSecretManager); addService(containerManager); Service webServer = @@ -136,6 +142,10 @@ public void run() { DefaultMetricsSystem.initialize("NodeManager"); + // StatusUpdater should be added last so that it get started last + // so that we make sure everything is up before registering with RM. + addService(nodeStatusUpdater); + super.init(conf); // TODO add local dirs to del } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java index 3ae0121549..641e74b801 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java @@ -30,13 +30,13 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.NodeHealthCheckerService; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; @@ -47,7 +47,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.server.RMNMSecurityInfoClass; -import org.apache.hadoop.yarn.server.YarnServerConfig; 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.RegisterNodeManagerRequest; @@ -56,6 +55,7 @@ import org.apache.hadoop.yarn.server.api.records.RegistrationResponse; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.util.Records; @@ -69,12 +69,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements private final Context context; private final Dispatcher dispatcher; + private ContainerTokenSecretManager containerTokenSecretManager; private long heartBeatInterval; private ResourceTracker resourceTracker; private String rmAddress; private Resource totalResource; private String containerManagerBindAddress; - private String nodeHttpAddress; private String hostName; private int containerManagerPort; private int httpPort; @@ -87,23 +87,25 @@ public class NodeStatusUpdaterImpl extends AbstractService implements private final NodeManagerMetrics metrics; public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher, - NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) { + NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics, + ContainerTokenSecretManager containerTokenSecretManager) { super(NodeStatusUpdaterImpl.class.getName()); this.healthChecker = healthChecker; this.context = context; this.dispatcher = dispatcher; this.metrics = metrics; + this.containerTokenSecretManager = containerTokenSecretManager; } @Override public synchronized void init(Configuration conf) { this.rmAddress = - conf.get(YarnServerConfig.RESOURCETRACKER_ADDRESS, - YarnServerConfig.DEFAULT_RESOURCETRACKER_BIND_ADDRESS); + conf.get(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS, + YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS); this.heartBeatInterval = - conf.getLong(NMConfig.HEARTBEAT_INTERVAL, - NMConfig.DEFAULT_HEARTBEAT_INTERVAL); - int memory = conf.getInt(NMConfig.NM_VMEM_GB, NMConfig.DEFAULT_NM_VMEM_GB); + conf.getLong(YarnConfiguration.NM_TO_RM_HEARTBEAT_INTERVAL_MS, + YarnConfiguration.DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS); + int memory = conf.getInt(YarnConfiguration.NM_VMEM_GB, YarnConfiguration.DEFAULT_NM_VMEM_GB); this.totalResource = recordFactory.newRecordInstance(Resource.class); this.totalResource.setMemory(memory * 1024); metrics.addResource(totalResource); @@ -113,13 +115,13 @@ public synchronized void init(Configuration conf) { @Override public void start() { String cmBindAddressStr = - getConfig().get(NMConfig.NM_BIND_ADDRESS, - NMConfig.DEFAULT_NM_BIND_ADDRESS); + getConfig().get(YarnConfiguration.NM_ADDRESS, + YarnConfiguration.DEFAULT_NM_ADDRESS); InetSocketAddress cmBindAddress = NetUtils.createSocketAddr(cmBindAddressStr); String httpBindAddressStr = - getConfig().get(NMConfig.NM_HTTP_BIND_ADDRESS, - NMConfig.DEFAULT_NM_HTTP_BIND_ADDRESS); + getConfig().get(YarnConfiguration.NM_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_NM_WEBAPP_ADDRESS); InetSocketAddress httpBindAddress = NetUtils.createSocketAddr(httpBindAddressStr); try { @@ -128,7 +130,6 @@ public void start() { this.httpPort = httpBindAddress.getPort(); this.containerManagerBindAddress = this.hostName + ":" + this.containerManagerPort; - this.nodeHttpAddress = this.hostName + ":" + this.httpPort; LOG.info("Configured ContainerManager Address is " + this.containerManagerBindAddress); // Registration has to be in start so that ContainerManager can get the @@ -176,8 +177,18 @@ private void registerWithRM() throws YarnRemoteException { this.secretKeyBytes = regResponse.getSecretKey().array(); } + // do this now so that its set before we start heartbeating to RM + if (UserGroupInformation.isSecurityEnabled()) { + LOG.info("Security enabled - updating secret keys now"); + // It is expected that status updater is started by this point and + // RM gives the shared secret in registration during StatusUpdater#start(). + this.containerTokenSecretManager.setSecretKey( + this.getContainerManagerBindAddress(), + this.getRMNMSharedSecret()); + } LOG.info("Registered with ResourceManager as " + this.containerManagerBindAddress + " with total resource of " + this.totalResource); + } @Override @@ -196,35 +207,28 @@ private NodeStatus getNodeStatus() { nodeStatus.setNodeId(this.nodeId); int numActiveContainers = 0; + List containersStatuses = new ArrayList(); for (Iterator> i = this.context.getContainers().entrySet().iterator(); i.hasNext();) { Entry e = i.next(); ContainerId containerId = e.getKey(); Container container = e.getValue(); - List applicationContainers = nodeStatus - .getContainers(container.getContainerID().getAppId()); - if (applicationContainers == null) { - applicationContainers = new ArrayList(); - nodeStatus.setContainers(container.getContainerID().getAppId(), - applicationContainers); - } - // Clone the container to send it to the RM - org.apache.hadoop.yarn.api.records.Container c = container.cloneAndGetContainer(); - c.setNodeId(this.nodeId); - c.setNodeHttpAddress(this.nodeHttpAddress); // TODO: don't set everytime. - applicationContainers.add(c); + org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus = + container.cloneAndGetContainerStatus(); + containersStatuses.add(containerStatus); ++numActiveContainers; - LOG.info("Sending out status for container: " + c); + LOG.info("Sending out status for container: " + containerStatus); - if (c.getState() == ContainerState.COMPLETE) { + if (containerStatus.getState() == ContainerState.COMPLETE) { // Remove i.remove(); LOG.info("Removed completed container " + containerId); } } + nodeStatus.setContainersStatuses(containersStatuses); LOG.debug(this.containerManagerBindAddress + " sending out status for " + numActiveContainers + " containers"); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java index 254ff2a671..ddfc1c5815 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java @@ -23,6 +23,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Map.Entry; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,6 +31,7 @@ import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.service.Service; @@ -40,9 +42,6 @@ public class AuxServices extends AbstractService private static final Log LOG = LogFactory.getLog(AuxServices.class); - public static final String AUX_SERVICES = "nodemanager.auxiluary.services"; - public static final String AUX_SERVICE_CLASS_FMT = - "nodemanager.aux.service.%s.class"; public final Map serviceMap; public final Map serviceMeta; @@ -72,16 +71,24 @@ Collection getServices() { * the the name of the service as defined in the configuration. */ public Map getMeta() { - return Collections.unmodifiableMap(serviceMeta); + Map metaClone = new HashMap( + serviceMeta.size()); + synchronized (serviceMeta) { + for (Entry entry : serviceMeta.entrySet()) { + metaClone.put(entry.getKey(), entry.getValue().duplicate()); + } + } + return metaClone; } @Override public void init(Configuration conf) { - Collection auxNames = conf.getStringCollection(AUX_SERVICES); + Collection auxNames = conf.getStringCollection( + YarnConfiguration.NM_AUX_SERVICES); for (final String sName : auxNames) { try { Class sClass = conf.getClass( - String.format(AUX_SERVICE_CLASS_FMT, sName), null, + String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, sName), null, AuxiliaryService.class); if (null == sClass) { throw new RuntimeException("No class defiend for " + sName); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java index 49bf3f0ef0..8e90552d86 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java @@ -18,8 +18,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_BIND_ADDRESS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_BIND_ADDRESS; import static org.apache.hadoop.yarn.service.Service.STATE.STARTED; import java.io.IOException; @@ -31,7 +29,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.io.DataInputByteBuffer; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.Credentials; @@ -67,7 +64,6 @@ import org.apache.hadoop.yarn.server.nodemanager.DeletionService; import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger; import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent; @@ -120,7 +116,8 @@ public class ContainerManagerImpl extends CompositeService implements public ContainerManagerImpl(Context context, ContainerExecutor exec, DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater, - NodeManagerMetrics metrics) { + NodeManagerMetrics metrics, ContainerTokenSecretManager + containerTokenSecretManager) { super(ContainerManagerImpl.class.getName()); this.context = context; dispatcher = new AsyncDispatcher(); @@ -135,12 +132,7 @@ public ContainerManagerImpl(Context context, ContainerExecutor exec, addService(containersLauncher); this.nodeStatusUpdater = nodeStatusUpdater; - // Create the secretManager if need be. - if (UserGroupInformation.isSecurityEnabled()) { - LOG.info("Security is enabled on NodeManager. " - + "Creating ContainerTokenSecretManager"); - this.containerTokenSecretManager = new ContainerTokenSecretManager(); - } + this.containerTokenSecretManager = containerTokenSecretManager; // Start configurable services auxiluaryServices = new AuxServices(); @@ -190,7 +182,7 @@ protected ContainersLauncher createContainersLauncher(Context context, @Override public void init(Configuration conf) { cmBindAddressStr = NetUtils.createSocketAddr( - conf.get(NM_BIND_ADDRESS, DEFAULT_NM_BIND_ADDRESS)); + conf.get(YarnConfiguration.NM_ADDRESS, YarnConfiguration.DEFAULT_NM_ADDRESS)); super.init(conf); } @@ -200,22 +192,14 @@ public void start() { // Enqueue user dirs in deletion context YarnRPC rpc = YarnRPC.create(getConfig()); - if (UserGroupInformation.isSecurityEnabled()) { - // This is fine as status updater is started before ContainerManager and - // RM gives the shared secret in registration during StatusUpdter#start() - // itself. - this.containerTokenSecretManager.setSecretKey( - this.nodeStatusUpdater.getContainerManagerBindAddress(), - this.nodeStatusUpdater.getRMNMSharedSecret()); - } Configuration cmConf = new Configuration(getConfig()); cmConf.setClass(YarnConfiguration.YARN_SECURITY_INFO, ContainerManagerSecurityInfo.class, SecurityInfo.class); server = rpc.getServer(ContainerManager.class, this, cmBindAddressStr, cmConf, this.containerTokenSecretManager, - cmConf.getInt(NMConfig.NM_CONTAINER_MGR_THREADS, - NMConfig.DEFAULT_NM_CONTAINER_MGR_THREADS)); + cmConf.getInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT, + YarnConfiguration.DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT)); LOG.info("ContainerManager started at " + cmBindAddressStr); server.start(); super.start(); @@ -266,7 +250,8 @@ public StartContainerResponse startContainer(StartContainerRequest request) Container container = new ContainerImpl(this.dispatcher, launchContext, credentials, metrics); ContainerId containerID = launchContext.getContainerId(); - ApplicationId applicationID = containerID.getAppId(); + ApplicationId applicationID = + containerID.getApplicationAttemptId().getApplicationId(); if (context.getContainers().putIfAbsent(containerID, container) != null) { NMAuditLogger.logFailure(launchContext.getUser(), AuditConstants.START_CONTAINER, "ContainerManagerImpl", @@ -311,10 +296,18 @@ public StopContainerResponse stopContainer(StopContainerRequest request) Container container = this.context.getContainers().get(containerID); if (container == null) { LOG.warn("Trying to stop unknown container " + containerID); - NMAuditLogger.logFailure(container.getUser(), + String userName; + try { + userName = UserGroupInformation.getCurrentUser().getUserName(); + } catch (IOException e) { + LOG.error("Error finding userName", e); + return response; + } + NMAuditLogger.logFailure(userName, AuditConstants.STOP_CONTAINER, "ContainerManagerImpl", "Trying to stop unknown container!", - containerID.getAppId(), containerID); + containerID.getApplicationAttemptId().getApplicationId(), + containerID); return response; // Return immediately. } dispatcher.getEventHandler().handle( @@ -326,7 +319,8 @@ public StopContainerResponse stopContainer(StopContainerRequest request) // should be the same or should be rejected by auth before here. NMAuditLogger.logSuccess(container.getUser(), AuditConstants.STOP_CONTAINER, "ContainerManageImpl", - containerID.getAppId(), containerID); + containerID.getApplicationAttemptId().getApplicationId(), + containerID); // TODO: Move this code to appropriate place once kill_container is // implemented. diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java index 288662155f..6b8007f3fe 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java @@ -25,7 +25,7 @@ public class ApplicationContainerFinishedEvent extends ApplicationEvent { public ApplicationContainerFinishedEvent( ContainerId containerID) { - super(containerID.getAppId(), + super(containerID.getApplicationAttemptId().getApplicationId(), ApplicationEventType.APPLICATION_CONTAINER_FINISHED); this.containerID = containerID; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationInitEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationInitEvent.java index 15c048a7fa..861d857950 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationInitEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationInitEvent.java @@ -25,7 +25,7 @@ public class ApplicationInitEvent extends ApplicationEvent { private final Container container; public ApplicationInitEvent(Container container) { - super(container.getContainerID().getAppId(), + super(container.getContainerID().getApplicationAttemptId().getApplicationId(), ApplicationEventType.INIT_APPLICATION); this.container = container; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java index 6bd29e851b..e5ba3f2993 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java @@ -40,8 +40,6 @@ public interface Container extends EventHandler { Map getLocalizedResources(); - org.apache.hadoop.yarn.api.records.Container cloneAndGetContainer(); - ContainerStatus cloneAndGetContainerStatus(); String toString(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java index 8a4439d32b..4e02c3aded 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java @@ -21,8 +21,10 @@ import java.net.URISyntaxException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; @@ -38,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.factories.RecordFactory; @@ -50,9 +53,8 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorContainerFinishedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainerStartMonitoringEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainerStopMonitoringEvent; @@ -72,7 +74,7 @@ public class ContainerImpl implements Container { private final Credentials credentials; private final NodeManagerMetrics metrics; private final ContainerLaunchContext launchContext; - private String exitCode = "NA"; + private int exitCode = YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS; private final StringBuilder diagnostics; private static final Log LOG = LogFactory.getLog(Container.class); @@ -81,6 +83,12 @@ public class ContainerImpl implements Container { new HashMap(); private final Map localizedResources = new HashMap(); + private final List publicRsrcs = + new ArrayList(); + private final List privateRsrcs = + new ArrayList(); + private final List appRsrcs = + new ArrayList(); public ContainerImpl(Dispatcher dispatcher, ContainerLaunchContext launchContext, Credentials creds, @@ -317,24 +325,6 @@ public ContainerState getContainerState() { } } - @Override - public - org.apache.hadoop.yarn.api.records.Container cloneAndGetContainer() { - this.readLock.lock(); - try { - org.apache.hadoop.yarn.api.records.Container c = - recordFactory.newRecordInstance( - org.apache.hadoop.yarn.api.records.Container.class); - c.setId(this.launchContext.getContainerId()); - c.setResource(this.launchContext.getResource()); - c.setState(getCurrentState()); - c.setContainerStatus(cloneAndGetContainerStatus()); - return c; - } finally { - this.readLock.unlock(); - } - } - @Override public ContainerLaunchContext getLaunchContext() { this.readLock.lock(); @@ -354,14 +344,14 @@ public ContainerStatus cloneAndGetContainerStatus() { containerStatus.setState(getCurrentState()); containerStatus.setContainerId(this.launchContext.getContainerId()); containerStatus.setDiagnostics(diagnostics.toString()); - containerStatus.setExitStatus(String.valueOf(exitCode)); + containerStatus.setExitStatus(exitCode); return containerStatus; } finally { this.readLock.unlock(); } } - @SuppressWarnings("fallthrough") + @SuppressWarnings({"fallthrough", "unchecked"}) private void finished() { switch (getContainerState()) { case EXITED_WITH_SUCCESS: @@ -369,7 +359,8 @@ private void finished() { metrics.completedContainer(); NMAuditLogger.logSuccess(getUser(), AuditConstants.FINISH_SUCCESS_CONTAINER, "ContainerImpl", - getContainerID().getAppId(), getContainerID()); + getContainerID().getApplicationAttemptId().getApplicationId(), + getContainerID()); break; case EXITED_WITH_FAILURE: metrics.endRunningContainer(); @@ -379,7 +370,8 @@ private void finished() { NMAuditLogger.logFailure(getUser(), AuditConstants.FINISH_FAILED_CONTAINER, "ContainerImpl", "Container failed with state: " + getContainerState(), - getContainerID().getAppId(), getContainerID()); + getContainerID().getApplicationAttemptId().getApplicationId(), + getContainerID()); break; case CONTAINER_CLEANEDUP_AFTER_KILL: metrics.endRunningContainer(); @@ -388,13 +380,15 @@ private void finished() { metrics.killedContainer(); NMAuditLogger.logSuccess(getUser(), AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl", - getContainerID().getAppId(), getContainerID()); + getContainerID().getApplicationAttemptId().getApplicationId(), + getContainerID()); } metrics.releaseContainer(getLaunchContext().getResource()); // Inform the application ContainerId containerID = getContainerID(); + @SuppressWarnings("rawtypes") EventHandler eventHandler = dispatcher.getEventHandler(); eventHandler.handle(new ApplicationContainerFinishedEvent(containerID)); // Remove the container from the resource-monitor @@ -404,6 +398,24 @@ private void finished() { containerID, exitCode)); } + @SuppressWarnings("unchecked") // dispatcher not typed + public void cleanup() { + Map> rsrc = + new HashMap>(); + if (!publicRsrcs.isEmpty()) { + rsrc.put(LocalResourceVisibility.PUBLIC, publicRsrcs); + } + if (!privateRsrcs.isEmpty()) { + rsrc.put(LocalResourceVisibility.PRIVATE, privateRsrcs); + } + if (!appRsrcs.isEmpty()) { + rsrc.put(LocalResourceVisibility.APPLICATION, appRsrcs); + } + dispatcher.getEventHandler().handle( + new ContainerLocalizationCleanupEvent(this, rsrc)); + } + static class ContainerTransition implements SingleArcTransition { @@ -424,27 +436,22 @@ public ContainerState transition(ContainerImpl container, container.metrics.initingContainer(); // Inform the AuxServices about the opaque serviceData - Map csd = ctxt.getAllServiceData(); + Map csd = ctxt.getServiceData(); if (csd != null) { // This can happen more than once per Application as each container may // have distinct service data for (Map.Entry service : csd.entrySet()) { container.dispatcher.getEventHandler().handle( new AuxServicesEvent(AuxServicesEventType.APPLICATION_INIT, - ctxt.getUser(), ctxt.getContainerId().getAppId(), + ctxt.getUser(), + ctxt.getContainerId().getApplicationAttemptId().getApplicationId(), service.getKey().toString(), service.getValue())); } } // Send requests for public, private resources - Map cntrRsrc = ctxt.getAllLocalResources(); + Map cntrRsrc = ctxt.getLocalResources(); if (!cntrRsrc.isEmpty()) { - ArrayList publicRsrc = - new ArrayList(); - ArrayList privateRsrc = - new ArrayList(); - ArrayList appRsrc = - new ArrayList(); try { for (Map.Entry rsrc : cntrRsrc.entrySet()) { try { @@ -453,13 +460,13 @@ public ContainerState transition(ContainerImpl container, container.pendingResources.put(req, rsrc.getKey()); switch (rsrc.getValue().getVisibility()) { case PUBLIC: - publicRsrc.add(req); + container.publicRsrcs.add(req); break; case PRIVATE: - privateRsrc.add(req); + container.privateRsrcs.add(req); break; case APPLICATION: - appRsrc.add(req); + container.appRsrcs.add(req); break; } } catch (URISyntaxException e) { @@ -471,27 +478,25 @@ public ContainerState transition(ContainerImpl container, } catch (URISyntaxException e) { // malformed resource; abort container launch LOG.warn("Failed to parse resource-request", e); - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationEvent( - LocalizationEventType.CLEANUP_CONTAINER_RESOURCES, container)); + container.cleanup(); container.metrics.endInitingContainer(); return ContainerState.LOCALIZATION_FAILED; } - if (!publicRsrc.isEmpty()) { - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationRequestEvent( - container, publicRsrc, LocalResourceVisibility.PUBLIC)); + Map> req = + new HashMap>(); + if (!container.publicRsrcs.isEmpty()) { + req.put(LocalResourceVisibility.PUBLIC, container.publicRsrcs); } - if (!privateRsrc.isEmpty()) { - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationRequestEvent( - container, privateRsrc, LocalResourceVisibility.PRIVATE)); + if (!container.privateRsrcs.isEmpty()) { + req.put(LocalResourceVisibility.PRIVATE, container.privateRsrcs); } - if (!appRsrc.isEmpty()) { - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationRequestEvent( - container, appRsrc, LocalResourceVisibility.APPLICATION)); + if (!container.appRsrcs.isEmpty()) { + req.put(LocalResourceVisibility.APPLICATION, container.appRsrcs); } + + container.dispatcher.getEventHandler().handle( + new ContainerLocalizationRequestEvent(container, req)); return ContainerState.LOCALIZING; } else { container.dispatcher.getEventHandler().handle( @@ -546,7 +551,6 @@ public void transition(ContainerImpl container, ContainerEvent event) { } } - @SuppressWarnings("unchecked") // dispatcher not typed static class ExitedWithSuccessTransition extends ContainerTransition { @Override public void transition(ContainerImpl container, ContainerEvent event) { @@ -554,31 +558,25 @@ public void transition(ContainerImpl container, ContainerEvent event) { // Inform the localizer to decrement reference counts and cleanup // resources. - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationEvent( - LocalizationEventType.CLEANUP_CONTAINER_RESOURCES, container)); + container.cleanup(); } } - @SuppressWarnings("unchecked") // dispatcher not typed static class ExitedWithFailureTransition extends ContainerTransition { @Override public void transition(ContainerImpl container, ContainerEvent event) { ContainerExitEvent exitEvent = (ContainerExitEvent) event; - container.exitCode = String.valueOf(exitEvent.getExitCode()); + container.exitCode = exitEvent.getExitCode(); // TODO: Add containerWorkDir to the deletion service. // TODO: Add containerOuputDir to the deletion service. // Inform the localizer to decrement reference counts and cleanup // resources. - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationEvent( - LocalizationEventType.CLEANUP_CONTAINER_RESOURCES, container)); + container.cleanup(); } } - @SuppressWarnings("unchecked") // dispatcher not typed static class ResourceFailedTransition implements SingleArcTransition { @Override @@ -592,30 +590,24 @@ public void transition(ContainerImpl container, ContainerEvent event) { // Inform the localizer to decrement reference counts and cleanup // resources. - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationEvent( - LocalizationEventType.CLEANUP_CONTAINER_RESOURCES, container)); + container.cleanup(); container.metrics.endInitingContainer(); } } - @SuppressWarnings("unchecked") // dispatcher not typed static class KillDuringLocalizationTransition implements SingleArcTransition { @Override public void transition(ContainerImpl container, ContainerEvent event) { // Inform the localizer to decrement reference counts and cleanup // resources. - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationEvent( - LocalizationEventType.CLEANUP_CONTAINER_RESOURCES, container)); + container.cleanup(); container.metrics.endInitingContainer(); ContainerKillEvent killEvent = (ContainerKillEvent) event; container.diagnostics.append(killEvent.getDiagnostic()).append("\n"); } } - @SuppressWarnings("unchecked") // dispatcher not typed static class LocalizedResourceDuringKillTransition implements SingleArcTransition { @Override @@ -647,23 +639,19 @@ public void transition(ContainerImpl container, ContainerEvent event) { } } - @SuppressWarnings("unchecked") // dispatcher not typed static class ContainerKilledTransition implements SingleArcTransition { @Override public void transition(ContainerImpl container, ContainerEvent event) { ContainerExitEvent exitEvent = (ContainerExitEvent) event; - container.exitCode = String.valueOf(exitEvent.getExitCode()); + container.exitCode = exitEvent.getExitCode(); // The process/process-grp is killed. Decrement reference counts and // cleanup resources - container.dispatcher.getEventHandler().handle( - new ContainerLocalizationEvent( - LocalizationEventType.CLEANUP_CONTAINER_RESOURCES, container)); + container.cleanup(); } } - @SuppressWarnings("unchecked") // dispatcher not typed static class ContainerDoneTransition implements SingleArcTransition { @Override @@ -697,7 +685,8 @@ public void handle(ContainerEvent event) { newState = stateMachine.doTransition(event.getType(), event); } catch (InvalidStateTransitonException e) { - LOG.warn("Can't handle this event at current state", e); + LOG.warn("Can't handle this event at current state: Current: [" + + oldState + "], eventType: [" + event.getType() + "]", e); } if (oldState != newState) { LOG.info("Container " + containerID + " transitioned from " diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java index 585ec852f3..1a34247c30 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java @@ -45,10 +45,10 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent; @@ -79,7 +79,7 @@ public ContainerLaunch(Configuration configuration, Dispatcher dispatcher, this.exec = exec; this.container = container; this.dispatcher = dispatcher; - this.logDirsSelector = new LocalDirAllocator(NMConfig.NM_LOG_DIR); + this.logDirsSelector = new LocalDirAllocator(YarnConfiguration.NM_LOG_DIRS); } @Override @@ -89,8 +89,8 @@ public Integer call() { final Map localResources = container.getLocalizedResources(); String containerIdStr = ConverterUtils.toString(container.getContainerID()); final String user = launchContext.getUser(); - final Map env = launchContext.getAllEnv(); - final List command = launchContext.getCommandList(); + final Map env = launchContext.getEnv(); + final List command = launchContext.getCommands(); int ret = -1; try { @@ -107,10 +107,9 @@ public Integer call() { newCmds.add(str.replace(ApplicationConstants.LOG_DIR_EXPANSION_VAR, containerLogDir.toUri().getPath())); } - launchContext.clearCommands(); - launchContext.addAllCommands(newCmds); + launchContext.setCommands(newCmds); - Map envs = launchContext.getAllEnv(); + Map envs = launchContext.getEnv(); Map newEnvs = new HashMap(envs.size()); for (Entry entry : envs.entrySet()) { newEnvs.put( @@ -119,13 +118,12 @@ public Integer call() { ApplicationConstants.LOG_DIR_EXPANSION_VAR, containerLogDir.toUri().getPath())); } - launchContext.clearEnv(); - launchContext.addAllEnv(newEnvs); + launchContext.setEnv(newEnvs); // /////////////////////////// End of variable expansion FileContext lfs = FileContext.getLocalFSFileContext(); LocalDirAllocator lDirAllocator = - new LocalDirAllocator(NMConfig.NM_LOCAL_DIR); // TODO + new LocalDirAllocator(YarnConfiguration.NM_LOCAL_DIRS); // TODO Path nmPrivateContainerScriptPath = lDirAllocator.getLocalPathForWrite( ResourceLocalizationService.NM_PRIVATE_DIR + Path.SEPARATOR @@ -152,8 +150,8 @@ public Integer call() { try { // /////////// Write out the container-script in the nmPrivate space. String[] localDirs = - this.conf.getStrings(NMConfig.NM_LOCAL_DIR, - NMConfig.DEFAULT_NM_LOCAL_DIR); + this.conf.getStrings(YarnConfiguration.NM_LOCAL_DIRS, + YarnConfiguration.DEFAULT_NM_LOCAL_DIRS); List appDirs = new ArrayList(localDirs.length); for (String localDir : localDirs) { Path usersdir = new Path(localDir, ContainerLocalizer.USERCACHE); @@ -170,7 +168,7 @@ public Integer call() { containerWorkDir, FINAL_CONTAINER_TOKENS_FILE).toUri().getPath()); writeLaunchEnv(containerScriptOutStream, env, localResources, - launchContext.getCommandList(), appDirs); + launchContext.getCommands(), appDirs); // /////////// End of writing out container-script // /////////// Write out the container-tokens in the nmPrivate space. diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java index ded9286138..048166ef25 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java @@ -103,7 +103,8 @@ public void handle(ContainersLauncherEvent event) { switch (event.getType()) { case LAUNCH_CONTAINER: Application app = - context.getApplications().get(containerId.getAppId()); + context.getApplications().get( + containerId.getApplicationAttemptId().getApplicationId()); ContainerLaunch launch = new ContainerLaunch(getConfig(), dispatcher, exec, app, event.getContainer()); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java index 9dae769515..fc302c2481 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java @@ -40,7 +40,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.LocalDirAllocator; @@ -274,7 +273,7 @@ private LocalizerStatus createStatus() throws InterruptedException { stat.setLocalPath( ConverterUtils.getYarnUrlFromPath(localPath)); stat.setLocalSize( - FileUtil.getDU(new File(localPath.getParent().toString()))); + FileUtil.getDU(new File(localPath.getParent().toUri()))); stat.setStatus(ResourceStatusType.FETCH_SUCCESS); } catch (ExecutionException e) { stat.setStatus(ResourceStatusType.FETCH_FAILURE); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java index 283c6d4f43..017431501f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java @@ -33,6 +33,7 @@ * {@link LocalResourceVisibility}. * */ + class LocalResourcesTrackerImpl implements LocalResourcesTracker { static final Log LOG = LogFactory.getLog(LocalResourcesTrackerImpl.class); @@ -83,7 +84,7 @@ public boolean contains(LocalResourceRequest resource) { @Override public boolean remove(LocalizedResource rem, DeletionService delService) { // current synchronization guaranteed by crude RLS event for cleanup - LocalizedResource rsrc = localrsrc.remove(rem.getRequest()); + LocalizedResource rsrc = localrsrc.get(rem.getRequest()); if (null == rsrc) { LOG.error("Attempt to remove absent resource: " + rem.getRequest() + " from " + getUser()); @@ -93,10 +94,11 @@ public boolean remove(LocalizedResource rem, DeletionService delService) { || ResourceState.DOWNLOADING.equals(rsrc.getState()) || rsrc != rem) { // internal error - LOG.error("Attempt to remove resource with non-zero refcount"); + LOG.error("Attempt to remove resource: " + rsrc + " with non-zero refcount"); assert false; return false; } + localrsrc.remove(rem.getRequest()); if (ResourceState.LOCALIZED.equals(rsrc.getState())) { delService.delete(getUser(), rsrc.getLocalPath()); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java index 2ba25bfaf4..1e02fe6898 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java @@ -120,7 +120,8 @@ public String toString() { for (ContainerId c : ref) { sb.append("(").append(c.toString()).append(")"); } - sb.append("],").append(getTimestamp()).append("}"); + sb.append("],").append(getTimestamp()).append(",") + .append(getState()).append("}"); return sb.toString(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java index 94a2eb6256..d120b5ccdf 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java @@ -22,6 +22,7 @@ import java.net.URISyntaxException; +import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; import java.util.Iterator; @@ -43,6 +44,7 @@ import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; @@ -50,18 +52,6 @@ import static org.apache.hadoop.fs.CreateFlag.CREATE; import static org.apache.hadoop.fs.CreateFlag.OVERWRITE; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_MAX_PUBLIC_FETCH_THREADS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_CACHE_CLEANUP_MS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_LOCALIZER_BIND_ADDRESS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_LOCAL_DIR; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_LOG_DIR; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_TARGET_CACHE_MB; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_CACHE_CLEANUP_MS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOCALIZER_BIND_ADDRESS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOCAL_DIR; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOG_DIR; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_MAX_PUBLIC_FETCH_THREADS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_TARGET_CACHE_MB; import java.io.IOException; import java.net.InetSocketAddress; @@ -75,7 +65,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; @@ -91,7 +80,6 @@ import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol; import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus; import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction; @@ -106,7 +94,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerResourceFailedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType; @@ -114,6 +102,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerResourceRequestEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceLocalizedEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceReleaseEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRequestEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerSecurityInfo; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenSecretManager; @@ -154,7 +143,7 @@ public ResourceLocalizationService(Dispatcher dispatcher, this.exec = exec; this.dispatcher = dispatcher; this.delService = delService; - this.localDirsSelector = new LocalDirAllocator(NMConfig.NM_LOCAL_DIR); + this.localDirsSelector = new LocalDirAllocator(YarnConfiguration.NM_LOCAL_DIRS); this.publicRsrc = new LocalResourcesTrackerImpl(null, dispatcher); this.cacheCleanup = new ScheduledThreadPoolExecutor(1); } @@ -174,7 +163,7 @@ public void init(Configuration conf) { // TODO queue deletions here, rather than NM init? FileContext lfs = getLocalFileContext(conf); String[] sLocalDirs = - conf.getStrings(NM_LOCAL_DIR, DEFAULT_NM_LOCAL_DIR); + conf.getStrings(YarnConfiguration.NM_LOCAL_DIRS, YarnConfiguration.DEFAULT_NM_LOCAL_DIRS); localDirs = new ArrayList(sLocalDirs.length); logDirs = new ArrayList(sLocalDirs.length); @@ -193,7 +182,7 @@ public void init(Configuration conf) { lfs.mkdir(sysdir, NM_PRIVATE_PERM, true); sysDirs.add(sysdir); } - String[] sLogdirs = conf.getStrings(NM_LOG_DIR, DEFAULT_NM_LOG_DIR); + String[] sLogdirs = conf.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS); for (String sLogdir : sLogdirs) { Path logdir = new Path(sLogdir); logDirs.add(logdir); @@ -206,12 +195,12 @@ public void init(Configuration conf) { logDirs = Collections.unmodifiableList(logDirs); sysDirs = Collections.unmodifiableList(sysDirs); cacheTargetSize = - conf.getLong(NM_TARGET_CACHE_MB, DEFAULT_NM_TARGET_CACHE_MB) << 20; + conf.getLong(YarnConfiguration.NM_LOCALIZER_CACHE_TARGET_SIZE_MB, YarnConfiguration.DEFAULT_NM_LOCALIZER_CACHE_TARGET_SIZE_MB) << 20; cacheCleanupPeriod = - conf.getLong(NM_CACHE_CLEANUP_MS, DEFAULT_NM_CACHE_CLEANUP_MS); + conf.getLong(YarnConfiguration.NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, YarnConfiguration.DEFAULT_NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS); localizationServerAddress = NetUtils.createSocketAddr( - conf.get(NM_LOCALIZER_BIND_ADDRESS, DEFAULT_NM_LOCALIZER_BIND_ADDRESS)); - localizerTracker = new LocalizerTracker(conf); + conf.get(YarnConfiguration.NM_LOCALIZER_ADDRESS, YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS)); + localizerTracker = createLocalizerTracker(conf); dispatcher.register(LocalizerEventType.class, localizerTracker); cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher), cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS); @@ -231,6 +220,10 @@ public void start() { super.start(); } + LocalizerTracker createLocalizerTracker(Configuration conf) { + return new LocalizerTracker(conf); + } + Server createServer() { YarnRPC rpc = YarnRPC.create(getConfig()); Configuration conf = new Configuration(getConfig()); // Clone to separate @@ -244,8 +237,8 @@ Server createServer() { return rpc.getServer(LocalizationProtocol.class, this, localizationServerAddress, conf, secretManager, - conf.getInt(NMConfig.NM_LOCALIZATION_THREADS, - NMConfig.DEFAULT_NM_LOCALIZATION_THREADS)); + conf.getInt(YarnConfiguration.NM_LOCALIZER_CLIENT_THREAD_COUNT, + YarnConfiguration.DEFAULT_NM_LOCALIZER_CLIENT_THREAD_COUNT)); } @@ -265,6 +258,9 @@ public void stop() { public void handle(LocalizationEvent event) { String userName; String appIDStr; + Container c; + Map> rsrcs; + LocalResourcesTracker tracker; // TODO: create log dir as $logdir/$user/$appId switch (event.getType()) { case INIT_APPLICATION_RESOURCES: @@ -289,28 +285,17 @@ public void handle(LocalizationEvent event) { case INIT_CONTAINER_RESOURCES: ContainerLocalizationRequestEvent rsrcReqs = (ContainerLocalizationRequestEvent) event; - Container c = rsrcReqs.getContainer(); + c = rsrcReqs.getContainer(); LocalizerContext ctxt = new LocalizerContext( c.getUser(), c.getContainerID(), c.getCredentials()); - final LocalResourcesTracker tracker; - LocalResourceVisibility vis = rsrcReqs.getVisibility(); - switch (vis) { - default: - case PUBLIC: - tracker = publicRsrc; - break; - case PRIVATE: - tracker = privateRsrc.get(c.getUser()); - break; - case APPLICATION: - tracker = - appRsrc.get(ConverterUtils.toString(c.getContainerID().getAppId())); - break; - } - // We get separate events one each for all resources of one visibility. So - // all the resources in this event are of the same visibility. - for (LocalResourceRequest req : rsrcReqs.getRequestedResources()) { - tracker.handle(new ResourceRequestEvent(req, vis, ctxt)); + rsrcs = rsrcReqs.getRequestedResources(); + for (Map.Entry> e : + rsrcs.entrySet()) { + tracker = getLocalResourcesTracker(e.getKey(), c.getUser(), + c.getContainerID().getApplicationAttemptId().getApplicationId()); + for (LocalResourceRequest req : e.getValue()) { + tracker.handle(new ResourceRequestEvent(req, e.getKey(), ctxt)); + } } break; case CACHE_CLEANUP: @@ -324,14 +309,25 @@ public void handle(LocalizationEvent event) { } break; case CLEANUP_CONTAINER_RESOURCES: - Container container = - ((ContainerLocalizationEvent)event).getContainer(); + ContainerLocalizationCleanupEvent rsrcCleanup = + (ContainerLocalizationCleanupEvent) event; + c = rsrcCleanup.getContainer(); + rsrcs = rsrcCleanup.getResources(); + for (Map.Entry> e : + rsrcs.entrySet()) { + tracker = getLocalResourcesTracker(e.getKey(), c.getUser(), + c.getContainerID().getApplicationAttemptId().getApplicationId()); + for (LocalResourceRequest req : e.getValue()) { + tracker.handle(new ResourceReleaseEvent(req, c.getContainerID())); + } + } // Delete the container directories - userName = container.getUser(); - String containerIDStr = container.toString(); + userName = c.getUser(); + String containerIDStr = c.toString(); appIDStr = - ConverterUtils.toString(container.getContainerID().getAppId()); + ConverterUtils.toString( + c.getContainerID().getApplicationAttemptId().getApplicationId()); for (Path localDir : localDirs) { // Delete the user-owned container-dir @@ -349,8 +345,7 @@ public void handle(LocalizationEvent event) { delService.delete(null, containerSysDir, new Path[] {}); } - dispatcher.getEventHandler().handle(new ContainerEvent( - container.getContainerID(), + dispatcher.getEventHandler().handle(new ContainerEvent(c.getContainerID(), ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP)); break; case DESTROY_APPLICATION_RESOURCES: @@ -392,6 +387,19 @@ public void handle(LocalizationEvent event) { } } + LocalResourcesTracker getLocalResourcesTracker( + LocalResourceVisibility visibility, String user, ApplicationId appId) { + switch (visibility) { + default: + case PUBLIC: + return publicRsrc; + case PRIVATE: + return privateRsrc.get(user); + case APPLICATION: + return appRsrc.get(ConverterUtils.toString(appId)); + } + } + /** * Sub-component handling the spawning of {@link ContainerLocalizer}s */ @@ -496,7 +504,7 @@ class PublicLocalizer extends Thread { PublicLocalizer(Configuration conf) { this(conf, getLocalFileContext(conf), Executors.newFixedThreadPool(conf.getInt( - NM_MAX_PUBLIC_FETCH_THREADS, DEFAULT_MAX_PUBLIC_FETCH_THREADS)), + YarnConfiguration.NM_LOCALIZER_FETCH_THREAD_COUNT, YarnConfiguration.DEFAULT_NM_LOCALIZER_FETCH_THREAD_COUNT)), new HashMap,LocalizerResourceRequestEvent>(), new HashMap>()); } @@ -539,6 +547,7 @@ public void addResource(LocalizerResourceRequestEvent request) { } @Override + @SuppressWarnings("unchecked") // dispatcher not typed public void run() { try { // TODO shutdown, better error handling esp. DU @@ -664,6 +673,7 @@ private LocalResource findNextResource() { } // TODO this sucks. Fix it later + @SuppressWarnings("unchecked") // dispatcher not typed LocalizerHeartbeatResponse update( List remoteResourceStatuses) { LocalizerHeartbeatResponse response = @@ -780,7 +790,9 @@ public void run() { // 2) exec initApplication and wait exec.startLocalizer(nmPrivateCTokensPath, localizationServerAddress, context.getUser(), - ConverterUtils.toString(context.getContainerId().getAppId()), + ConverterUtils.toString( + context.getContainerId(). + getApplicationAttemptId().getApplicationId()), localizerId, localDirs); // TODO handle ExitCodeException separately? } catch (Exception e) { @@ -808,6 +820,7 @@ public CacheCleanup(Dispatcher dispatcher) { } @Override + @SuppressWarnings("unchecked") // dispatcher not typed public void run() { dispatcher.getEventHandler().handle( new LocalizationEvent(LocalizationEventType.CACHE_CLEANUP)); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationCleanupEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationCleanupEvent.java new file mode 100644 index 0000000000..6120259e29 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationCleanupEvent.java @@ -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. +*/ +package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event; + +import java.util.Collection; +import java.util.Map; + +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest; + +public class ContainerLocalizationCleanupEvent extends + ContainerLocalizationEvent { + + private final Map> + rsrc; + + /** + * Event requesting the cleanup of the rsrc. + * @param c + * @param rsrc + */ + public ContainerLocalizationCleanupEvent(Container c, + Map> rsrc) { + super(LocalizationEventType.CLEANUP_CONTAINER_RESOURCES, c); + this.rsrc = rsrc; + } + + public + Map> + getResources() { + return rsrc; + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationRequestEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationRequestEvent.java index eeb735421d..4cb2e5cd19 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationRequestEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationRequestEvent.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event; import java.util.Collection; +import java.util.Map; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; @@ -26,27 +27,23 @@ public class ContainerLocalizationRequestEvent extends ContainerLocalizationEvent { - private final LocalResourceVisibility vis; - private final Collection reqs; + private final Map> + rsrc; /** - * Event requesting the localization of the reqs all with visibility vis + * Event requesting the localization of the rsrc. * @param c - * @param reqs - * @param vis + * @param rsrc */ public ContainerLocalizationRequestEvent(Container c, - Collection reqs, LocalResourceVisibility vis) { + Map> rsrc) { super(LocalizationEventType.INIT_CONTAINER_RESOURCES, c); - this.vis = vis; - this.reqs = reqs; + this.rsrc = rsrc; } - public LocalResourceVisibility getVisibility() { - return vis; + public + Map> + getRequestedResources() { + return rsrc; } - - public Collection getRequestedResources() { - return reqs; - } -} +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ResourceReleaseEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ResourceReleaseEvent.java index 4ab798282d..3cdf1aa147 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ResourceReleaseEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ResourceReleaseEvent.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event; -import java.net.URISyntaxException; - import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest; @@ -26,8 +24,8 @@ public class ResourceReleaseEvent extends ResourceEvent { private final ContainerId container; - public ResourceReleaseEvent(LocalResourceRequest rsrc, ContainerId container) - throws URISyntaxException { + public ResourceReleaseEvent(LocalResourceRequest rsrc, + ContainerId container) { super(rsrc, ResourceEventType.RELEASE); this.container = container; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AggregatedLogFormat.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AggregatedLogFormat.java index 251b391f40..eb22ce22b7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AggregatedLogFormat.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AggregatedLogFormat.java @@ -42,6 +42,7 @@ import org.apache.hadoop.io.file.tfile.TFile; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.util.ConverterUtils; public class AggregatedLogFormat { @@ -89,8 +90,11 @@ public LogValue(String[] rootLogDirs, ContainerId containerId) { public void write(DataOutputStream out) throws IOException { for (String rootLogDir : this.rootLogDirs) { File appLogDir = - new File(rootLogDir, ConverterUtils.toString(this.containerId - .getAppId())); + new File(rootLogDir, + ConverterUtils.toString( + this.containerId.getApplicationAttemptId(). + getApplicationId()) + ); File containerLogDir = new File(appLogDir, ConverterUtils.toString(this.containerId)); @@ -148,8 +152,8 @@ public FSDataOutputStream run() throws Exception { // 256KB minBlockSize : Expected log size for each container too this.writer = new TFile.Writer(this.fsDataOStream, 256 * 1024, conf.get( - LogAggregationService.LOG_COMPRESSION_TYPE, - LogAggregationService.DEFAULT_COMPRESSION_TYPE), null, conf); + YarnConfiguration.NM_LOG_AGG_COMPRESSION_TYPE, + YarnConfiguration.DEFAULT_NM_LOG_AGG_COMPRESSION_TYPE), null, conf); } public void append(LogKey logKey, LogValue logValue) throws IOException { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java index c5eadfd572..974455c460 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java @@ -18,9 +18,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_BIND_ADDRESS; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_BIND_ADDRESS; - import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; @@ -43,12 +40,9 @@ import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorAppFinishedEvent; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorAppStartedEvent; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorContainerFinishedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorEvent; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -65,13 +59,6 @@ public class LogAggregationService extends AbstractService implements Path remoteRootLogDir; private String nodeFile; - static final String LOG_COMPRESSION_TYPE = NMConfig.NM_PREFIX - + "logaggregation.log_compression_type"; - static final String DEFAULT_COMPRESSION_TYPE = "none"; - - private static final String LOG_RENTENTION_POLICY_CONFIG_KEY = - NMConfig.NM_PREFIX + "logaggregation.retain-policy"; - private final ConcurrentMap appLogAggregators; private final ExecutorService threadPool; @@ -86,17 +73,17 @@ public LogAggregationService(DeletionService deletionService) { public synchronized void init(Configuration conf) { this.localRootLogDirs = - conf.getStrings(NMConfig.NM_LOG_DIR, NMConfig.DEFAULT_NM_LOG_DIR); + conf.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS); this.remoteRootLogDir = - new Path(conf.get(NMConfig.REMOTE_USER_LOG_DIR, - NMConfig.DEFAULT_REMOTE_APP_LOG_DIR)); + new Path(conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, + YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); super.init(conf); } @Override public synchronized void start() { String address = - getConfig().get(NM_BIND_ADDRESS, DEFAULT_NM_BIND_ADDRESS); + getConfig().get(YarnConfiguration.NM_ADDRESS, YarnConfiguration.DEFAULT_NM_ADDRESS); InetSocketAddress cmBindAddress = NetUtils.createSocketAddr(address); try { this.nodeFile = @@ -185,11 +172,13 @@ private void stopContainer(ContainerId containerId, String exitCode) { // A container is complete. Put this containers' logs up for aggregation if // this containers' logs are needed. - if (!this.appLogAggregators.containsKey(containerId.getAppId())) { + if (!this.appLogAggregators.containsKey( + containerId.getApplicationAttemptId().getApplicationId())) { throw new YarnException("Application is not initialized yet for " + containerId); } - this.appLogAggregators.get(containerId.getAppId()) + this.appLogAggregators.get( + containerId.getApplicationAttemptId().getApplicationId()) .startContainerLogAggregation(containerId, exitCode.equals("0")); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogDumper.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogDumper.java index 1caf36ed1c..d9d961dc65 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogDumper.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogDumper.java @@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogKey; import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogReader; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -106,8 +105,8 @@ public int run(String[] args) throws Exception { return -1; } else { Path remoteRootLogDir = - new Path(getConf().get(NMConfig.REMOTE_USER_LOG_DIR, - NMConfig.DEFAULT_REMOTE_APP_LOG_DIR)); + new Path(getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, + YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); AggregatedLogFormat.LogReader reader = new AggregatedLogFormat.LogReader(getConf(), LogAggregationService.getRemoteNodeLogFileForApp( @@ -151,8 +150,8 @@ private int dumpAContainerLogs(String containerIdStr, dumpAllContainersLogs(ApplicationId appId, DataOutputStream out) throws IOException { Path remoteRootLogDir = - new Path(getConf().get(NMConfig.REMOTE_USER_LOG_DIR, - NMConfig.DEFAULT_REMOTE_APP_LOG_DIR)); + new Path(getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, + YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); Path remoteAppLogDir = LogAggregationService.getRemoteAppLogDir(remoteRootLogDir, appId); RemoteIterator nodeFiles = diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/event/LogAggregatorContainerFinishedEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/event/LogAggregatorContainerFinishedEvent.java index 117d30cf19..68ec27a73a 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/event/LogAggregatorContainerFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/event/LogAggregatorContainerFinishedEvent.java @@ -23,10 +23,10 @@ public class LogAggregatorContainerFinishedEvent extends LogAggregatorEvent { private final ContainerId containerId; - private final String exitCode; + private final int exitCode; public LogAggregatorContainerFinishedEvent(ContainerId containerId, - String exitCode) { + int exitCode) { super(LogAggregatorEventType.CONTAINER_FINISHED); this.containerId = containerId; this.exitCode = exitCode; @@ -36,7 +36,7 @@ public ContainerId getContainerId() { return this.containerId; } - public String getExitCode() { + public int getExitCode() { return this.exitCode; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index c435e3abe4..2b077949c4 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -29,11 +29,11 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerId; +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.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree; @@ -45,14 +45,6 @@ public class ContainersMonitorImpl extends AbstractService implements final static Log LOG = LogFactory .getLog(ContainersMonitorImpl.class); - private final static String MONITORING_INTERVAL_CONFIG_KEY = - NMConfig.NM_PREFIX + "containers-monitor.monitoring-interval"; - public static final String RESOURCE_CALCULATOR_PLUGIN_CONFIG_KEY = - NMConfig.NM_PREFIX + "containers-monitor.resourcecalculatorplugin"; - public static final String NM_RESERVED_PHYSICALMEMORY_MB = - NMConfig.NM_PREFIX + "reserved-physical-memory.mb"; - - private final static int MONITORING_INTERVAL_DEFAULT = 3000; private long monitoringInterval; private MonitoringThread monitoringThread; @@ -96,11 +88,11 @@ public ContainersMonitorImpl(ContainerExecutor exec, @Override public synchronized void init(Configuration conf) { this.monitoringInterval = - conf.getLong(MONITORING_INTERVAL_CONFIG_KEY, - MONITORING_INTERVAL_DEFAULT); + conf.getLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, + YarnConfiguration.DEFAULT_NM_CONTAINER_MON_INTERVAL_MS); Class clazz = - conf.getClass(RESOURCE_CALCULATOR_PLUGIN_CONFIG_KEY, null, + conf.getClass(YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR, null, ResourceCalculatorPlugin.class); this.resourceCalculatorPlugin = ResourceCalculatorPlugin.getResourceCalculatorPlugin(clazz, conf); @@ -120,7 +112,7 @@ public synchronized void init(Configuration conf) { // ///////// Virtual memory configuration ////// this.maxVmemAllottedForContainers = - conf.getLong(NMConfig.NM_VMEM_GB, NMConfig.DEFAULT_NM_VMEM_GB); + conf.getLong(YarnConfiguration.NM_VMEM_GB, YarnConfiguration.DEFAULT_NM_VMEM_GB); this.maxVmemAllottedForContainers = this.maxVmemAllottedForContainers * 1024 * 1024 * 1024L; //Normalize @@ -131,7 +123,7 @@ public synchronized void init(Configuration conf) { // ///////// Physical memory configuration ////// long reservedPmemOnNM = - conf.getLong(NM_RESERVED_PHYSICALMEMORY_MB, DISABLED_MEMORY_LIMIT); + conf.getLong(YarnConfiguration.NM_RESERVED_MEMORY_MB, DISABLED_MEMORY_LIMIT); reservedPmemOnNM = reservedPmemOnNM == DISABLED_MEMORY_LIMIT ? DISABLED_MEMORY_LIMIT diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java index 31fa4a5787..68b0686a25 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java @@ -18,9 +18,6 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_LOG_DIR; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOG_DIR; - import java.io.File; import java.io.FileReader; import java.io.IOException; @@ -33,10 +30,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.nodemanager.Context; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -64,7 +61,7 @@ public static class ContainersLogsBlock extends HtmlBlock implements @Inject public ContainersLogsBlock(Configuration conf, Context context) { this.conf = conf; - this.logsSelector = new LocalDirAllocator(NMConfig.NM_LOG_DIR); + this.logsSelector = new LocalDirAllocator(YarnConfiguration.NM_LOG_DIRS); this.nmContext = context; this.recordFactory = RecordFactoryProvider.getRecordFactory(conf); } @@ -98,7 +95,8 @@ protected void render(Block html) { logFile = new File(this.logsSelector .getLocalPathToRead( - ConverterUtils.toString(containerId.getAppId()) + ConverterUtils.toString( + containerId.getApplicationAttemptId().getApplicationId()) + Path.SEPARATOR + $(CONTAINER_ID) + Path.SEPARATOR + $(CONTAINER_LOG_TYPE), this.conf).toUri() @@ -176,10 +174,12 @@ protected void render(Block html) { static List getContainerLogDirs(Configuration conf, ContainerId containerId) { String[] logDirs = - conf.getStrings(NM_LOG_DIR, DEFAULT_NM_LOG_DIR); + conf.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS); List containerLogDirs = new ArrayList(logDirs.length); for (String logDir : logDirs) { - String appIdStr = ConverterUtils.toString(containerId.getAppId()); + String appIdStr = + ConverterUtils.toString( + containerId.getApplicationAttemptId().getApplicationId()); File appLogDir = new File(logDir, appIdStr); String containerIdStr = ConverterUtils.toString(containerId); containerLogDirs.add(new File(appLogDir, containerIdStr)); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java index 1f53817b09..27be38a029 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.nodemanager.Context; @@ -69,10 +70,14 @@ protected void render(Block html) { ConverterUtils.toContainerId(this.recordFactory, $(CONTAINER_ID)); Container container = this.nmContext.getContainers().get(containerID); ContainerStatus containerData = container.cloneAndGetContainerStatus(); + int exitCode = containerData.getExitStatus(); + String exiStatus = + (exitCode == YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS) ? + "N/A" : String.valueOf(exitCode); info("Container information") ._("ContainerID", $(CONTAINER_ID)) ._("ContainerState", container.getContainerState()) - ._("ExitStatus", containerData.getExitStatus()) + ._("ExitStatus", exiStatus) ._("Diagnostics", containerData.getDiagnostics()) ._("User", container.getUser()) ._("TotalMemoryNeeded", diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMController.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMController.java index ba8e41bfd6..a4343d1ab8 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMController.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMController.java @@ -21,7 +21,6 @@ import static org.apache.hadoop.yarn.util.StringHelper.join; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.webapp.Controller; import com.google.inject.Inject; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java index cbf12e1986..307e87eccd 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java @@ -24,8 +24,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.Context; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.ResourceView; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.webapp.WebApp; @@ -52,8 +52,8 @@ public synchronized void init(Configuration conf) { @Override public synchronized void start() { - String bindAddress = getConfig().get(NMConfig.NM_HTTP_BIND_ADDRESS, - NMConfig.DEFAULT_NM_HTTP_BIND_ADDRESS); + String bindAddress = getConfig().get(YarnConfiguration.NM_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_NM_WEBAPP_ADDRESS); LOG.info("Instantiating NMWebApp at " + bindAddress); try { this.webApp = diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java index 9ac6f2cfdd..c4db3b6af9 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java @@ -20,12 +20,15 @@ import static org.junit.Assert.fail; +import java.util.Collection; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent; @@ -57,8 +60,8 @@ public class DummyContainerManager extends ContainerManagerImpl { public DummyContainerManager(Context context, ContainerExecutor exec, DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater, - NodeManagerMetrics metrics) { - super(context, exec, deletionContext, nodeStatusUpdater, metrics); + NodeManagerMetrics metrics, ContainerTokenSecretManager containerTokenSecretManager) { + super(context, exec, deletionContext, nodeStatusUpdater, metrics, containerTokenSecretManager); } @Override @@ -79,14 +82,17 @@ public void handle(LocalizationEvent event) { ContainerLocalizationRequestEvent rsrcReqs = (ContainerLocalizationRequestEvent) event; // simulate localization of all requested resources - for (LocalResourceRequest req : rsrcReqs.getRequestedResources()) { - LOG.info("DEBUG: " + req + ":" + - rsrcReqs.getContainer().getContainerID()); - dispatcher.getEventHandler().handle( - new ContainerResourceLocalizedEvent( - rsrcReqs.getContainer().getContainerID(), req, - new Path("file:///local" + req.getPath().toUri().getPath()))); - } + for (Collection rc : rsrcReqs + .getRequestedResources().values()) { + for (LocalResourceRequest req : rc) { + LOG.info("DEBUG: " + req + ":" + + rsrcReqs.getContainer().getContainerID()); + dispatcher.getEventHandler().handle( + new ContainerResourceLocalizedEvent(rsrcReqs.getContainer() + .getContainerID(), req, new Path("file:///local" + + req.getPath().toUri().getPath()))); + } + } break; case CLEANUP_CONTAINER_RESOURCES: Container container = diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java index 54ee1f4f53..6ee220b674 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager; import org.junit.After; @@ -119,13 +120,13 @@ public void testLocalFilesCleanup() throws InterruptedException, private boolean shouldRunTest() { return System - .getProperty(LinuxContainerExecutor.CONTAINER_EXECUTOR_EXEC_KEY) != null; + .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null; } @Override protected ContainerExecutor createContainerExecutor() { - super.conf.set(LinuxContainerExecutor.CONTAINER_EXECUTOR_EXEC_KEY, System - .getProperty(LinuxContainerExecutor.CONTAINER_EXECUTOR_EXEC_KEY)); + super.conf.set(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH, System + .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH)); LinuxContainerExecutor linuxContainerExecutor = new LinuxContainerExecutor(); linuxContainerExecutor.setConf(super.conf); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java index 420be7c0e6..7fa43b7c78 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java @@ -21,8 +21,6 @@ import java.io.File; import java.io.IOException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.NodeHealthCheckerService; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; @@ -37,9 +35,9 @@ 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.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.server.api.ResourceTracker; import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; @@ -48,8 +46,8 @@ public class TestEventFlow { - private static final Log LOG = LogFactory.getLog(TestEventFlow.class); - private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + private static final RecordFactory recordFactory = + RecordFactoryProvider.getRecordFactory(null); private static File localDir = new File("target", TestEventFlow.class.getName() + "-localDir").getAbsoluteFile(); @@ -74,9 +72,10 @@ public void testSuccessfulContainerLaunch() throws InterruptedException, Context context = new NMContext(); YarnConfiguration conf = new YarnConfiguration(); - conf.set(NMConfig.NM_LOCAL_DIR, localDir.getAbsolutePath()); - conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath()); - conf.set(NMConfig.REMOTE_USER_LOG_DIR, remoteLogDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, + remoteLogDir.getAbsolutePath()); ContainerExecutor exec = new DefaultContainerExecutor(); exec.setConf(conf); @@ -84,8 +83,9 @@ public void testSuccessfulContainerLaunch() throws InterruptedException, Dispatcher dispatcher = new AsyncDispatcher(); NodeHealthCheckerService healthChecker = null; NodeManagerMetrics metrics = NodeManagerMetrics.create(); + ContainerTokenSecretManager containerTokenSecretManager = new ContainerTokenSecretManager(); NodeStatusUpdater nodeStatusUpdater = - new NodeStatusUpdaterImpl(context, dispatcher, healthChecker, metrics) { + new NodeStatusUpdaterImpl(context, dispatcher, healthChecker, metrics, containerTokenSecretManager) { @Override protected ResourceTracker getRMClient() { return new LocalRMInterface(); @@ -98,27 +98,36 @@ protected void startStatusUpdater() { }; DummyContainerManager containerManager = - new DummyContainerManager(context, exec, del, nodeStatusUpdater, metrics); + new DummyContainerManager(context, exec, del, nodeStatusUpdater, + metrics, containerTokenSecretManager); containerManager.init(conf); containerManager.start(); - ContainerLaunchContext launchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class); + ContainerLaunchContext launchContext = + recordFactory.newRecordInstance(ContainerLaunchContext.class); ContainerId cID = recordFactory.newRecordInstance(ContainerId.class); - cID.setAppId(recordFactory.newRecordInstance(ApplicationId.class)); - ApplicationAttemptId atId = recordFactory.newRecordInstance(ApplicationAttemptId.class); - atId.setApplicationId(cID.getAppId()); - cID.setAppAttemptId(atId); + ApplicationId applicationId = + recordFactory.newRecordInstance(ApplicationId.class); + applicationId.setClusterTimestamp(0); + applicationId.setId(0); + ApplicationAttemptId applicationAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); + applicationAttemptId.setApplicationId(applicationId); + applicationAttemptId.setAttemptId(0); + cID.setApplicationAttemptId(applicationAttemptId); launchContext.setContainerId(cID); launchContext.setUser("testing"); launchContext.setResource(recordFactory.newRecordInstance(Resource.class)); - StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class); + StartContainerRequest request = + recordFactory.newRecordInstance(StartContainerRequest.class); request.setContainerLaunchContext(launchContext); containerManager.startContainer(request); BaseContainerManagerTest.waitForContainerState(containerManager, cID, ContainerState.RUNNING); - StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class); + StopContainerRequest stopRequest = + recordFactory.newRecordInstance(StopContainerRequest.class); stopRequest.setContainerId(cID); containerManager.stopContainer(stopRequest); BaseContainerManagerTest.waitForContainerState(containerManager, cID, diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java index b642279206..8a6dabf1b2 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger; import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants; import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.Keys; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.net.NetUtils; @@ -211,7 +212,8 @@ public void ping() { public void testNMAuditLoggerWithIP() throws Exception { Configuration conf = new Configuration(); // start the IPC server - Server server = RPC.getServer(new MyTestRPCServer(), "0.0.0.0", 0, conf); + Server server = RPC.getServer(TestProtocol.class, + new MyTestRPCServer(), "0.0.0.0", 0, 5, true, conf, null); server.start(); InetSocketAddress addr = NetUtils.getConnectAddress(server); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java index bec29965fd..2f0e3f54d3 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java @@ -21,6 +21,10 @@ import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentMap; import org.apache.commons.logging.Log; @@ -28,10 +32,13 @@ import org.apache.hadoop.NodeHealthCheckerService; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -47,9 +54,12 @@ import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse; import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.api.records.RegistrationResponse; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; +import org.apache.hadoop.yarn.service.Service; import org.apache.hadoop.yarn.service.Service.STATE; import org.junit.After; import org.junit.Assert; @@ -58,24 +68,38 @@ public class TestNodeStatusUpdater { + // temp fix until metrics system can auto-detect itself running in unit test: + static { + DefaultMetricsSystem.setMiniClusterMode(true); + } + static final Log LOG = LogFactory.getLog(TestNodeStatusUpdater.class); static final Path basedir = new Path("target", TestNodeStatusUpdater.class.getName()); - private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + private static final RecordFactory recordFactory = RecordFactoryProvider + .getRecordFactory(null); int heartBeatID = 0; volatile Error nmStartError = null; + private final List registeredNodes = new ArrayList(); + + @After + public void tearDown() { + this.registeredNodes.clear(); + DefaultMetricsSystem.shutdown(); + } private class MyResourceTracker implements ResourceTracker { - private Context context; + private final Context context; public MyResourceTracker(Context context) { this.context = context; } @Override - public RegisterNodeManagerResponse registerNodeManager(RegisterNodeManagerRequest request) throws YarnRemoteException { + public RegisterNodeManagerResponse registerNodeManager( + RegisterNodeManagerRequest request) throws YarnRemoteException { NodeId nodeId = request.getNodeId(); Resource resource = request.getResource(); LOG.info("Registering " + nodeId.toString()); @@ -86,33 +110,60 @@ public RegisterNodeManagerResponse registerNodeManager(RegisterNodeManagerReques Assert.fail(e.getMessage()); } Assert.assertEquals(5 * 1024, resource.getMemory()); - RegistrationResponse regResponse = recordFactory.newRecordInstance(RegistrationResponse.class); - - RegisterNodeManagerResponse response = recordFactory.newRecordInstance(RegisterNodeManagerResponse.class); + registeredNodes.add(nodeId); + RegistrationResponse regResponse = recordFactory + .newRecordInstance(RegistrationResponse.class); + + RegisterNodeManagerResponse response = recordFactory + .newRecordInstance(RegisterNodeManagerResponse.class); response.setRegistrationResponse(regResponse); return response; } - ApplicationId applicationID = recordFactory.newRecordInstance(ApplicationId.class); - ApplicationAttemptId appAttemptID = recordFactory.newRecordInstance(ApplicationAttemptId.class); - ContainerId firstContainerID = recordFactory.newRecordInstance(ContainerId.class); - ContainerId secondContainerID = recordFactory.newRecordInstance(ContainerId.class); + ApplicationId applicationID = recordFactory + .newRecordInstance(ApplicationId.class); + ApplicationAttemptId appAttemptID = recordFactory + .newRecordInstance(ApplicationAttemptId.class); + ContainerId firstContainerID = recordFactory + .newRecordInstance(ContainerId.class); + ContainerId secondContainerID = recordFactory + .newRecordInstance(ContainerId.class); + + private Map> getAppToContainerStatusMap( + List containers) { + Map> map = + new HashMap>(); + for (ContainerStatus cs : containers) { + ApplicationId applicationId = + cs.getContainerId().getApplicationAttemptId().getApplicationId(); + List appContainers = map.get(applicationId); + if (appContainers == null) { + appContainers = new ArrayList(); + map.put(applicationId, appContainers); + } + appContainers.add(cs); + } + return map; + } @Override - public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request) throws YarnRemoteException { + public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request) + throws YarnRemoteException { NodeStatus nodeStatus = request.getNodeStatus(); LOG.info("Got heartbeat number " + heartBeatID); nodeStatus.setResponseId(heartBeatID++); + Map> appToContainers = + getAppToContainerStatusMap(nodeStatus.getContainersStatuses()); if (heartBeatID == 1) { - Assert.assertEquals(0, nodeStatus.getAllContainers().size()); + Assert.assertEquals(0, nodeStatus.getContainersStatuses().size()); // Give a container to the NM. applicationID.setId(heartBeatID); appAttemptID.setApplicationId(applicationID); - firstContainerID.setAppId(applicationID); - firstContainerID.setAppAttemptId(appAttemptID); + firstContainerID.setApplicationAttemptId(appAttemptID); firstContainerID.setId(heartBeatID); - ContainerLaunchContext launchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class); + ContainerLaunchContext launchContext = recordFactory + .newRecordInstance(ContainerLaunchContext.class); launchContext.setContainerId(firstContainerID); launchContext.setResource(recordFactory.newRecordInstance(Resource.class)); launchContext.getResource().setMemory(2); @@ -121,11 +172,9 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request) throws } else if (heartBeatID == 2) { // Checks on the RM end Assert.assertEquals("Number of applications should only be one!", 1, - nodeStatus.getAllContainers().size()); + nodeStatus.getContainersStatuses().size()); Assert.assertEquals("Number of container for the app should be one!", - 1, nodeStatus.getContainers(applicationID).size()); - Assert.assertEquals(2, nodeStatus.getContainers(applicationID).get(0) - .getResource().getMemory()); + 1, appToContainers.get(applicationID).size()); // Checks on the NM end ConcurrentMap activeContainers = @@ -135,10 +184,10 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request) throws // Give another container to the NM. applicationID.setId(heartBeatID); appAttemptID.setApplicationId(applicationID); - secondContainerID.setAppId(applicationID); - secondContainerID.setAppAttemptId(appAttemptID); + secondContainerID.setApplicationAttemptId(appAttemptID); secondContainerID.setId(heartBeatID); - ContainerLaunchContext launchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class); + ContainerLaunchContext launchContext = recordFactory + .newRecordInstance(ContainerLaunchContext.class); launchContext.setContainerId(secondContainerID); launchContext.setResource(recordFactory.newRecordInstance(Resource.class)); launchContext.getResource().setMemory(3); @@ -147,23 +196,21 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request) throws } else if (heartBeatID == 3) { // Checks on the RM end Assert.assertEquals("Number of applications should only be one!", 1, - nodeStatus.getAllContainers().size()); + appToContainers.size()); Assert.assertEquals("Number of container for the app should be two!", - 2, nodeStatus.getContainers(applicationID).size()); - Assert.assertEquals(2, nodeStatus.getContainers(applicationID).get(0) - .getResource().getMemory()); - Assert.assertEquals(3, nodeStatus.getContainers(applicationID).get(1) - .getResource().getMemory()); + 2, appToContainers.get(applicationID).size()); // Checks on the NM end ConcurrentMap activeContainers = this.context.getContainers(); Assert.assertEquals(2, activeContainers.size()); } - HeartbeatResponse response = recordFactory.newRecordInstance(HeartbeatResponse.class); + HeartbeatResponse response = recordFactory + .newRecordInstance(HeartbeatResponse.class); response.setResponseId(heartBeatID); - NodeHeartbeatResponse nhResponse = recordFactory.newRecordInstance(NodeHeartbeatResponse.class); + NodeHeartbeatResponse nhResponse = recordFactory + .newRecordInstance(NodeHeartbeatResponse.class); nhResponse.setHeartbeatResponse(response); return nhResponse; } @@ -173,8 +220,10 @@ private class MyNodeStatusUpdater extends NodeStatusUpdaterImpl { private Context context; public MyNodeStatusUpdater(Context context, Dispatcher dispatcher, - NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) { - super(context, dispatcher, healthChecker, metrics); + NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics, + ContainerTokenSecretManager containerTokenSecretManager) { + super(context, dispatcher, healthChecker, metrics, + containerTokenSecretManager); this.context = context; } @@ -200,21 +249,23 @@ public void testNMRegistration() throws InterruptedException { final NodeManager nm = new NodeManager() { @Override protected NodeStatusUpdater createNodeStatusUpdater(Context context, - Dispatcher dispatcher, NodeHealthCheckerService healthChecker) { + Dispatcher dispatcher, NodeHealthCheckerService healthChecker, + ContainerTokenSecretManager containerTokenSecretManager) { return new MyNodeStatusUpdater(context, dispatcher, healthChecker, - metrics); + metrics, containerTokenSecretManager); } }; - YarnConfiguration conf = new YarnConfiguration(); - conf.setInt(NMConfig.NM_VMEM_GB, 5); // 5GB - conf.set(NMConfig.NM_BIND_ADDRESS, "127.0.0.1:12345"); - conf.set(NMConfig.NM_LOCALIZER_BIND_ADDRESS, "127.0.0.1:12346"); - conf.set(NMConfig.NM_LOG_DIR, new Path(basedir, "logs").toUri().getPath()); - conf.set(NMConfig.REMOTE_USER_LOG_DIR, new Path(basedir, "remotelogs") - .toUri().getPath()); - conf.set(NMConfig.NM_LOCAL_DIR, new Path(basedir, "nm0").toUri().getPath()); + YarnConfiguration conf = createNMConfig(); nm.init(conf); + + // verify that the last service is the nodeStatusUpdater (ie registration + // with RM) + Object[] services = nm.getServices().toArray(); + Object lastService = services[services.length-1]; + Assert.assertTrue("last service is NOT the node status updater", + lastService instanceof NodeStatusUpdater); + new Thread() { public void run() { try { @@ -244,7 +295,75 @@ public void run() { while (heartBeatID <= 3) { Thread.sleep(500); } + Assert.assertEquals("Number of registered NMs is wrong!!", 1, + this.registeredNodes.size()); nm.stop(); } + + /** + * Verifies that if for some reason NM fails to start ContainerManager RPC + * server, RM is oblivious to NM's presence. The behaviour is like this + * because otherwise, NM will report to RM even if all its servers are not + * started properly, RM will think that the NM is alive and will retire the NM + * only after NM_EXPIRY interval. See MAPREDUCE-2749. + */ + @Test + public void testNoRegistrationWhenNMServicesFail() { + + final NodeManager nm = new NodeManager() { + @Override + protected NodeStatusUpdater createNodeStatusUpdater(Context context, + Dispatcher dispatcher, NodeHealthCheckerService healthChecker, + ContainerTokenSecretManager containerTokenSecretManager) { + return new MyNodeStatusUpdater(context, dispatcher, healthChecker, + metrics, containerTokenSecretManager); + } + + @Override + protected ContainerManagerImpl createContainerManager(Context context, + ContainerExecutor exec, DeletionService del, + NodeStatusUpdater nodeStatusUpdater, + ContainerTokenSecretManager containerTokenSecretManager) { + return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater, + metrics, containerTokenSecretManager) { + @Override + public void start() { + // Simulating failure of starting RPC server + throw new YarnException("Starting of RPC Server failed"); + } + }; + } + }; + + YarnConfiguration conf = createNMConfig(); + nm.init(conf); + try { + nm.start(); + Assert.fail("NM should have failed to start. Didn't get exception!!"); + } catch (Exception e) { + Assert.assertEquals("Starting of RPC Server failed", e.getCause() + .getMessage()); + } + + Assert.assertEquals("NM state is wrong!", Service.STATE.INITED, nm + .getServiceState()); + + Assert.assertEquals("Number of registered nodes is wrong!", 0, + this.registeredNodes.size()); + } + + private YarnConfiguration createNMConfig() { + YarnConfiguration conf = new YarnConfiguration(); + conf.setInt(YarnConfiguration.NM_VMEM_GB, 5); // 5GB + conf.set(YarnConfiguration.NM_ADDRESS, "127.0.0.1:12345"); + conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "127.0.0.1:12346"); + conf.set(YarnConfiguration.NM_LOG_DIRS, new Path(basedir, "logs").toUri() + .getPath()); + conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, new Path(basedir, + "remotelogs").toUri().getPath()); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, new Path(basedir, "nm0") + .toUri().getPath()); + return conf; + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBLocalizerRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBLocalizerRPC.java index eda80ed347..fdee897095 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBLocalizerRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBLocalizerRPC.java @@ -22,6 +22,7 @@ import org.apache.avro.ipc.Server; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.YarnRPC; @@ -39,7 +40,7 @@ public class TestPBLocalizerRPC { static RecordFactory createPBRecordFactory() { Configuration conf = new Configuration(); - conf.set(RecordFactoryProvider.RPC_SERIALIZER_KEY, "protocolbuffers"); + conf.set(YarnConfiguration.IPC_SERIALIZER_TYPE, "protocolbuffers"); return RecordFactoryProvider.getRecordFactory(conf); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java index c546e7d596..8996b1ebfd 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java @@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.RPCUtil; @@ -46,7 +47,7 @@ public class TestPBRecordImpl { static RecordFactory createPBRecordFactory() { Configuration conf = new Configuration(); - conf.set(RecordFactoryProvider.RPC_SERIALIZER_KEY, "protocolbuffers"); + conf.set(YarnConfiguration.IPC_SERIALIZER_TYPE, "protocolbuffers"); return RecordFactoryProvider.getRecordFactory(conf); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java index 6ddb72919c..2a366ec2fc 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java @@ -41,13 +41,13 @@ import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.server.api.ResourceTracker; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; import org.apache.hadoop.yarn.server.nodemanager.LocalRMInterface; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl; @@ -68,6 +68,7 @@ public abstract class BaseContainerManagerTest { protected static File localLogDir; protected static File remoteLogDir; protected static File tmpDir; + protected ContainerTokenSecretManager containerTokenSecretManager = new ContainerTokenSecretManager(); protected final NodeManagerMetrics metrics = NodeManagerMetrics.create(); @@ -95,7 +96,7 @@ public BaseContainerManagerTest() throws UnsupportedFileSystemException { protected String user = "nobody"; protected NodeStatusUpdater nodeStatusUpdater = new NodeStatusUpdaterImpl( - context, new AsyncDispatcher(), null, metrics) { + context, new AsyncDispatcher(), null, metrics, this.containerTokenSecretManager) { @Override protected ResourceTracker getRMClient() { return new LocalRMInterface(); @@ -129,10 +130,10 @@ public void setup() throws IOException { LOG.info("Created tmpDir in " + tmpDir.getAbsolutePath()); String bindAddress = "0.0.0.0:5555"; - conf.set(NMConfig.NM_BIND_ADDRESS, bindAddress); - conf.set(NMConfig.NM_LOCAL_DIR, localDir.getAbsolutePath()); - conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath()); - conf.set(NMConfig.REMOTE_USER_LOG_DIR, remoteLogDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_ADDRESS, bindAddress); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath()); // Default delSrvc delSrvc = new DeletionService(exec) { @@ -148,7 +149,7 @@ public void delete(String user, Path subDir, Path[] baseDirs) { exec = createContainerExecutor(); containerManager = new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater, - metrics); + metrics, this.containerTokenSecretManager); containerManager.init(conf); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java index 2324708150..d52647c8cb 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent; @@ -95,10 +96,10 @@ static class ServiceB extends LightService { @Test public void testAuxEventDispatch() { Configuration conf = new Configuration(); - conf.setStrings(AuxServices.AUX_SERVICES, new String[] { "Asrv", "Bsrv" }); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Asrv"), + conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { "Asrv", "Bsrv" }); + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Asrv"), ServiceA.class, Service.class); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Bsrv"), + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Bsrv"), ServiceB.class, Service.class); conf.setInt("A.expected.init", 1); conf.setInt("B.expected.stop", 1); @@ -123,10 +124,10 @@ public void testAuxEventDispatch() { @Test public void testAuxServices() { Configuration conf = new Configuration(); - conf.setStrings(AuxServices.AUX_SERVICES, new String[] { "Asrv", "Bsrv" }); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Asrv"), + conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { "Asrv", "Bsrv" }); + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Asrv"), ServiceA.class, Service.class); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Bsrv"), + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Bsrv"), ServiceB.class, Service.class); final AuxServices aux = new AuxServices(); aux.init(conf); @@ -154,10 +155,10 @@ public void testAuxServices() { @Test public void testAuxServicesMeta() { Configuration conf = new Configuration(); - conf.setStrings(AuxServices.AUX_SERVICES, new String[] { "Asrv", "Bsrv" }); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Asrv"), + conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { "Asrv", "Bsrv" }); + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Asrv"), ServiceA.class, Service.class); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Bsrv"), + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Bsrv"), ServiceB.class, Service.class); final AuxServices aux = new AuxServices(); aux.init(conf); @@ -191,10 +192,10 @@ public void testAuxServicesMeta() { @Test public void testAuxUnexpectedStop() { Configuration conf = new Configuration(); - conf.setStrings(AuxServices.AUX_SERVICES, new String[] { "Asrv", "Bsrv" }); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Asrv"), + conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { "Asrv", "Bsrv" }); + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Asrv"), ServiceA.class, Service.class); - conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Bsrv"), + conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Bsrv"), ServiceB.class, Service.class); final AuxServices aux = new AuxServices(); aux.init(conf); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java index 702100386b..87460d045c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java @@ -23,7 +23,11 @@ import java.io.FileReader; import java.io.IOException; import java.io.PrintWriter; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import junit.framework.Assert; @@ -53,6 +57,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.util.ConverterUtils; import org.junit.Test; @@ -66,6 +71,20 @@ public TestContainerManager() throws UnsupportedFileSystemException { LOG = LogFactory.getLog(TestContainerManager.class); } + private ContainerId createContainerId() { + ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class); + appId.setClusterTimestamp(0); + appId.setId(0); + ApplicationAttemptId appAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); + appAttemptId.setApplicationId(appId); + appAttemptId.setAttemptId(1); + ContainerId containerId = + recordFactory.newRecordInstance(ContainerId.class); + containerId.setApplicationAttemptId(appAttemptId); + return containerId; + } + @Test public void testContainerManagerInitialization() throws IOException { @@ -74,14 +93,9 @@ public void testContainerManagerInitialization() throws IOException { // Just do a query for a non-existing container. boolean throwsException = false; try { - GetContainerStatusRequest request = recordFactory.newRecordInstance(GetContainerStatusRequest.class); - ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class); - ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class); - appAttemptId.setApplicationId(appId); - appAttemptId.setAttemptId(1); - ContainerId cId = recordFactory.newRecordInstance(ContainerId.class); - cId.setAppId(appId); - cId.setAppAttemptId(appAttemptId); + GetContainerStatusRequest request = + recordFactory.newRecordInstance(GetContainerStatusRequest.class); + ContainerId cId = createContainerId(); request.setContainerId(cId); containerManager.getContainerStatus(request); } catch (YarnRemoteException e) { @@ -106,20 +120,14 @@ public void testContainerSetup() throws IOException, InterruptedException { ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class); // ////// Construct the Container-id - ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class); - ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class); - appAttemptId.setApplicationId(appId); - appAttemptId.setAttemptId(1); - ContainerId cId = recordFactory.newRecordInstance(ContainerId.class); - cId.setAppId(appId); - cId.setAppAttemptId(appAttemptId); + ContainerId cId = createContainerId(); container.setContainerId(cId); container.setUser(user); // ////// Construct the container-spec. - ContainerLaunchContext containerLaunchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class); -// containerLaunchContext.resources = new HashMap(); + ContainerLaunchContext containerLaunchContext = + recordFactory.newRecordInstance(ContainerLaunchContext.class); URL resource_alpha = ConverterUtils.getYarnUrlFromPath(localFS .makeQualified(new Path(file.getAbsolutePath()))); @@ -130,14 +138,17 @@ public void testContainerSetup() throws IOException, InterruptedException { rsrc_alpha.setType(LocalResourceType.FILE); rsrc_alpha.setTimestamp(file.lastModified()); String destinationFile = "dest_file"; - containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha); + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setUser(container.getUser()); containerLaunchContext.setContainerId(container.getContainerId()); containerLaunchContext.setResource(recordFactory .newRecordInstance(Resource.class)); -// containerLaunchContext.command = new ArrayList(); - StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class); + StartContainerRequest startRequest = + recordFactory.newRecordInstance(StartContainerRequest.class); startRequest.setContainerLaunchContext(containerLaunchContext); containerManager.startContainer(startRequest); @@ -146,7 +157,7 @@ public void testContainerSetup() throws IOException, InterruptedException { ContainerState.COMPLETE); // Now ascertain that the resources are localised correctly. - // TODO: Don't we need clusterStamp in localDir? + ApplicationId appId = cId.getApplicationAttemptId().getApplicationId(); String appIDStr = ConverterUtils.toString(appId); String containerIDStr = ConverterUtils.toString(cId); File userCacheDir = new File(localDir, ContainerLocalizer.USERCACHE); @@ -186,41 +197,41 @@ public void testContainerLaunchAndStop() throws IOException, PrintWriter fileWriter = new PrintWriter(scriptFile); File processStartFile = new File(tmpDir, "start_file.txt").getAbsoluteFile(); - fileWriter.write("\numask 0"); // So that start file is readable by the test. + fileWriter.write("\numask 0"); // So that start file is readable by the test fileWriter.write("\necho Hello World! > " + processStartFile); fileWriter.write("\necho $$ >> " + processStartFile); fileWriter.write("\nexec sleep 100"); fileWriter.close(); - ContainerLaunchContext containerLaunchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class); + ContainerLaunchContext containerLaunchContext = + recordFactory.newRecordInstance(ContainerLaunchContext.class); // ////// Construct the Container-id - ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class); - ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class); - appAttemptId.setApplicationId(appId); - appAttemptId.setAttemptId(1); - ContainerId cId = recordFactory.newRecordInstance(ContainerId.class); - cId.setAppId(appId); - cId.setAppAttemptId(appAttemptId); + ContainerId cId = createContainerId(); containerLaunchContext.setContainerId(cId); containerLaunchContext.setUser(user); -// containerLaunchContext.resources =new HashMap(); URL resource_alpha = ConverterUtils.getYarnUrlFromPath(localFS .makeQualified(new Path(scriptFile.getAbsolutePath()))); - LocalResource rsrc_alpha = recordFactory.newRecordInstance(LocalResource.class); + LocalResource rsrc_alpha = + recordFactory.newRecordInstance(LocalResource.class); rsrc_alpha.setResource(resource_alpha); rsrc_alpha.setSize(-1); rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION); rsrc_alpha.setType(LocalResourceType.FILE); rsrc_alpha.setTimestamp(scriptFile.lastModified()); String destinationFile = "dest_file"; - containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha); + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setUser(containerLaunchContext.getUser()); - containerLaunchContext.addCommand("/bin/bash"); - containerLaunchContext.addCommand(scriptFile.getAbsolutePath()); + List commands = new ArrayList(); + commands.add("/bin/bash"); + commands.add(scriptFile.getAbsolutePath()); + containerLaunchContext.setCommands(commands); containerLaunchContext.setResource(recordFactory .newRecordInstance(Resource.class)); containerLaunchContext.getResource().setMemory(100 * 1024 * 1024); @@ -263,10 +274,12 @@ public void testContainerLaunchAndStop() throws IOException, BaseContainerManagerTest.waitForContainerState(containerManager, cId, ContainerState.COMPLETE); - GetContainerStatusRequest gcsRequest = recordFactory.newRecordInstance(GetContainerStatusRequest.class); + GetContainerStatusRequest gcsRequest = + recordFactory.newRecordInstance(GetContainerStatusRequest.class); gcsRequest.setContainerId(cId); - ContainerStatus containerStatus = containerManager.getContainerStatus(gcsRequest).getStatus(); - Assert.assertEquals(String.valueOf(ExitCode.KILLED.getExitCode()), + ContainerStatus containerStatus = + containerManager.getContainerStatus(gcsRequest).getStatus(); + Assert.assertEquals(ExitCode.KILLED.getExitCode(), containerStatus.getExitStatus()); // Assert that the process is not alive anymore @@ -281,8 +294,10 @@ public void testLocalFilesCleanup() throws InterruptedException, // Real del service delSrvc = new DeletionService(exec); delSrvc.init(conf); + ContainerTokenSecretManager containerTokenSecretManager = new + ContainerTokenSecretManager(); containerManager = new ContainerManagerImpl(context, exec, delSrvc, - nodeStatusUpdater, metrics); + nodeStatusUpdater, metrics, containerTokenSecretManager); containerManager.init(conf); containerManager.start(); @@ -297,13 +312,8 @@ public void testLocalFilesCleanup() throws InterruptedException, ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class); // ////// Construct the Container-id - ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class); - ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class); - appAttemptId.setApplicationId(appId); - appAttemptId.setAttemptId(1); - ContainerId cId = recordFactory.newRecordInstance(ContainerId.class); - cId.setAppId(appId); - cId.setAppAttemptId(appAttemptId); + ContainerId cId = createContainerId(); + ApplicationId appId = cId.getApplicationAttemptId().getApplicationId(); container.setContainerId(cId); container.setUser(user); @@ -322,7 +332,10 @@ public void testLocalFilesCleanup() throws InterruptedException, rsrc_alpha.setType(LocalResourceType.FILE); rsrc_alpha.setTimestamp(file.lastModified()); String destinationFile = "dest_file"; - containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha); + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setUser(container.getUser()); containerLaunchContext.setContainerId(container.getContainerId()); containerLaunchContext.setResource(recordFactory @@ -337,7 +350,8 @@ public void testLocalFilesCleanup() throws InterruptedException, BaseContainerManagerTest.waitForContainerState(containerManager, cId, ContainerState.COMPLETE); - BaseContainerManagerTest.waitForApplicationState(containerManager, cId.getAppId(), + BaseContainerManagerTest.waitForApplicationState(containerManager, + cId.getApplicationAttemptId().getApplicationId(), ApplicationState.RUNNING); // Now ascertain that the resources are localised correctly. @@ -369,7 +383,8 @@ public void testLocalFilesCleanup() throws InterruptedException, containerManager.handle(new CMgrCompletedAppsEvent(Arrays .asList(new ApplicationId[] { appId }))); - BaseContainerManagerTest.waitForApplicationState(containerManager, cId.getAppId(), + BaseContainerManagerTest.waitForApplicationState(containerManager, + cId.getApplicationAttemptId().getApplicationId(), ApplicationState.FINISHED); // Now ascertain that the resources are localised correctly. diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java index 2dd60b683f..04d400ad18 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java @@ -17,208 +17,203 @@ */ package org.apache.hadoop.yarn.server.nodemanager.containermanager.container; -import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.argThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + import java.net.URISyntaxException; - import java.nio.ByteBuffer; - +import java.util.AbstractMap.SimpleEntry; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; import java.util.Map; -import java.util.Random; import java.util.Map.Entry; -import java.util.AbstractMap.SimpleEntry; +import java.util.Random; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.URL; +import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.server.nodemanager.NodeManager; +import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent; - +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType; +import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import org.apache.hadoop.yarn.util.BuilderUtils; import org.junit.Test; -import static org.junit.Assert.*; - import org.mockito.ArgumentMatcher; -import static org.mockito.Mockito.*; public class TestContainer { final NodeManagerMetrics metrics = NodeManagerMetrics.create(); + /** * Verify correct container request events sent to localizer. */ @Test - @SuppressWarnings("unchecked") // mocked generic public void testLocalizationRequest() throws Exception { - DrainDispatcher dispatcher = new DrainDispatcher(); - dispatcher.init(null); + WrappedContainer wc = null; try { - dispatcher.start(); - EventHandler localizerBus = mock(EventHandler.class); - dispatcher.register(LocalizationEventType.class, localizerBus); - // null serviceData; no registered AuxServicesEventType handler - - ContainerLaunchContext ctxt = mock(ContainerLaunchContext.class); - ContainerId cId = getMockContainerId(7, 314159265358979L, 4344); - when(ctxt.getUser()).thenReturn("yak"); - when(ctxt.getContainerId()).thenReturn(cId); - - Random r = new Random(); - long seed = r.nextLong(); - r.setSeed(seed); - System.out.println("testLocalizationRequest seed: " + seed); - final Map localResources = createLocalResources(r); - when(ctxt.getAllLocalResources()).thenReturn(localResources); - - final Container c = newContainer(dispatcher, ctxt); - assertEquals(ContainerState.NEW, c.getContainerState()); + wc = new WrappedContainer(7, 314159265358979L, 4344, "yak"); + assertEquals(ContainerState.NEW, wc.c.getContainerState()); + wc.initContainer(); // Verify request for public/private resources to localizer - c.handle(new ContainerEvent(cId, ContainerEventType.INIT_CONTAINER)); - dispatcher.await(); - ContainerReqMatcher matchesPublicReq = - new ContainerReqMatcher(localResources, - EnumSet.of(LocalResourceVisibility.PUBLIC)); - ContainerReqMatcher matchesPrivateReq = - new ContainerReqMatcher(localResources, - EnumSet.of(LocalResourceVisibility.PRIVATE)); - ContainerReqMatcher matchesAppReq = - new ContainerReqMatcher(localResources, - EnumSet.of(LocalResourceVisibility.APPLICATION)); - verify(localizerBus).handle(argThat(matchesPublicReq)); - verify(localizerBus).handle(argThat(matchesPrivateReq)); - verify(localizerBus).handle(argThat(matchesAppReq)); - assertEquals(ContainerState.LOCALIZING, c.getContainerState()); - } finally { - dispatcher.stop(); + ResourcesRequestedMatcher matchesReq = + new ResourcesRequestedMatcher(wc.localResources, EnumSet.of( + LocalResourceVisibility.PUBLIC, LocalResourceVisibility.PRIVATE, + LocalResourceVisibility.APPLICATION)); + verify(wc.localizerBus).handle(argThat(matchesReq)); + assertEquals(ContainerState.LOCALIZING, wc.c.getContainerState()); } + finally { + if (wc != null) { + wc.finished(); + } + } } /** * Verify container launch when all resources already cached. */ @Test - @SuppressWarnings("unchecked") // mocked generic public void testLocalizationLaunch() throws Exception { - DrainDispatcher dispatcher = new DrainDispatcher(); - dispatcher.init(null); + WrappedContainer wc = null; try { - dispatcher.start(); - EventHandler localizerBus = mock(EventHandler.class); - dispatcher.register(LocalizationEventType.class, localizerBus); - EventHandler launcherBus = - mock(EventHandler.class); - dispatcher.register(ContainersLauncherEventType.class, launcherBus); - // null serviceData; no registered AuxServicesEventType handler - - ContainerLaunchContext ctxt = mock(ContainerLaunchContext.class); - ContainerId cId = getMockContainerId(8, 314159265358979L, 4344); - when(ctxt.getUser()).thenReturn("yak"); - when(ctxt.getContainerId()).thenReturn(cId); - - Random r = new Random(); - long seed = r.nextLong(); - r.setSeed(seed); - System.out.println("testLocalizationLaunch seed: " + seed); - final Map localResources = createLocalResources(r); - when(ctxt.getAllLocalResources()).thenReturn(localResources); - final Container c = newContainer(dispatcher, ctxt); - assertEquals(ContainerState.NEW, c.getContainerState()); - - c.handle(new ContainerEvent(cId, ContainerEventType.INIT_CONTAINER)); - dispatcher.await(); - - // Container prepared for localization events - Path cache = new Path("file:///cache"); - Map localPaths = new HashMap(); - for (Entry rsrc : localResources.entrySet()) { - assertEquals(ContainerState.LOCALIZING, c.getContainerState()); - LocalResourceRequest req = new LocalResourceRequest(rsrc.getValue()); - Path p = new Path(cache, rsrc.getKey()); - localPaths.put(p, rsrc.getKey()); - // rsrc copied to p - c.handle(new ContainerResourceLocalizedEvent(c.getContainerID(), req, p)); - } - dispatcher.await(); + wc = new WrappedContainer(8, 314159265358979L, 4344, "yak"); + assertEquals(ContainerState.NEW, wc.c.getContainerState()); + wc.initContainer(); + Map localPaths = wc.localizeResources(); // all resources should be localized - assertEquals(ContainerState.LOCALIZED, c.getContainerState()); - for (Entry loc : c.getLocalizedResources().entrySet()) { + assertEquals(ContainerState.LOCALIZED, wc.c.getContainerState()); + for (Entry loc : wc.c.getLocalizedResources().entrySet()) { assertEquals(localPaths.remove(loc.getKey()), loc.getValue()); } assertTrue(localPaths.isEmpty()); + final WrappedContainer wcf = wc; // verify container launch ArgumentMatcher matchesContainerLaunch = new ArgumentMatcher() { @Override public boolean matches(Object o) { ContainersLauncherEvent launchEvent = (ContainersLauncherEvent) o; - return c == launchEvent.getContainer(); + return wcf.c == launchEvent.getContainer(); } }; - verify(launcherBus).handle(argThat(matchesContainerLaunch)); + verify(wc.launcherBus).handle(argThat(matchesContainerLaunch)); } finally { - dispatcher.stop(); + if (wc != null) { + wc.finished(); + } } } + @Test + @SuppressWarnings("unchecked") // mocked generic + public void testCleanupOnFailure() throws Exception { + WrappedContainer wc = null; + try { + wc = new WrappedContainer(10, 314159265358979L, 4344, "yak"); + wc.initContainer(); + wc.localizeResources(); + wc.launchContainer(); + reset(wc.localizerBus); + wc.containerFailed(ExitCode.KILLED.getExitCode()); + assertEquals(ContainerState.EXITED_WITH_FAILURE, + wc.c.getContainerState()); + verifyCleanupCall(wc); + } + finally { + if (wc != null) { + wc.finished(); + } + } + } + + @Test + @SuppressWarnings("unchecked") // mocked generic + public void testCleanupOnSuccess() throws Exception { + WrappedContainer wc = null; + try { + wc = new WrappedContainer(11, 314159265358979L, 4344, "yak"); + wc.initContainer(); + wc.localizeResources(); + wc.launchContainer(); + reset(wc.localizerBus); + wc.containerSuccessful(); + assertEquals(ContainerState.EXITED_WITH_SUCCESS, + wc.c.getContainerState()); + + verifyCleanupCall(wc); + } + finally { + if (wc != null) { + wc.finished(); + } + } + } + + @Test + @SuppressWarnings("unchecked") // mocked generic + public void testCleanupOnKillRequest() throws Exception { + WrappedContainer wc = null; + try { + wc = new WrappedContainer(12, 314159265358979L, 4344, "yak"); + wc.initContainer(); + wc.localizeResources(); + wc.launchContainer(); + reset(wc.localizerBus); + wc.killContainer(); + assertEquals(ContainerState.KILLING, wc.c.getContainerState()); + wc.containerKilledOnRequest(); + + verifyCleanupCall(wc); + } finally { + if (wc != null) { + wc.finished(); + } + } + } + /** * Verify serviceData correctly sent. */ @Test - @SuppressWarnings("unchecked") // mocked generic public void testServiceData() throws Exception { - DrainDispatcher dispatcher = new DrainDispatcher(); - dispatcher.init(null); - dispatcher.start(); + WrappedContainer wc = null; try { - EventHandler localizerBus = mock(EventHandler.class); - dispatcher.register(LocalizationEventType.class, localizerBus); - EventHandler auxBus = mock(EventHandler.class); - dispatcher.register(AuxServicesEventType.class, auxBus); - EventHandler launchBus = mock(EventHandler.class); - dispatcher.register(ContainersLauncherEventType.class, launchBus); - - ContainerLaunchContext ctxt = mock(ContainerLaunchContext.class); - final ContainerId cId = getMockContainerId(9, 314159265358979L, 4344); - when(ctxt.getUser()).thenReturn("yak"); - when(ctxt.getContainerId()).thenReturn(cId); - when(ctxt.getAllLocalResources()).thenReturn( - Collections.emptyMap()); - - Random r = new Random(); - long seed = r.nextLong(); - r.setSeed(seed); - System.out.println("testServiceData seed: " + seed); - final Map serviceData = createServiceData(r); - when(ctxt.getAllServiceData()).thenReturn(serviceData); - - final Container c = newContainer(dispatcher, ctxt); - assertEquals(ContainerState.NEW, c.getContainerState()); - - // Verify propagation of service data to AuxServices - c.handle(new ContainerEvent(cId, ContainerEventType.INIT_CONTAINER)); - dispatcher.await(); - for (final Map.Entry e : serviceData.entrySet()) { + wc = new WrappedContainer(9, 314159265358979L, 4344, "yak", false, true); + assertEquals(ContainerState.NEW, wc.c.getContainerState()); + wc.initContainer(); + + for (final Map.Entry e : wc.serviceData.entrySet()) { ArgumentMatcher matchesServiceReq = new ArgumentMatcher() { @Override @@ -228,9 +223,10 @@ public boolean matches(Object o) { && 0 == e.getValue().compareTo(evt.getServiceData()); } }; - verify(auxBus).handle(argThat(matchesServiceReq)); + verify(wc.auxBus).handle(argThat(matchesServiceReq)); } + final WrappedContainer wcf = wc; // verify launch on empty resource request ArgumentMatcher matchesLaunchReq = new ArgumentMatcher() { @@ -238,61 +234,103 @@ public boolean matches(Object o) { public boolean matches(Object o) { ContainersLauncherEvent evt = (ContainersLauncherEvent) o; return evt.getType() == ContainersLauncherEventType.LAUNCH_CONTAINER - && cId == evt.getContainer().getContainerID(); + && wcf.cId == evt.getContainer().getContainerID(); } }; - verify(launchBus).handle(argThat(matchesLaunchReq)); + verify(wc.launcherBus).handle(argThat(matchesLaunchReq)); } finally { - dispatcher.stop(); + if (wc != null) { + wc.finished(); + } } } - // Accept iff the resource request payload matches. - static class ContainerReqMatcher extends ArgumentMatcher { + private void verifyCleanupCall(WrappedContainer wc) throws Exception { + ResourcesReleasedMatcher matchesReq = + new ResourcesReleasedMatcher(wc.localResources, EnumSet.of( + LocalResourceVisibility.PUBLIC, LocalResourceVisibility.PRIVATE, + LocalResourceVisibility.APPLICATION)); + verify(wc.localizerBus).handle(argThat(matchesReq)); + } + + private static class ResourcesReleasedMatcher extends + ArgumentMatcher { final HashSet resources = - new HashSet(); - ContainerReqMatcher(Map allResources, + new HashSet(); + + ResourcesReleasedMatcher(Map allResources, EnumSet vis) throws URISyntaxException { - for (Entry e : allResources.entrySet()) { + for (Entry e : allResources.entrySet()) { if (vis.contains(e.getValue().getVisibility())) { resources.add(new LocalResourceRequest(e.getValue())); } } } + @Override public boolean matches(Object o) { - ContainerLocalizationRequestEvent evt = (ContainerLocalizationRequestEvent) o; + if (!(o instanceof ContainerLocalizationCleanupEvent)) { + return false; + } + ContainerLocalizationCleanupEvent evt = + (ContainerLocalizationCleanupEvent) o; final HashSet expected = - new HashSet(resources); - for (LocalResourceRequest rsrc : evt.getRequestedResources()) { - if (!expected.remove(rsrc)) { - return false; + new HashSet(resources); + for (Collection rc : evt.getResources().values()) { + for (LocalResourceRequest rsrc : rc) { + if (!expected.remove(rsrc)) { + return false; + } } } return expected.isEmpty(); } } - static Entry getMockRsrc(Random r, - LocalResourceVisibility vis) { - LocalResource rsrc = mock(LocalResource.class); + // Accept iff the resource payload matches. + private static class ResourcesRequestedMatcher extends + ArgumentMatcher { + final HashSet resources = + new HashSet(); - String name = Long.toHexString(r.nextLong()); - URL uri = mock(org.apache.hadoop.yarn.api.records.URL.class); - when(uri.getScheme()).thenReturn("file"); - when(uri.getHost()).thenReturn(null); - when(uri.getFile()).thenReturn("/local/" + vis + "/" + name); + ResourcesRequestedMatcher(Map allResources, + EnumSet vis) throws URISyntaxException { + for (Entry e : allResources.entrySet()) { + if (vis.contains(e.getValue().getVisibility())) { + resources.add(new LocalResourceRequest(e.getValue())); + } + } + } - when(rsrc.getResource()).thenReturn(uri); - when(rsrc.getSize()).thenReturn(r.nextInt(1024) + 1024L); - when(rsrc.getTimestamp()).thenReturn(r.nextInt(1024) + 2048L); - when(rsrc.getType()).thenReturn(LocalResourceType.FILE); - when(rsrc.getVisibility()).thenReturn(vis); - - return new SimpleEntry(name, rsrc); + @Override + public boolean matches(Object o) { + ContainerLocalizationRequestEvent evt = + (ContainerLocalizationRequestEvent) o; + final HashSet expected = + new HashSet(resources); + for (Collection rc : evt.getRequestedResources() + .values()) { + for (LocalResourceRequest rsrc : rc) { + if (!expected.remove(rsrc)) { + return false; + } + } + } + return expected.isEmpty(); + } } - static Map createLocalResources(Random r) { + private static Entry getMockRsrc(Random r, + LocalResourceVisibility vis) { + String name = Long.toHexString(r.nextLong()); + URL url = BuilderUtils.newURL("file", null, 0, "/local" + vis + "/" + name); + LocalResource rsrc = + BuilderUtils.newLocalResource(url, LocalResourceType.FILE, vis, + r.nextInt(1024) + 1024L, r.nextInt(1024) + 2048L); + return new SimpleEntry(name, rsrc); + } + + private static Map createLocalResources(Random r) { Map localResources = new HashMap(); for (int i = r.nextInt(5) + 5; i >= 0; --i) { @@ -313,17 +351,7 @@ static Map createLocalResources(Random r) { return localResources; } - static ContainerId getMockContainerId(int appId, long timestamp, int id) { - ApplicationId aId = mock(ApplicationId.class); - when(aId.getId()).thenReturn(appId); - when(aId.getClusterTimestamp()).thenReturn(timestamp); - ContainerId cId = mock(ContainerId.class); - when(cId.getId()).thenReturn(id); - when(cId.getAppId()).thenReturn(aId); - return cId; - } - - static Map createServiceData(Random r) { + private static Map createServiceData(Random r) { Map serviceData = new HashMap(); for (int i = r.nextInt(5) + 5; i >= 0; --i) { @@ -335,7 +363,134 @@ static Map createServiceData(Random r) { return serviceData; } - Container newContainer(Dispatcher disp, ContainerLaunchContext ctx) { + private Container newContainer(Dispatcher disp, ContainerLaunchContext ctx) { return new ContainerImpl(disp, ctx, null, metrics); } + + @SuppressWarnings("unchecked") + private class WrappedContainer { + final DrainDispatcher dispatcher; + final EventHandler localizerBus; + final EventHandler launcherBus; + final EventHandler monitorBus; + final EventHandler auxBus; + + final ContainerLaunchContext ctxt; + final ContainerId cId; + final Container c; + final Map localResources; + final Map serviceData; + final String user; + + WrappedContainer(int appId, long timestamp, int id, String user) { + this(appId, timestamp, id, user, true, false); + } + + WrappedContainer(int appId, long timestamp, int id, String user, + boolean withLocalRes, boolean withServiceData) { + dispatcher = new DrainDispatcher(); + dispatcher.init(null); + + localizerBus = mock(EventHandler.class); + launcherBus = mock(EventHandler.class); + monitorBus = mock(EventHandler.class); + auxBus = mock(EventHandler.class); + dispatcher.register(LocalizationEventType.class, localizerBus); + dispatcher.register(ContainersLauncherEventType.class, launcherBus); + dispatcher.register(ContainersMonitorEventType.class, monitorBus); + dispatcher.register(AuxServicesEventType.class, auxBus); + this.user = user; + + ctxt = mock(ContainerLaunchContext.class); + cId = BuilderUtils.newContainerId(appId, 1, timestamp, id); + when(ctxt.getUser()).thenReturn(this.user); + when(ctxt.getContainerId()).thenReturn(cId); + + Resource resource = BuilderUtils.newResource(1024); + when(ctxt.getResource()).thenReturn(resource); + + if (withLocalRes) { + Random r = new Random(); + long seed = r.nextLong(); + r.setSeed(seed); + System.out.println("WrappedContainerLocalResource seed: " + seed); + localResources = createLocalResources(r); + } else { + localResources = Collections. emptyMap(); + } + when(ctxt.getLocalResources()).thenReturn(localResources); + + if (withServiceData) { + Random r = new Random(); + long seed = r.nextLong(); + r.setSeed(seed); + System.out.println("ServiceData seed: " + seed); + serviceData = createServiceData(r); + } else { + serviceData = Collections. emptyMap(); + } + when(ctxt.getServiceData()).thenReturn(serviceData); + + c = newContainer(dispatcher, ctxt); + dispatcher.start(); + } + + private void drainDispatcherEvents() { + dispatcher.await(); + } + + public void finished() { + dispatcher.stop(); + } + + public void initContainer() { + c.handle(new ContainerEvent(cId, ContainerEventType.INIT_CONTAINER)); + drainDispatcherEvents(); + } + + public Map localizeResources() throws URISyntaxException { + Path cache = new Path("file:///cache"); + Map localPaths = new HashMap(); + for (Entry rsrc : localResources.entrySet()) { + assertEquals(ContainerState.LOCALIZING, c.getContainerState()); + LocalResourceRequest req = new LocalResourceRequest(rsrc.getValue()); + Path p = new Path(cache, rsrc.getKey()); + localPaths.put(p, rsrc.getKey()); + // rsrc copied to p + c.handle(new ContainerResourceLocalizedEvent(c.getContainerID(), + req, p)); + } + drainDispatcherEvents(); + return localPaths; + } + + public void launchContainer() { + c.handle(new ContainerEvent(cId, ContainerEventType.CONTAINER_LAUNCHED)); + drainDispatcherEvents(); + } + + public void containerSuccessful() { + c.handle(new ContainerEvent(cId, + ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS)); + drainDispatcherEvents(); + } + + public void containerFailed(int exitCode) { + c.handle(new ContainerExitEvent(cId, + ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, exitCode)); + drainDispatcherEvents(); + } + + public void killContainer() { + c.handle(new ContainerKillEvent(cId, "KillRequest")); + drainDispatcherEvents(); + } + + public void containerKilledOnRequest() { + c.handle(new ContainerExitEvent(cId, + ContainerEventType.CONTAINER_KILLED_ON_REQUEST, ExitCode.KILLED + .getExitCode())); + drainDispatcherEvents(); + } + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalizedResource.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalizedResource.java index 2a9f44530a..a1c6bb8479 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalizedResource.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalizedResource.java @@ -19,6 +19,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.LocalResource; @@ -48,9 +49,12 @@ static ContainerId getMockContainer(int id) { ApplicationId appId = mock(ApplicationId.class); when(appId.getClusterTimestamp()).thenReturn(314159265L); when(appId.getId()).thenReturn(3); + ApplicationAttemptId appAttemptId = mock(ApplicationAttemptId.class); + when(appAttemptId.getApplicationId()).thenReturn(appId); + when(appAttemptId.getAttemptId()).thenReturn(0); ContainerId container = mock(ContainerId.class); when(container.getId()).thenReturn(id); - when(container.getAppId()).thenReturn(appId); + when(container.getApplicationAttemptId()).thenReturn(appAttemptId); return container; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java index c64f41416e..b0174ad06e 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java @@ -21,10 +21,17 @@ import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Random; +import java.util.Set; + +import junit.framework.Assert; import org.apache.avro.ipc.Server; import org.apache.hadoop.conf.Configuration; @@ -46,6 +53,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.URL; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.EventHandler; @@ -62,11 +70,15 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.LocalizerTracker; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerEventType; +import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.hadoop.yarn.util.Records; import org.junit.Test; import static org.junit.Assert.*; @@ -74,8 +86,6 @@ import org.mockito.ArgumentMatcher; import static org.mockito.Mockito.*; -import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOCAL_DIR; - public class TestResourceLocalizationService { static final Path basedir = @@ -110,7 +120,7 @@ public void testLocalizationInit() throws Exception { localDirs.add(lfs.makeQualified(new Path(basedir, i + ""))); sDirs[i] = localDirs.get(i).toString(); } - conf.setStrings(NM_LOCAL_DIR, sDirs); + conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS, sDirs); // initialize ResourceLocalizationService locService.init(conf); @@ -133,6 +143,190 @@ public void testLocalizationInit() throws Exception { } } + @Test + @SuppressWarnings("unchecked") // mocked generics + public void testResourceRelease() throws Exception { + Configuration conf = new Configuration(); + AbstractFileSystem spylfs = + spy(FileContext.getLocalFSFileContext().getDefaultFileSystem()); + final FileContext lfs = FileContext.getFileContext(spylfs, conf); + doNothing().when(spylfs).mkdir( + isA(Path.class), isA(FsPermission.class), anyBoolean()); + + List localDirs = new ArrayList(); + String[] sDirs = new String[4]; + for (int i = 0; i < 4; ++i) { + localDirs.add(lfs.makeQualified(new Path(basedir, i + ""))); + sDirs[i] = localDirs.get(i).toString(); + } + conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS, sDirs); + + Server ignore = mock(Server.class); + LocalizerTracker mockLocallilzerTracker = mock(LocalizerTracker.class); + DrainDispatcher dispatcher = new DrainDispatcher(); + dispatcher.init(conf); + dispatcher.start(); + EventHandler applicationBus = mock(EventHandler.class); + dispatcher.register(ApplicationEventType.class, applicationBus); + EventHandler containerBus = mock(EventHandler.class); + dispatcher.register(ContainerEventType.class, containerBus); + //Ignore actual localization + EventHandler localizerBus = mock(EventHandler.class); + dispatcher.register(LocalizerEventType.class, localizerBus); + + ContainerExecutor exec = mock(ContainerExecutor.class); + DeletionService delService = new DeletionService(exec); + delService.init(null); + delService.start(); + + ResourceLocalizationService rawService = + new ResourceLocalizationService(dispatcher, exec, delService); + ResourceLocalizationService spyService = spy(rawService); + doReturn(ignore).when(spyService).createServer(); + doReturn(mockLocallilzerTracker).when(spyService).createLocalizerTracker( + isA(Configuration.class)); + doReturn(lfs).when(spyService) + .getLocalFileContext(isA(Configuration.class)); + try { + spyService.init(conf); + spyService.start(); + + final String user = "user0"; + // init application + final Application app = mock(Application.class); + final ApplicationId appId = + BuilderUtils.newApplicationId(314159265358979L, 3); + when(app.getUser()).thenReturn(user); + when(app.getAppId()).thenReturn(appId); + spyService.handle(new ApplicationLocalizationEvent( + LocalizationEventType.INIT_APPLICATION_RESOURCES, app)); + dispatcher.await(); + + //Get a handle on the trackers after they're setup with INIT_APP_RESOURCES + LocalResourcesTracker appTracker = + spyService.getLocalResourcesTracker( + LocalResourceVisibility.APPLICATION, user, appId); + LocalResourcesTracker privTracker = + spyService.getLocalResourcesTracker(LocalResourceVisibility.PRIVATE, + user, appId); + LocalResourcesTracker pubTracker = + spyService.getLocalResourcesTracker(LocalResourceVisibility.PUBLIC, + user, appId); + + // init container. + final Container c = getMockContainer(appId, 42); + + // init resources + Random r = new Random(); + long seed = r.nextLong(); + System.out.println("SEED: " + seed); + r.setSeed(seed); + + // Send localization requests for one resource of each type. + final LocalResource privResource = getPrivateMockedResource(r); + final LocalResourceRequest privReq = + new LocalResourceRequest(privResource); + + final LocalResource pubResource = getPublicMockedResource(r); + final LocalResourceRequest pubReq = new LocalResourceRequest(pubResource); + final LocalResource pubResource2 = getPublicMockedResource(r); + final LocalResourceRequest pubReq2 = + new LocalResourceRequest(pubResource2); + + final LocalResource appResource = getAppMockedResource(r); + final LocalResourceRequest appReq = new LocalResourceRequest(appResource); + + Map> req = + new HashMap>(); + req.put(LocalResourceVisibility.PRIVATE, + Collections.singletonList(privReq)); + req.put(LocalResourceVisibility.PUBLIC, + Collections.singletonList(pubReq)); + req.put(LocalResourceVisibility.APPLICATION, + Collections.singletonList(appReq)); + + Map> req2 = + new HashMap>(); + req2.put(LocalResourceVisibility.PRIVATE, + Collections.singletonList(privReq)); + req2.put(LocalResourceVisibility.PUBLIC, + Collections.singletonList(pubReq2)); + + Set pubRsrcs = new HashSet(); + pubRsrcs.add(pubReq); + pubRsrcs.add(pubReq2); + + // Send Request event + spyService.handle(new ContainerLocalizationRequestEvent(c, req)); + spyService.handle(new ContainerLocalizationRequestEvent(c, req2)); + dispatcher.await(); + + int privRsrcCount = 0; + for (LocalizedResource lr : privTracker) { + privRsrcCount++; + Assert.assertEquals("Incorrect reference count", 2, lr.getRefCount()); + Assert.assertEquals(privReq, lr.getRequest()); + } + Assert.assertEquals(1, privRsrcCount); + + int pubRsrcCount = 0; + for (LocalizedResource lr : pubTracker) { + pubRsrcCount++; + Assert.assertEquals("Incorrect reference count", 1, lr.getRefCount()); + pubRsrcs.remove(lr.getRequest()); + } + Assert.assertEquals(0, pubRsrcs.size()); + Assert.assertEquals(2, pubRsrcCount); + + int appRsrcCount = 0; + for (LocalizedResource lr : appTracker) { + appRsrcCount++; + Assert.assertEquals("Incorrect reference count", 1, lr.getRefCount()); + Assert.assertEquals(appReq, lr.getRequest()); + } + Assert.assertEquals(1, appRsrcCount); + + //Send Cleanup Event + spyService.handle(new ContainerLocalizationCleanupEvent(c, req)); + req2.remove(LocalResourceVisibility.PRIVATE); + spyService.handle(new ContainerLocalizationCleanupEvent(c, req2)); + dispatcher.await(); + + pubRsrcs.add(pubReq); + pubRsrcs.add(pubReq2); + + privRsrcCount = 0; + for (LocalizedResource lr : privTracker) { + privRsrcCount++; + Assert.assertEquals("Incorrect reference count", 1, lr.getRefCount()); + Assert.assertEquals(privReq, lr.getRequest()); + } + Assert.assertEquals(1, privRsrcCount); + + pubRsrcCount = 0; + for (LocalizedResource lr : pubTracker) { + pubRsrcCount++; + Assert.assertEquals("Incorrect reference count", 0, lr.getRefCount()); + pubRsrcs.remove(lr.getRequest()); + } + Assert.assertEquals(0, pubRsrcs.size()); + Assert.assertEquals(2, pubRsrcCount); + + appRsrcCount = 0; + for (LocalizedResource lr : appTracker) { + appRsrcCount++; + Assert.assertEquals("Incorrect reference count", 0, lr.getRefCount()); + Assert.assertEquals(appReq, lr.getRequest()); + } + Assert.assertEquals(1, appRsrcCount); + } finally { + dispatcher.stop(); + delService.stop(); + } + } + @Test @SuppressWarnings("unchecked") // mocked generics public void testLocalizationHeartbeat() throws Exception { @@ -149,7 +343,7 @@ public void testLocalizationHeartbeat() throws Exception { localDirs.add(lfs.makeQualified(new Path(basedir, i + ""))); sDirs[i] = localDirs.get(i).toString(); } - conf.setStrings(NM_LOCAL_DIR, sDirs); + conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS, sDirs); Server ignore = mock(Server.class); DrainDispatcher dispatcher = new DrainDispatcher(); @@ -176,9 +370,8 @@ public void testLocalizationHeartbeat() throws Exception { // init application final Application app = mock(Application.class); - final ApplicationId appId = mock(ApplicationId.class); - when(appId.getClusterTimestamp()).thenReturn(314159265358979L); - when(appId.getId()).thenReturn(3); + final ApplicationId appId = + BuilderUtils.newApplicationId(314159265358979L, 3); when(app.getUser()).thenReturn("user0"); when(app.getAppId()).thenReturn(appId); spyService.handle(new ApplicationLocalizationEvent( @@ -206,11 +399,13 @@ public boolean matches(Object o) { doReturn(out).when(spylfs).createInternal(isA(Path.class), isA(EnumSet.class), isA(FsPermission.class), anyInt(), anyShort(), anyLong(), isA(Progressable.class), anyInt(), anyBoolean()); - final LocalResource resource = getMockResource(r); + final LocalResource resource = getPrivateMockedResource(r); final LocalResourceRequest req = new LocalResourceRequest(resource); - spyService.handle(new ContainerLocalizationRequestEvent( - c, Collections.singletonList(req), - LocalResourceVisibility.PRIVATE)); + Map> rsrcs = + new HashMap>(); + rsrcs.put(LocalResourceVisibility.PRIVATE, Collections.singletonList(req)); + spyService.handle(new ContainerLocalizationRequestEvent(c, rsrcs)); // Sigh. Thread init of private localizer not accessible Thread.sleep(500); dispatcher.await(); @@ -266,42 +461,44 @@ public boolean matches(Object o) { } } - static URL getPath(String path) { - URL uri = mock(org.apache.hadoop.yarn.api.records.URL.class); - when(uri.getScheme()).thenReturn("file"); - when(uri.getHost()).thenReturn(null); - when(uri.getFile()).thenReturn(path); - return uri; + private static URL getPath(String path) { + URL url = BuilderUtils.newURL("file", null, 0, path); + return url; } - static LocalResource getMockResource(Random r) { - LocalResource rsrc = mock(LocalResource.class); - + private static LocalResource getMockedResource(Random r, + LocalResourceVisibility vis) { String name = Long.toHexString(r.nextLong()); - URL uri = getPath("/local/PRIVATE/" + name); - - when(rsrc.getResource()).thenReturn(uri); - when(rsrc.getSize()).thenReturn(r.nextInt(1024) + 1024L); - when(rsrc.getTimestamp()).thenReturn(r.nextInt(1024) + 2048L); - when(rsrc.getType()).thenReturn(LocalResourceType.FILE); - when(rsrc.getVisibility()).thenReturn(LocalResourceVisibility.PRIVATE); + URL url = getPath("/local/PRIVATE/" + name); + LocalResource rsrc = + BuilderUtils.newLocalResource(url, LocalResourceType.FILE, vis, + r.nextInt(1024) + 1024L, r.nextInt(1024) + 2048L); return rsrc; } + + private static LocalResource getAppMockedResource(Random r) { + return getMockedResource(r, LocalResourceVisibility.APPLICATION); + } + + private static LocalResource getPublicMockedResource(Random r) { + return getMockedResource(r, LocalResourceVisibility.PUBLIC); + } + + private static LocalResource getPrivateMockedResource(Random r) { + return getMockedResource(r, LocalResourceVisibility.PRIVATE); + } - static Container getMockContainer(ApplicationId appId, int id) { + private static Container getMockContainer(ApplicationId appId, int id) { Container c = mock(Container.class); - ApplicationAttemptId appAttemptId = Records.newRecord(ApplicationAttemptId.class); - appAttemptId.setApplicationId(appId); - appAttemptId.setAttemptId(1); - ContainerId cId = Records.newRecord(ContainerId.class); - cId.setAppAttemptId(appAttemptId); - cId.setAppId(appId); - cId.setId(id); + ApplicationAttemptId appAttemptId = + BuilderUtils.newApplicationAttemptId(appId, 1); + ContainerId cId = BuilderUtils.newContainerId(appAttemptId, id); when(c.getUser()).thenReturn("user0"); when(c.getContainerID()).thenReturn(cId); Credentials creds = new Credentials(); creds.addToken(new Text("tok" + id), getToken(id)); when(c.getCredentials()).thenReturn(creds); + when(c.toString()).thenReturn(cId.toString()); return c; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java index 7a39374bc7..164039e037 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java @@ -25,8 +25,10 @@ import java.io.IOException; import java.io.PrintWriter; import java.io.Writer; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import junit.framework.Assert; @@ -47,11 +49,11 @@ import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.URL; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogKey; import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogReader; @@ -92,8 +94,8 @@ public void tearDown() throws IOException, InterruptedException { public void testLocalFileDeletionAfterUpload() throws IOException { this.delSrvc = new DeletionService(createContainerExecutor()); this.delSrvc.init(conf); - this.conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath()); - this.conf.set(NMConfig.REMOTE_USER_LOG_DIR, + this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath()); + this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, this.remoteRootLogDir.getAbsolutePath()); LogAggregationService logAggregationService = new LogAggregationService(this.delSrvc); @@ -118,8 +120,8 @@ public void testLocalFileDeletionAfterUpload() throws IOException { BuilderUtils.newContainerId(recordFactory, application1, appAttemptId, 1); // Simulate log-file creation writeContainerLogs(app1LogDir, container11); - logAggregationService.handle(new LogAggregatorContainerFinishedEvent( - container11, "0")); + logAggregationService.handle( + new LogAggregatorContainerFinishedEvent(container11, 0)); logAggregationService.handle(new LogAggregatorAppFinishedEvent( application1)); @@ -140,8 +142,8 @@ public void testLocalFileDeletionAfterUpload() throws IOException { @Test public void testNoContainerOnNode() { - this.conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath()); - this.conf.set(NMConfig.REMOTE_USER_LOG_DIR, + this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath()); + this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, this.remoteRootLogDir.getAbsolutePath()); LogAggregationService logAggregationService = new LogAggregationService(this.delSrvc); @@ -173,8 +175,8 @@ public void testNoContainerOnNode() { @Test public void testMultipleAppsLogAggregation() throws IOException { - this.conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath()); - this.conf.set(NMConfig.REMOTE_USER_LOG_DIR, + this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath()); + this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, this.remoteRootLogDir.getAbsolutePath()); LogAggregationService logAggregationService = new LogAggregationService(this.delSrvc); @@ -192,17 +194,19 @@ public void testMultipleAppsLogAggregation() throws IOException { application1, this.user, null, ContainerLogsRetentionPolicy.ALL_CONTAINERS)); - ApplicationAttemptId appAttemptId1 = recordFactory.newRecordInstance(ApplicationAttemptId.class); + ApplicationAttemptId appAttemptId1 = + recordFactory.newRecordInstance(ApplicationAttemptId.class); appAttemptId1.setApplicationId(application1); ContainerId container11 = BuilderUtils.newContainerId(recordFactory, application1, appAttemptId1, 1); // Simulate log-file creation writeContainerLogs(app1LogDir, container11); - logAggregationService.handle(new LogAggregatorContainerFinishedEvent( - container11, "0")); + logAggregationService.handle( + new LogAggregatorContainerFinishedEvent(container11, 0)); ApplicationId application2 = BuilderUtils.newApplicationId(1234, 2); - ApplicationAttemptId appAttemptId2 = recordFactory.newRecordInstance(ApplicationAttemptId.class); + ApplicationAttemptId appAttemptId2 = + recordFactory.newRecordInstance(ApplicationAttemptId.class); appAttemptId1.setApplicationId(application2); File app2LogDir = @@ -214,19 +218,22 @@ public void testMultipleAppsLogAggregation() throws IOException { ContainerId container21 = - BuilderUtils.newContainerId(recordFactory, application2, appAttemptId2, 1); + BuilderUtils.newContainerId(recordFactory, application2, + appAttemptId2, 1); writeContainerLogs(app2LogDir, container21); - logAggregationService.handle(new LogAggregatorContainerFinishedEvent( - container21, "0")); + logAggregationService.handle( + new LogAggregatorContainerFinishedEvent(container21, 0)); ContainerId container12 = - BuilderUtils.newContainerId(recordFactory, application1, appAttemptId1, 2); + BuilderUtils.newContainerId(recordFactory, application1, appAttemptId1, + 2); writeContainerLogs(app1LogDir, container12); - logAggregationService.handle(new LogAggregatorContainerFinishedEvent( - container12, "0")); + logAggregationService.handle( + new LogAggregatorContainerFinishedEvent(container12, 0)); ApplicationId application3 = BuilderUtils.newApplicationId(1234, 3); - ApplicationAttemptId appAttemptId3 = recordFactory.newRecordInstance(ApplicationAttemptId.class); + ApplicationAttemptId appAttemptId3 = + recordFactory.newRecordInstance(ApplicationAttemptId.class); appAttemptId1.setApplicationId(application3); File app3LogDir = @@ -237,28 +244,32 @@ public void testMultipleAppsLogAggregation() throws IOException { ContainerLogsRetentionPolicy.AM_AND_FAILED_CONTAINERS_ONLY)); ContainerId container31 = - BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, 1); + BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, + 1); writeContainerLogs(app3LogDir, container31); - logAggregationService.handle(new LogAggregatorContainerFinishedEvent( - container31, "0")); + logAggregationService.handle( + new LogAggregatorContainerFinishedEvent(container31, 0)); ContainerId container32 = - BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, 2); + BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, + 2); writeContainerLogs(app3LogDir, container32); - logAggregationService.handle(new LogAggregatorContainerFinishedEvent( - container32, "1")); // Failed container + logAggregationService.handle( + new LogAggregatorContainerFinishedEvent(container32, 1)); // Failed ContainerId container22 = - BuilderUtils.newContainerId(recordFactory, application2, appAttemptId2, 2); + BuilderUtils.newContainerId(recordFactory, application2, appAttemptId2, + 2); writeContainerLogs(app2LogDir, container22); - logAggregationService.handle(new LogAggregatorContainerFinishedEvent( - container22, "0")); + logAggregationService.handle( + new LogAggregatorContainerFinishedEvent(container22, 0)); ContainerId container33 = - BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, 3); + BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, + 3); writeContainerLogs(app3LogDir, container33); - logAggregationService.handle(new LogAggregatorContainerFinishedEvent( - container33, "0")); + logAggregationService.handle( + new LogAggregatorContainerFinishedEvent(container33, 0)); logAggregationService.handle(new LogAggregatorAppFinishedEvent( application2)); @@ -387,8 +398,15 @@ public void testLogAggregationForRealContainerLaunch() throws IOException, // ////// Construct the Container-id ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class); + appId.setClusterTimestamp(0); + appId.setId(0); + ApplicationAttemptId appAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); + appAttemptId.setApplicationId(appId); + appAttemptId.setAttemptId(1); ContainerId cId = recordFactory.newRecordInstance(ContainerId.class); - cId.setAppId(appId); + cId.setId(0); + cId.setApplicationAttemptId(appAttemptId); containerLaunchContext.setContainerId(cId); containerLaunchContext.setUser(this.user); @@ -404,10 +422,15 @@ public void testLogAggregationForRealContainerLaunch() throws IOException, rsrc_alpha.setType(LocalResourceType.FILE); rsrc_alpha.setTimestamp(scriptFile.lastModified()); String destinationFile = "dest_file"; - containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha); + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setUser(containerLaunchContext.getUser()); - containerLaunchContext.addCommand("/bin/bash"); - containerLaunchContext.addCommand(scriptFile.getAbsolutePath()); + List commands = new ArrayList(); + commands.add("/bin/bash"); + commands.add(scriptFile.getAbsolutePath()); + containerLaunchContext.setCommands(commands); containerLaunchContext.setResource(recordFactory .newRecordInstance(Resource.class)); containerLaunchContext.getResource().setMemory(100 * 1024 * 1024); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java index c884cb4638..4a2a49c815 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java @@ -26,6 +26,10 @@ import java.io.FileReader; import java.io.IOException; import java.io.PrintWriter; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.regex.Pattern; import junit.framework.Assert; @@ -47,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.URL; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal; import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; @@ -70,7 +75,7 @@ public TestContainersMonitor() throws UnsupportedFileSystemException { @Before public void setup() throws IOException { conf.setClass( - ContainersMonitorImpl.RESOURCE_CALCULATOR_PLUGIN_CONFIG_KEY, + YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR, LinuxResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class); super.setup(); } @@ -191,13 +196,15 @@ public void testContainerKillOnMemoryOverflow() throws IOException, // ////// Construct the Container-id ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class); - ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class); + appId.setClusterTimestamp(0); + appId.setId(0); + ApplicationAttemptId appAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); appAttemptId.setApplicationId(appId); appAttemptId.setAttemptId(1); ContainerId cId = recordFactory.newRecordInstance(ContainerId.class); - cId.setAppId(appId); cId.setId(0); - cId.setAppAttemptId(appAttemptId); + cId.setApplicationAttemptId(appAttemptId); containerLaunchContext.setContainerId(cId); containerLaunchContext.setUser(user); @@ -213,10 +220,15 @@ public void testContainerKillOnMemoryOverflow() throws IOException, rsrc_alpha.setType(LocalResourceType.FILE); rsrc_alpha.setTimestamp(scriptFile.lastModified()); String destinationFile = "dest_file"; - containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha); + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setUser(containerLaunchContext.getUser()); - containerLaunchContext.addCommand("/bin/bash"); - containerLaunchContext.addCommand(scriptFile.getAbsolutePath()); + List commands = new ArrayList(); + commands.add("/bin/bash"); + commands.add(scriptFile.getAbsolutePath()); + containerLaunchContext.setCommands(commands); containerLaunchContext.setResource(recordFactory .newRecordInstance(Resource.class)); containerLaunchContext.getResource().setMemory(8 * 1024 * 1024); @@ -250,7 +262,7 @@ public void testContainerKillOnMemoryOverflow() throws IOException, gcsRequest.setContainerId(cId); ContainerStatus containerStatus = containerManager.getContainerStatus(gcsRequest).getStatus(); - Assert.assertEquals(String.valueOf(ExitCode.KILLED.getExitCode()), + Assert.assertEquals(ExitCode.KILLED.getExitCode(), containerStatus.getExitStatus()); String expectedMsgPattern = "Container \\[pid=" + pid + ",containerID=" + cId diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java index 7ad19d2ad1..f84af413bc 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java @@ -28,12 +28,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +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.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.nodemanager.Context; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.NodeManager; import org.apache.hadoop.yarn.server.nodemanager.ResourceView; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; @@ -72,7 +72,7 @@ public long getPmemAllocatedForContainers() { }; WebServer server = new WebServer(nmContext, resourceView); Configuration conf = new Configuration(); - conf.set(NMConfig.NM_LOCAL_DIR, testRootDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath()); server.init(conf); server.start(); @@ -112,7 +112,9 @@ public ContainerState getContainerState() { }; nmContext.getContainers().put(containerId, container); //TODO: Gross hack. Fix in code. - nmContext.getApplications().get(containerId.getAppId()).getContainers() + ApplicationId applicationId = + containerId.getApplicationAttemptId().getApplicationId(); + nmContext.getApplications().get(applicationId).getContainers() .put(containerId, container); writeContainerLogs(conf, nmContext, containerId); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java index 2ec3c59497..2fd8eb3b17 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/security/admin/AdminSecurityInfo.java @@ -43,7 +43,7 @@ public Class annotationType() { @Override public String serverPrincipal() { - return YarnConfiguration.RM_SERVER_PRINCIPAL_KEY; + return YarnConfiguration.RM_PRINCIPAL; } @Override diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java index 01508a2142..1fc34f0dfd 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java @@ -83,12 +83,12 @@ public AdminService(Configuration conf, ResourceScheduler scheduler, public void init(Configuration conf) { super.init(conf); String bindAddress = - conf.get(RMConfig.ADMIN_ADDRESS, - RMConfig.DEFAULT_ADMIN_BIND_ADDRESS); + conf.get(YarnConfiguration.RM_ADMIN_ADDRESS, + YarnConfiguration.RM_ADMIN_ADDRESS); masterServiceAddress = NetUtils.createSocketAddr(bindAddress); adminAcl = new AccessControlList( - conf.get(RMConfig.RM_ADMIN_ACL, RMConfig.DEFAULT_RM_ADMIN_ACL)); + conf.get(YarnConfiguration.RM_ADMIN_ACL, YarnConfiguration.DEFAULT_RM_ADMIN_ACL)); } public void start() { @@ -100,8 +100,8 @@ public void start() { this.server = rpc.getServer(RMAdminProtocol.class, this, masterServiceAddress, serverConf, null, - serverConf.getInt(RMConfig.RM_ADMIN_THREADS, - RMConfig.DEFAULT_RM_ADMIN_THREADS)); + serverConf.getInt(YarnConfiguration.RM_ADMIN_CLIENT_THREAD_COUNT, + YarnConfiguration.DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT)); this.server.start(); super.start(); } @@ -219,7 +219,7 @@ public RefreshAdminAclsResponse refreshAdminAcls( Configuration conf = new Configuration(); adminAcl = new AccessControlList( - conf.get(RMConfig.RM_ADMIN_ACL, RMConfig.DEFAULT_RM_ADMIN_ACL)); + conf.get(YarnConfiguration.RM_ADMIN_ACL, YarnConfiguration.DEFAULT_RM_ADMIN_ACL)); RMAuditLogger.logSuccess(user.getShortUserName(), "refreshAdminAcls", "AdminService"); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationACLsManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationACLsManager.java index f72675f2f2..e48bfd6735 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationACLsManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationACLsManager.java @@ -25,6 +25,7 @@ import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; +import org.apache.hadoop.yarn.conf.YarnConfiguration; @InterfaceAudience.Private public class ApplicationACLsManager { @@ -36,7 +37,8 @@ public ApplicationACLsManager(Configuration conf) { } public boolean areACLsEnabled() { - return conf.getBoolean(RMConfig.RM_ACLS_ENABLED, false); + return conf.getBoolean(YarnConfiguration.RM_ACL_ENABLE, + YarnConfiguration.DEFAULT_RM_ACL_ENABLE); } /** diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java index 5f6f7d8b4c..436ed23d6e 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java @@ -90,8 +90,8 @@ public ApplicationMasterService(RMContext rmContext, @Override public void init(Configuration conf) { String bindAddress = - conf.get(YarnConfiguration.SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_SCHEDULER_BIND_ADDRESS); + conf.get(YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS); masterServiceAddress = NetUtils.createSocketAddr(bindAddress); super.init(conf); } @@ -105,8 +105,8 @@ public void start() { this.server = rpc.getServer(AMRMProtocol.class, this, masterServiceAddress, serverConf, this.appTokenManager, - serverConf.getInt(RMConfig.RM_AM_THREADS, - RMConfig.DEFAULT_RM_AM_THREADS)); + serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT, + YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT)); this.server.start(); super.start(); } @@ -232,8 +232,8 @@ public AllocateResponse allocate(AllocateRequest request) RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId); AMResponse response = recordFactory.newRecordInstance(AMResponse.class); - response.addAllNewContainers(allocation.getContainers()); - response.addAllFinishedContainers(appAttempt + response.setAllocatedContainers(allocation.getContainers()); + response.setCompletedContainersStatuses(appAttempt .pullJustFinishedContainers()); response.setResponseId(lastResponse.getResponseId() + 1); response.setAvailableResources(allocation.getResourceLimit()); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index dc6f6a796a..593d6525a6 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -31,7 +31,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.UserGroupInformation; @@ -58,10 +57,8 @@ 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.Container; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.QueueInfo; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; @@ -74,7 +71,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; @@ -95,6 +91,7 @@ public class ClientRMService extends AbstractService implements final private YarnScheduler scheduler; final private RMContext rmContext; private final AMLivelinessMonitor amLivelinessMonitor; + private final RMAppManager rmAppManager; private String clientServiceBindAddress; private Server server; @@ -104,18 +101,20 @@ public class ClientRMService extends AbstractService implements private ApplicationACLsManager aclsManager; private Map applicationACLs; - public ClientRMService(RMContext rmContext, YarnScheduler scheduler) { + public ClientRMService(RMContext rmContext, YarnScheduler scheduler, + RMAppManager rmAppManager) { super(ClientRMService.class.getName()); this.scheduler = scheduler; this.rmContext = rmContext; this.amLivelinessMonitor = rmContext.getAMLivelinessMonitor(); + this.rmAppManager = rmAppManager; } @Override public void init(Configuration conf) { clientServiceBindAddress = - conf.get(YarnConfiguration.APPSMANAGER_ADDRESS, - YarnConfiguration.DEFAULT_APPSMANAGER_BIND_ADDRESS); + conf.get(YarnConfiguration.RM_ADDRESS, + YarnConfiguration.DEFAULT_RM_ADDRESS); clientBindAddress = NetUtils.createSocketAddr(clientServiceBindAddress); @@ -138,8 +137,8 @@ public void start() { rpc.getServer(ClientRMProtocol.class, this, clientBindAddress, clientServerConf, null, - clientServerConf.getInt(RMConfig.RM_CLIENT_THREADS, - RMConfig.DEFAULT_RM_CLIENT_THREADS)); + clientServerConf.getInt(YarnConfiguration.RM_CLIENT_THREAD_COUNT, + YarnConfiguration.DEFAULT_RM_CLIENT_THREAD_COUNT)); this.server.start(); super.start(); } @@ -205,8 +204,10 @@ public SubmitApplicationResponse submitApplication( throw new IOException("Application with id " + applicationId + " is already present! Cannot add a duplicate!"); } - this.rmContext.getDispatcher().getEventHandler().handle( - new RMAppManagerSubmitEvent(submissionContext)); + // This needs to be synchronous as the client can query + // immediately following the submission to get the application status. + // So call handle directly and do not send an event. + rmAppManager.handle(new RMAppManagerSubmitEvent(submissionContext)); LOG.info("Application with id " + applicationId.getId() + " submitted by user " + user + " with " + submissionContext); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NMLivelinessMonitor.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NMLivelinessMonitor.java index aa9b354a9e..ab89fbcbcc 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NMLivelinessMonitor.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NMLivelinessMonitor.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.SystemClock; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent; @@ -38,11 +39,11 @@ public NMLivelinessMonitor(Dispatcher d) { public void init(Configuration conf) { super.init(conf); - setExpireInterval(conf.getInt(RMConfig.NM_EXPIRY_INTERVAL, - RMConfig.DEFAULT_NM_EXPIRY_INTERVAL)); + setExpireInterval(conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, + YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS)); setMonitorInterval(conf.getInt( - RMConfig.NMLIVELINESS_MONITORING_INTERVAL, - RMConfig.DEFAULT_NMLIVELINESS_MONITORING_INTERVAL)); + YarnConfiguration.RM_NM_LIVENESS_MONITOR_INTERVAL_MS, + YarnConfiguration.DEFAULT_RM_NM_LIVENESS_MONITOR_INTERVAL_MS)); } @Override diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java index e131653823..84c53957f0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java @@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.HostsFileReader; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.service.AbstractService; public class NodesListManager extends AbstractService{ @@ -48,18 +49,18 @@ public void init(Configuration conf) { try { this.hostsReader = new HostsFileReader( - conf.get(RMConfig.RM_NODES_INCLUDE_FILE, - RMConfig.DEFAULT_RM_NODES_INCLUDE_FILE), - conf.get(RMConfig.RM_NODES_EXCLUDE_FILE, - RMConfig.DEFAULT_RM_NODES_EXCLUDE_FILE) + conf.get(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, + YarnConfiguration.DEFAULT_RM_NODES_INCLUDE_FILE_PATH), + conf.get(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, + YarnConfiguration.DEFAULT_RM_NODES_EXCLUDE_FILE_PATH) ); printConfiguredHosts(); } catch (IOException ioe) { LOG.warn("Failed to init hostsReader, disabling", ioe); try { this.hostsReader = - new HostsFileReader(RMConfig.DEFAULT_RM_NODES_INCLUDE_FILE, - RMConfig.DEFAULT_RM_NODES_EXCLUDE_FILE); + new HostsFileReader(YarnConfiguration.DEFAULT_RM_NODES_INCLUDE_FILE_PATH, + YarnConfiguration.DEFAULT_RM_NODES_EXCLUDE_FILE_PATH); } catch (IOException ioe2) { // Should *never* happen this.hostsReader = null; @@ -74,10 +75,10 @@ private void printConfiguredHosts() { return; } - LOG.debug("hostsReader: in=" + conf.get(RMConfig.RM_NODES_INCLUDE_FILE, - RMConfig.DEFAULT_RM_NODES_INCLUDE_FILE) + " out=" + - conf.get(RMConfig.RM_NODES_EXCLUDE_FILE, - RMConfig.DEFAULT_RM_NODES_EXCLUDE_FILE)); + LOG.debug("hostsReader: in=" + conf.get(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, + YarnConfiguration.DEFAULT_RM_NODES_INCLUDE_FILE_PATH) + " out=" + + conf.get(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, + YarnConfiguration.DEFAULT_RM_NODES_EXCLUDE_FILE_PATH)); for (String include : hostsReader.getHosts()) { LOG.debug("include: " + include); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index 0324908f2a..9a86dfd457 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -18,32 +18,29 @@ package org.apache.hadoop.yarn.server.resourcemanager; import java.io.IOException; -import java.util.List; import java.util.LinkedList; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier; -import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager; import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager; -import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; -import org.apache.hadoop.util.StringUtils; /** * This class manages the list of applications for the resource manager. @@ -52,7 +49,7 @@ public class RMAppManager implements EventHandler { private static final Log LOG = LogFactory.getLog(RMAppManager.class); - private int completedAppsMax = RMConfig.DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX; + private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS; private LinkedList completedApps = new LinkedList(); private final RMContext rmContext; @@ -70,8 +67,8 @@ public RMAppManager(RMContext context, ClientToAMSecretManager this.masterService = masterService; this.conf = conf; setCompletedAppsMax(conf.getInt( - RMConfig.EXPIRE_APPLICATIONS_COMPLETED_MAX, - RMConfig.DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX)); + YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, + YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS)); } /** @@ -154,7 +151,7 @@ public static void logAppSummary(RMApp app) { } } - protected void setCompletedAppsMax(int max) { + protected synchronized void setCompletedAppsMax(int max) { this.completedAppsMax = max; } @@ -213,7 +210,7 @@ protected synchronized void checkAppNumCompletedLimit() { } } - protected void submitApplication(ApplicationSubmissionContext submissionContext) { + protected synchronized void submitApplication(ApplicationSubmissionContext submissionContext) { ApplicationId applicationId = submissionContext.getApplicationId(); RMApp application = null; try { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMConfig.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMConfig.java deleted file mode 100644 index c3fbf7610e..0000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMConfig.java +++ /dev/null @@ -1,93 +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; - -import org.apache.hadoop.yarn.conf.YarnConfiguration; - -public class RMConfig { - public static final String RM_KEYTAB = YarnConfiguration.RM_PREFIX - + "keytab"; - public static final String ZK_ADDRESS = YarnConfiguration.RM_PREFIX - + "zookeeper.address"; - public static final String ZK_SESSION_TIMEOUT = YarnConfiguration.RM_PREFIX - + "zookeeper.session.timeout"; - public static final String ADMIN_ADDRESS = YarnConfiguration.RM_PREFIX - + "admin.address"; - public static final String AM_MAX_RETRIES = YarnConfiguration.RM_PREFIX - + "application.max.retries"; - public static final int DEFAULT_ZK_TIMEOUT = 60000; - public static final int DEFAULT_AM_MAX_RETRIES = 3; - public static final int DEFAULT_AM_EXPIRY_INTERVAL = 600000; - public static final String NM_EXPIRY_INTERVAL = YarnConfiguration.RM_PREFIX - + "nodemanager.expiry.interval"; - public static final int DEFAULT_NM_EXPIRY_INTERVAL = 600000; - public static final String DEFAULT_ADMIN_BIND_ADDRESS = "0.0.0.0:8141"; - public static final String RESOURCE_SCHEDULER = YarnConfiguration.RM_PREFIX - + "scheduler"; - public static final String RM_STORE = YarnConfiguration.RM_PREFIX + "store"; - public static final String AMLIVELINESS_MONITORING_INTERVAL = - YarnConfiguration.RM_PREFIX - + "amliveliness-monitor.monitoring-interval"; - public static final int DEFAULT_AMLIVELINESS_MONITORING_INTERVAL = 1000; - public static final String CONTAINER_LIVELINESS_MONITORING_INTERVAL - = YarnConfiguration.RM_PREFIX - + "amliveliness-monitor.monitoring-interval"; - public static final int DEFAULT_CONTAINER_LIVELINESS_MONITORING_INTERVAL = 600000; - public static final String NMLIVELINESS_MONITORING_INTERVAL = - YarnConfiguration.RM_PREFIX - + "nmliveliness-monitor.monitoring-interval"; - public static final int DEFAULT_NMLIVELINESS_MONITORING_INTERVAL = 1000; - - public static final String RM_RESOURCE_TRACKER_THREADS = - YarnConfiguration.RM_PREFIX + "resource.tracker.threads"; - public static final int DEFAULT_RM_RESOURCE_TRACKER_THREADS = 10; - - public static final String RM_CLIENT_THREADS = - YarnConfiguration.RM_PREFIX + "client.threads"; - public static final int DEFAULT_RM_CLIENT_THREADS = 10; - - public static final String RM_AM_THREADS = - YarnConfiguration.RM_PREFIX + "am.threads"; - public static final int DEFAULT_RM_AM_THREADS = 10; - - public static final String RM_ADMIN_THREADS = - YarnConfiguration.RM_PREFIX + "admin.threads"; - public static final int DEFAULT_RM_ADMIN_THREADS = 1; - - /* key for looking up the acls configuration for acls checking for application */ - public static final String RM_ACLS_ENABLED = YarnConfiguration.RM_PREFIX + - "acls.enabled"; - - public static final String RM_ADMIN_ACL = - YarnConfiguration.RM_PREFIX + "admin.acl"; - public static final String DEFAULT_RM_ADMIN_ACL = "*"; - - public static final String RM_NODES_INCLUDE_FILE = - YarnConfiguration.RM_PREFIX + "nodes.include"; - public static final String DEFAULT_RM_NODES_INCLUDE_FILE = ""; - - public static final String RM_NODES_EXCLUDE_FILE = - YarnConfiguration.RM_PREFIX + "nodes.exclude"; - public static final String DEFAULT_RM_NODES_EXCLUDE_FILE = ""; - - // the maximum number of completed applications RM keeps - public static String EXPIRE_APPLICATIONS_COMPLETED_MAX = - YarnConfiguration.RM_PREFIX + "expire.applications.completed.max"; - public static final int DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX = 10000; -} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index fd42e5f1ff..b578fee818 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -99,7 +99,7 @@ public class ResourceManager extends CompositeService implements Recoverable { protected NMLivelinessMonitor nmLivelinessMonitor; protected NodesListManager nodesListManager; private SchedulerEventDispatcher schedulerDispatcher; - private RMAppManager rmAppManager; + protected RMAppManager rmAppManager; private final AtomicBoolean shutdown = new AtomicBoolean(false); private WebApp webApp; @@ -176,13 +176,13 @@ public synchronized void init(Configuration conf) { masterService = createApplicationMasterService(); addService(masterService) ; - clientRM = createClientRMService(); - addService(clientRM); - this.rmAppManager = createRMAppManager(); // Register event handler for RMAppManagerEvents this.rmDispatcher.register(RMAppManagerEventType.class, this.rmAppManager); + + clientRM = createClientRMService(); + addService(clientRM); adminService = createAdminService(); addService(adminService); @@ -202,7 +202,7 @@ protected void addIfService(Object object) { protected ResourceScheduler createScheduler() { return ReflectionUtils.newInstance( - conf.getClass(RMConfig.RESOURCE_SCHEDULER, + conf.getClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, ResourceScheduler.class), this.conf); } @@ -384,8 +384,8 @@ public void handle(RMNodeEvent event) { protected void startWepApp() { webApp = WebApps.$for("yarn", masterService).at( - conf.get(YarnConfiguration.RM_WEBAPP_BIND_ADDRESS, - YarnConfiguration.DEFAULT_RM_WEBAPP_BIND_ADDRESS)). + conf.get(YarnConfiguration.RM_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS)). start(new RMWebApp(this)); } @@ -415,8 +415,8 @@ public void start() { } protected void doSecureLogin() throws IOException { - SecurityUtil.login(conf, RMConfig.RM_KEYTAB, - YarnConfiguration.RM_SERVER_PRINCIPAL_KEY); + SecurityUtil.login(conf, YarnConfiguration.RM_KEYTAB, + YarnConfiguration.RM_PRINCIPAL); } @Override @@ -441,7 +441,7 @@ protected ResourceTrackerService createResourceTrackerService() { } protected ClientRMService createClientRMService() { - return new ClientRMService(this.rmContext, scheduler); + return new ClientRMService(this.rmContext, scheduler, this.rmAppManager); } protected ApplicationMasterService createApplicationMasterService() { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java index cfab347c06..ba543813ac 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java @@ -20,10 +20,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicInteger; import javax.crypto.SecretKey; @@ -31,7 +27,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.Node; import org.apache.hadoop.security.SecurityInfo; @@ -44,7 +39,6 @@ import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.server.RMNMSecurityInfoClass; -import org.apache.hadoop.yarn.server.YarnServerConfig; 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; @@ -102,8 +96,8 @@ public ResourceTrackerService(RMContext rmContext, @Override public synchronized void init(Configuration conf) { String resourceTrackerBindAddress = - conf.get(YarnServerConfig.RESOURCETRACKER_ADDRESS, - YarnServerConfig.DEFAULT_RESOURCETRACKER_BIND_ADDRESS); + conf.get(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS, + YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS); resourceTrackerAddress = NetUtils.createSocketAddr(resourceTrackerBindAddress); RackResolver.init(conf); @@ -123,8 +117,8 @@ public synchronized void start() { this.server = rpc.getServer(ResourceTracker.class, this, resourceTrackerAddress, rtServerConf, null, - rtServerConf.getInt(RMConfig.RM_RESOURCE_TRACKER_THREADS, - RMConfig.DEFAULT_RM_RESOURCE_TRACKER_THREADS)); + rtServerConf.getInt(YarnConfiguration.RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT, + YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT)); this.server.start(); } @@ -253,7 +247,7 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request) // 4. Send status to RMNode, saving the latest response. this.rmContext.getDispatcher().getEventHandler().handle( new RMNodeStatusEvent(nodeId, remoteNodeStatus.getNodeHealthStatus(), - remoteNodeStatus.getAllContainers(), latestResponse)); + remoteNodeStatus.getContainersStatuses(), latestResponse)); nodeHeartBeatResponse.setHeartbeatResponse(latestResponse); return nodeHeartBeatResponse; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java index 6eae783f78..1a10993bb0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java @@ -33,7 +33,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.io.DataInputByteBuffer; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Text; @@ -80,7 +79,8 @@ public class AMLauncher implements Runnable { private final RMAppAttempt application; private final Configuration conf; - private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + private final RecordFactory recordFactory = + RecordFactoryProvider.getRecordFactory(null); private final ApplicationTokenSecretManager applicationTokenSecretManager; private final ClientToAMSecretManager clientToAMSecretManager; private final AMLauncherEventType eventType; @@ -88,9 +88,9 @@ public class AMLauncher implements Runnable { @SuppressWarnings("rawtypes") private final EventHandler handler; - @SuppressWarnings("unchecked") public AMLauncher(RMContext rmContext, RMAppAttempt application, - AMLauncherEventType eventType,ApplicationTokenSecretManager applicationTokenSecretManager, + AMLauncherEventType eventType, + ApplicationTokenSecretManager applicationTokenSecretManager, ClientToAMSecretManager clientToAMSecretManager, Configuration conf) { this.application = application; this.conf = new Configuration(conf); // Just not to touch the sec-info class @@ -107,7 +107,8 @@ private void connect() throws IOException { ContainerId masterContainerID = application.getMasterContainer().getId(); containerMgrProxy = - getContainerMgrProxy(masterContainerID.getAppId()); + getContainerMgrProxy( + masterContainerID.getApplicationAttemptId().getApplicationId()); } private void launch() throws IOException { @@ -170,12 +171,12 @@ private ContainerLaunchContext createAMContainerLaunchContext( // Construct the actual Container ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class); - container.addAllCommands(applicationMasterContext.getCommandList()); + container.setCommands(applicationMasterContext.getCommandList()); StringBuilder mergedCommand = new StringBuilder(); String failCount = Integer.toString(application.getAppAttemptId() .getAttemptId()); List commandList = new ArrayList(); - for (String str : container.getCommandList()) { + for (String str : container.getCommands()) { // This is out-right wrong. AM FAIL count should be passed via env. String result = str.replaceFirst(ApplicationConstants.AM_FAIL_COUNT_STRING, @@ -183,21 +184,21 @@ private ContainerLaunchContext createAMContainerLaunchContext( mergedCommand.append(result).append(" "); commandList.add(result); } - container.clearCommands(); - container.addAllCommands(commandList); + container.setCommands(commandList); /** add the failed count to the app master command line */ LOG.info("Command to launch container " + containerID + " : " + mergedCommand); - container.addAllEnv(applicationMasterContext.getAllEnvironment()); - - container.addAllEnv(setupTokensInEnv(applicationMasterContext)); + Map environment = + applicationMasterContext.getAllEnvironment(); + environment.putAll(setupTokensInEnv(applicationMasterContext)); + container.setEnv(environment); // Construct the actual Container container.setContainerId(containerID); container.setUser(applicationMasterContext.getUser()); container.setResource(applicationMasterContext.getMasterCapability()); - container.addAllLocalResources(applicationMasterContext.getAllResourcesTodo()); + container.setLocalResources(applicationMasterContext.getAllResourcesTodo()); container.setContainerTokens(applicationMasterContext.getFsTokensTodo()); return container; } @@ -225,8 +226,8 @@ private Map setupTokensInEnv( new Token(id, this.applicationTokenSecretManager); String schedulerAddressStr = - this.conf.get(YarnConfiguration.SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_SCHEDULER_BIND_ADDRESS); + this.conf.get(YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS); InetSocketAddress unresolvedAddr = NetUtils.createSocketAddr(schedulerAddressStr); String resolvedAddr = diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java index 5f12348d85..d1ef1d1400 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java @@ -37,7 +37,6 @@ public class ApplicationMasterLauncher extends AbstractService implements private static final Log LOG = LogFactory.getLog( ApplicationMasterLauncher.class); private final ThreadPoolExecutor launcherPool; - private final EventHandler handler; private LauncherThread launcherHandlingThread; private final BlockingQueue masterEvents @@ -52,7 +51,6 @@ public ApplicationMasterLauncher(ApplicationTokenSecretManager RMContext context) { super(ApplicationMasterLauncher.class.getName()); this.context = context; - this.handler = context.getDispatcher().getEventHandler(); /* register to dispatcher */ this.context.getDispatcher().register(AMLauncherEventType.class, this); this.launcherPool = new ThreadPoolExecutor(1, 10, 1, @@ -67,14 +65,16 @@ public void start() { super.start(); } - protected Runnable createRunnableLauncher(RMAppAttempt application, AMLauncherEventType event) { + protected Runnable createRunnableLauncher(RMAppAttempt application, + AMLauncherEventType event) { Runnable launcher = new AMLauncher(context, application, event, applicationTokenSecretManager, clientToAMSecretManager, getConfig()); return launcher; } private void launch(RMAppAttempt application) { - Runnable launcher = createRunnableLauncher(application, AMLauncherEventType.LAUNCH); + Runnable launcher = createRunnableLauncher(application, + AMLauncherEventType.LAUNCH); masterEvents.add(launcher); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/StoreFactory.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/StoreFactory.java index edc5d53bb5..91b33ca6db 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/StoreFactory.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/StoreFactory.java @@ -23,14 +23,14 @@ import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.yarn.api.records.ApplicationMaster; import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore; public class StoreFactory { public static Store getStore(Configuration conf) { Store store = ReflectionUtils.newInstance( - conf.getClass(RMConfig.RM_STORE, + conf.getClass(YarnConfiguration.RM_STORE, MemStore.class, Store.class), conf); return store; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java index baaca6bad6..f2e401a6ba 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java @@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationMasterProto; @@ -48,10 +49,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; -import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -91,9 +89,9 @@ public void process(WatchedEvent arg0) { public ZKStore(Configuration conf) throws IOException { this.conf = conf; - this.ZK_ADDRESS = conf.get(RMConfig.ZK_ADDRESS); - this.ZK_TIMEOUT = conf.getInt(RMConfig.ZK_SESSION_TIMEOUT, - RMConfig.DEFAULT_ZK_TIMEOUT); + this.ZK_ADDRESS = conf.get(YarnConfiguration.RM_ZK_STORE_ADDRESS); + this.ZK_TIMEOUT = conf.getInt(YarnConfiguration.RM_ZK_STORE_TIMEOUT_MS, + YarnConfiguration.DEFAULT_RM_ZK_STORE_TIMEOUT_MS); zkClient = new ZooKeeper(this.ZK_ADDRESS, this.ZK_TIMEOUT, createZKWatcher() @@ -123,10 +121,9 @@ private NodeReportPBImpl createNodeManagerInfo(RMNode rmNode) { public synchronized void storeNode(RMNode node) throws IOException { /** create a storage node and store it in zk **/ if (!doneWithRecovery) return; - NodeReportPBImpl nodeManagerInfo = createNodeManagerInfo(node); - // TODO FinBugs - will be fixed after the subsequent fixme - byte[] bytes = nodeManagerInfo.getProto().toByteArray(); // TODO: FIXMEVinodkv +// NodeReportPBImpl nodeManagerInfo = createNodeManagerInfo(node); +// byte[] bytes = nodeManagerInfo.getProto().toByteArray(); // try { // zkClient.create(NODES + Integer.toString(node.getNodeID().getId()), bytes, null, // CreateMode.PERSISTENT); @@ -181,7 +178,8 @@ public synchronized NodeId getNextNodeId() throws IOException { } private String containerPathFromContainerId(ContainerId containerId) { - String appString = ConverterUtils.toString(containerId.getAppId()); + String appString = ConverterUtils.toString( + containerId.getApplicationAttemptId().getApplicationId()); return appString + "/" + containerId.getId(); } @@ -198,7 +196,10 @@ public void storeMasterContainer(Container container) throws IOException { ContainerPBImpl containerPBImpl = (ContainerPBImpl) container; try { - zkClient.setData(APPS + ConverterUtils.toString(container.getId().getAppId()) + + zkClient.setData(APPS + + ConverterUtils.toString( + container.getId().getApplicationAttemptId().getApplicationId()) + + ZK_PATH_SEPARATOR + APP_MASTER_CONTAINER , containerPBImpl.getProto().toByteArray(), -1); } catch(InterruptedException ie) { @@ -476,12 +477,12 @@ private void load() throws IOException { continue; } int httpPort = Integer.valueOf(m.group(1)); - // TODO: FindBugs Valid. Fix - RMNode nm = new RMNodeImpl(node.getNodeId(), null, - hostName, cmPort, httpPort, - ResourceTrackerService.resolve(node.getNodeId().getHost()), - node.getCapability()); - nodeManagers.add(nm); + // TODO: FindBugs warns passing null below. Commenting this for later. +// RMNode nm = new RMNodeImpl(node.getNodeId(), null, +// hostName, cmPort, httpPort, +// ResourceTrackerService.resolve(node.getNodeId().getHost()), +// node.getCapability()); +// nodeManagers.add(nm); } readLastNodeId(); /* make sure we get all the applications */ diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java index 8604bd3d2d..015c76163e 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java @@ -37,9 +37,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationState; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent; @@ -181,8 +181,8 @@ public RMAppImpl(ApplicationId applicationId, RMContext rmContext, this.masterService = masterService; this.startTime = System.currentTimeMillis(); - this.maxRetries = conf.getInt(RMConfig.AM_MAX_RETRIES, - RMConfig.DEFAULT_AM_MAX_RETRIES); + this.maxRetries = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, + YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES); ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); this.readLock = lock.readLock(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AMLivelinessMonitor.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AMLivelinessMonitor.java index 63775ac4a4..2f8d82e9f7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AMLivelinessMonitor.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AMLivelinessMonitor.java @@ -24,7 +24,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor; public class AMLivelinessMonitor extends AbstractLivelinessMonitor { @@ -38,10 +37,10 @@ public AMLivelinessMonitor(Dispatcher d) { public void init(Configuration conf) { super.init(conf); - setExpireInterval(conf.getInt(YarnConfiguration.AM_EXPIRY_INTERVAL, - RMConfig.DEFAULT_AM_EXPIRY_INTERVAL)); - setMonitorInterval(conf.getInt(RMConfig.AMLIVELINESS_MONITORING_INTERVAL, - RMConfig.DEFAULT_AMLIVELINESS_MONITORING_INTERVAL)); + setExpireInterval(conf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, + YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS)); + setMonitorInterval(conf.getInt(YarnConfiguration.RM_AM_LIVENESS_MONITOR_INTERVAL_MS, + YarnConfiguration.DEFAULT_RM_AM_LIVENESS_MONITOR_INTERVAL_MS)); } @Override diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java index 861de522cb..98025fb6ca 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java @@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.event.EventHandler; @@ -47,9 +48,9 @@ public interface RMAppAttempt extends EventHandler{ Set getRanNodes(); - List pullJustFinishedContainers(); + List pullJustFinishedContainers(); - List getJustFinishedContainers(); + List getJustFinishedContainers(); Container getMasterContainer(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java index a0b5b22fe7..6daff1d88e 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java @@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -97,8 +98,8 @@ public class RMAppAttemptImpl implements RMAppAttempt { //nodes on while this attempt's containers ran private final Set ranNodes = new HashSet(); - private final List justFinishedContainers = - new ArrayList(); + private final List justFinishedContainers = + new ArrayList(); private Container masterContainer; private float progress = 0; @@ -333,7 +334,7 @@ public float getProgress() { } @Override - public List getJustFinishedContainers() { + public List getJustFinishedContainers() { this.readLock.lock(); try { return this.justFinishedContainers; @@ -343,11 +344,11 @@ public List getJustFinishedContainers() { } @Override - public List pullJustFinishedContainers() { + public List pullJustFinishedContainers() { this.writeLock.lock(); try { - List returnList = new ArrayList( + List returnList = new ArrayList( this.justFinishedContainers.size()); returnList.addAll(this.justFinishedContainers); this.justFinishedContainers.clear(); @@ -705,11 +706,13 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, RMAppAttemptContainerFinishedEvent containerFinishedEvent = (RMAppAttemptContainerFinishedEvent) event; - Container container = containerFinishedEvent.getContainer(); + ContainerStatus containerStatus = + containerFinishedEvent.getContainerStatus(); // Is this container the AmContainer? If the finished container is same as // the AMContainer, AppAttempt fails - if (appAttempt.masterContainer.getId().equals(container.getId())) { + if (appAttempt.masterContainer.getId().equals( + containerStatus.getContainerId())) { new FinalTransition(RMAppAttemptState.FAILED).transition( appAttempt, containerFinishedEvent); return RMAppAttemptState.FAILED; @@ -718,7 +721,7 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, // Normal container. // Put it in completedcontainers list - appAttempt.justFinishedContainers.add(container); + appAttempt.justFinishedContainers.add(containerStatus); return RMAppAttemptState.RUNNING; } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptContainerFinishedEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptContainerFinishedEvent.java index 750ec07173..3660597d3f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptContainerFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptContainerFinishedEvent.java @@ -19,22 +19,22 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; public class RMAppAttemptContainerFinishedEvent extends RMAppAttemptEvent { - private final Container container; + private final ContainerStatus containerStatus; public RMAppAttemptContainerFinishedEvent(ApplicationAttemptId appAttemptId, - Container container) { + ContainerStatus containerStatus) { super(appAttemptId, RMAppAttemptEventType.CONTAINER_FINISHED); - this.container = container; + this.containerStatus = containerStatus; } - public Container getContainer() { - return this.container; + public ContainerStatus getContainerStatus() { + return this.containerStatus; } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/ContainerAllocationExpirer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/ContainerAllocationExpirer.java index fdd467e7e9..9216364ff5 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/ContainerAllocationExpirer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/ContainerAllocationExpirer.java @@ -21,9 +21,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.SystemClock; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor; @@ -40,10 +40,10 @@ public ContainerAllocationExpirer(Dispatcher d) { public void init(Configuration conf) { super.init(conf); setExpireInterval(conf.getInt( - RMConfig.CONTAINER_LIVELINESS_MONITORING_INTERVAL, - RMConfig.DEFAULT_CONTAINER_LIVELINESS_MONITORING_INTERVAL)); - setMonitorInterval(conf.getInt(RMConfig.AMLIVELINESS_MONITORING_INTERVAL, - RMConfig.DEFAULT_AMLIVELINESS_MONITORING_INTERVAL)); + YarnConfiguration.RM_CONTAINER_LIVENESS_MONITOR_INTERVAL_MS, + YarnConfiguration.DEFAULT_RM_CONTAINER_LIVENESS_MONITOR_INTERVAL_MS)); + setMonitorInterval(conf.getInt(YarnConfiguration.RM_AM_LIVENESS_MONITOR_INTERVAL_MS, + YarnConfiguration.DEFAULT_RM_AM_LIVENESS_MONITOR_INTERVAL_MS)); } @Override diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerFinishedEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerFinishedEvent.java index 740287de4f..d760e7dc3a 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerFinishedEvent.java @@ -26,8 +26,8 @@ public class RMContainerFinishedEvent extends RMContainerEvent { private final ContainerStatus remoteContainerStatus; public RMContainerFinishedEvent(ContainerId containerId, - ContainerStatus containerStatus) { - super(containerId, RMContainerEventType.FINISHED); + ContainerStatus containerStatus, RMContainerEventType event) { + super(containerId, event); this.remoteContainerStatus = containerStatus; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java index 07bc1cdeec..ffdd23c438 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java @@ -92,7 +92,7 @@ RMContainerEventType.KILL, new KillTransition()) // Transitions from RUNNING state .addTransition(RMContainerState.RUNNING, RMContainerState.COMPLETED, - RMContainerEventType.FINISHED, new ContainerCompletedTransition()) + RMContainerEventType.FINISHED, new FinishedTransition()) .addTransition(RMContainerState.RUNNING, RMContainerState.KILLED, RMContainerEventType.KILL, new KillTransition()) @@ -273,10 +273,16 @@ private static class FinishedTransition extends BaseTransition { @Override public void transition(RMContainerImpl container, RMContainerEvent event) { + RMContainerFinishedEvent finishedEvent = (RMContainerFinishedEvent) event; + + // Update container-status for diagnostics. Today we completely + // replace it on finish. We may just need to update diagnostics. + container.container.setContainerStatus(finishedEvent + .getRemoteContainerStatus()); // Inform AppAttempt container.eventHandler.handle(new RMAppAttemptContainerFinishedEvent( - container.appAttemptId, container.container)); + container.appAttemptId, container.container.getContainerStatus())); } } @@ -312,22 +318,4 @@ public void transition(RMContainerImpl container, RMContainerEvent event) { } } - private static final class ContainerCompletedTransition extends - FinishedTransition { - - @Override - public void transition(RMContainerImpl container, RMContainerEvent event) { - - RMContainerFinishedEvent finishedEvent = (RMContainerFinishedEvent) event; - - // Update container-status for diagnostics. Today we completely - // replace it on finish. We may just need to update diagnostics. - // ^TODO - container.container.setContainerStatus(finishedEvent - .getRemoteContainerStatus()); - - // Inform appAttempt - super.transition(container, event); - } - } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java index ceb28434f4..dd8d7f840f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java @@ -23,7 +23,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -36,9 +35,9 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.net.Node; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; @@ -87,8 +86,8 @@ public class RMNodeImpl implements RMNode, EventHandler { .newRecordInstance(NodeHealthStatus.class); /* set of containers that have just launched */ - private final Map justLaunchedContainers = - new HashMap(); + private final Map justLaunchedContainers = + new HashMap(); /* set of containers that need to be cleaned */ @@ -355,43 +354,29 @@ public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { // Filter the map to only obtain just launched containers and finished // containers. - Map> remoteAppContainersMap = statusEvent - .getContainersCollection(); - Map> containersMapForScheduler = new HashMap>( - remoteAppContainersMap.size()); - for (Entry> entrySet : remoteAppContainersMap - .entrySet()) { - - ApplicationId appId = entrySet.getKey(); - List remoteContainerList = entrySet.getValue(); - - if (!containersMapForScheduler.containsKey(appId)) { - containersMapForScheduler.put(appId, new ArrayList( - remoteContainerList.size())); - } - List entryForThisApp = containersMapForScheduler - .get(appId); - - for (Container remoteContainer : remoteContainerList) { - - // Process running containers - ContainerId containerId = remoteContainer.getId(); - if (remoteContainer.getState() == ContainerState.RUNNING) { - if (!rmNode.justLaunchedContainers.containsKey(containerId)) { - // Just launched container. RM knows about it the first time. - rmNode.justLaunchedContainers.put(containerId, remoteContainer); - entryForThisApp.add(remoteContainer); - } - } else { - // A finished container - rmNode.justLaunchedContainers.remove(containerId); - entryForThisApp.add(remoteContainer); + List newlyLaunchedContainers = + new ArrayList(); + List completedContainers = + new ArrayList(); + for (ContainerStatus remoteContainer : statusEvent.getContainers()) { + // Process running containers + ContainerId containerId = remoteContainer.getContainerId(); + if (remoteContainer.getState() == ContainerState.RUNNING) { + if (!rmNode.justLaunchedContainers.containsKey(containerId)) { + // Just launched container. RM knows about it the first time. + rmNode.justLaunchedContainers.put(containerId, remoteContainer); + newlyLaunchedContainers.add(remoteContainer); } + } else { + // A finished container + rmNode.justLaunchedContainers.remove(containerId); + completedContainers.add(remoteContainer); } } rmNode.context.getDispatcher().getEventHandler().handle( - new NodeUpdateSchedulerEvent(rmNode, containersMapForScheduler)); + new NodeUpdateSchedulerEvent(rmNode, newlyLaunchedContainers, + completedContainers)); return RMNodeState.RUNNING; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java index 08f2b68971..e4a2930168 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java @@ -19,10 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmnode; import java.util.List; -import java.util.Map; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse; @@ -30,11 +28,11 @@ public class RMNodeStatusEvent extends RMNodeEvent { private final NodeHealthStatus nodeHealthStatus; - private Map> containersCollection; + private List containersCollection; private final HeartbeatResponse latestResponse; public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus, - Map> collection, + List collection, HeartbeatResponse latestResponse) { super(nodeId, RMNodeEventType.STATUS_UPDATE); this.nodeHealthStatus = nodeHealthStatus; @@ -46,7 +44,7 @@ public NodeHealthStatus getNodeHealthStatus() { return this.nodeHealthStatus; } - public Map> getContainersCollection() { + public List getContainers() { return this.containersCollection; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java index 13101fc95e..b4037aaeaf 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; @@ -161,26 +162,21 @@ synchronized public void containerLaunchedOnNode(ContainerId containerId) { RMContainerEventType.LAUNCHED)); } - public synchronized void killContainers( - SchedulerApp application) { - } - synchronized public void containerCompleted(RMContainer rmContainer, - RMContainerEventType event) { + ContainerStatus containerStatus, RMContainerEventType event) { Container container = rmContainer.getContainer(); ContainerId containerId = container.getId(); // Inform the container - if (event.equals(RMContainerEventType.FINISHED)) { - // Have to send diagnostics for finished containers. - rmContainer.handle(new RMContainerFinishedEvent(containerId, - container.getContainerStatus())); - } else { - rmContainer.handle(new RMContainerEvent(containerId, event)); - } + rmContainer.handle( + new RMContainerFinishedEvent( + containerId, + containerStatus, + event) + ); LOG.info("Completed container: " + rmContainer.getContainerId() + - " in state: " + rmContainer.getState()); + " in state: " + rmContainer.getState() + " event:" + event); // Remove from the list of containers liveContainers.remove(rmContainer.getContainerId()); @@ -219,7 +215,8 @@ synchronized public RMContainer allocate(NodeType type, SchedulerNode node, Resources.addTo(currentConsumption, container.getResource()); if (LOG.isDebugEnabled()) { - LOG.debug("allocate: applicationId=" + container.getId().getAppId() + LOG.debug("allocate: applicationAttemptId=" + + container.getId().getApplicationAttemptId() + " container=" + container.getId() + " host=" + container.getNodeId().getHost() + " type=" + type); } @@ -278,10 +275,7 @@ public synchronized RMContainer getRMContainer(ContainerId id) { } synchronized public void resetSchedulingOpportunities(Priority priority) { - Integer schedulingOpportunities = - this.schedulingOpportunities.get(priority); - schedulingOpportunities = 0; - this.schedulingOpportunities.put(priority, schedulingOpportunities); + this.schedulingOpportunities.put(priority, Integer.valueOf(0)); } synchronized public void addSchedulingOpportunity(Priority priority) { @@ -305,9 +299,7 @@ synchronized public int getSchedulingOpportunities(Priority priority) { } synchronized void resetReReservations(Priority priority) { - Integer reReservations = this.reReservations.get(priority); - reReservations = 0; - this.reReservations.put(priority, reReservations); + this.reReservations.put(priority, Integer.valueOf(0)); } synchronized void addReReservation(Priority priority) { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java index a666478035..7e51841495 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java @@ -198,8 +198,8 @@ public synchronized void reserveResource( } // Cannot reserve more than one application on a given node! - if (!this.reservedContainer.getContainer().getId().getAppAttemptId().equals( - reservedContainer.getContainer().getId().getAppAttemptId())) { + if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals( + reservedContainer.getContainer().getId().getApplicationAttemptId())) { throw new IllegalStateException("Trying to reserve" + " container " + reservedContainer + " for application " + application.getApplicationId() + @@ -221,7 +221,7 @@ public synchronized void reserveResource( public synchronized void unreserveResource(SchedulerApp application) { // Cannot unreserve for wrong application... ApplicationAttemptId reservedApplication = - reservedContainer.getContainer().getId().getAppAttemptId(); + reservedContainer.getContainer().getId().getApplicationAttemptId(); if (!reservedApplication.equals( application.getApplicationAttemptId())) { throw new IllegalStateException("Trying to unreserve " + diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java new file mode 100644 index 0000000000..8e7ebd9fac --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.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.yarn.server.resourcemanager.scheduler; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; + +/** + * Utilities shared by schedulers. + */ +@Private +@Unstable +public class SchedulerUtils { + + private static final RecordFactory recordFactory = + RecordFactoryProvider.getRecordFactory(null); + + public static final String RELEASED_CONTAINER = + "Container released by application"; + + public static final String LOST_CONTAINER = + "Container released on a *lost* node"; + + public static final String COMPLETED_APPLICATION = + "Container of a completed application"; + + public static final String EXPIRED_CONTAINER = + "Container expired since it was unused"; + + public static final String UNRESERVED_CONTAINER = + "Container reservation no longer required."; + + /** + * Utility to create a {@link ContainerStatus} during exceptional + * circumstances. + * + * @param containerId {@link ContainerId} of returned/released/lost container. + * @param diagnostics diagnostic message + * @return ContainerStatus for an returned/released/lost + * container + */ + public static ContainerStatus createAbnormalContainerStatus( + ContainerId containerId, String diagnostics) { + ContainerStatus containerStatus = + recordFactory.newRecordInstance(ContainerStatus.class); + containerStatus.setContainerId(containerId); + containerStatus.setDiagnostics(diagnostics); + containerStatus.setExitStatus( + YarnConfiguration.ABORTED_CONTAINER_EXIT_STATUS); + containerStatus.setState(ContainerState.COMPLETE); + return containerStatus; + } + + +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/Queue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java similarity index 92% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/Queue.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index 446ff8f822..f2c9533a22 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/Queue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -26,6 +26,7 @@ import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.Resource; @@ -35,18 +36,18 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; /** - * Queue represents a node in the tree of + * CSQueue represents a node in the tree of * hierarchical queues in the {@link CapacityScheduler}. */ @Stable @Private -public interface Queue +public interface CSQueue extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue { /** * Get the parent Queue. * @return the parent queue */ - public Queue getParent(); + public CSQueue getParent(); /** * Get the queue name. @@ -122,7 +123,7 @@ public interface Queue * Get child queues * @return child queues */ - public List getChildQueues(); + public List getChildQueues(); /** * Check if the user has permission to perform the operation @@ -165,11 +166,14 @@ public void submitApplication(SchedulerApp application, String user, * @param node node on which the container completed * @param container completed container, * null if it was just a reservation + * @param containerStatus ContainerStatus for the completed + * container * @param event event to be sent to the container */ public void completedContainer(Resource clusterResource, SchedulerApp application, SchedulerNode node, - RMContainer container, RMContainerEventType event); + RMContainer container, ContainerStatus containerStatus, + RMContainerEventType event); /** * Get the number of applications in the queue. @@ -183,7 +187,7 @@ public void completedContainer(Resource clusterResource, * @param queue new queue to re-initalize from * @param clusterResource resources in the cluster */ - public void reinitialize(Queue queue, Resource clusterResource) + public void reinitialize(CSQueue queue, Resource clusterResource) throws IOException; /** diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index 4fff1ff1a4..abbe0365bb 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -36,10 +36,9 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.Lock; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.QueueInfo; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; @@ -59,11 +58,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; @@ -80,14 +79,14 @@ public class CapacityScheduler private static final Log LOG = LogFactory.getLog(CapacityScheduler.class); - private Queue root; + private CSQueue root; private final static List EMPTY_CONTAINER_LIST = new ArrayList(); - static final Comparator queueComparator = new Comparator() { + static final Comparator queueComparator = new Comparator() { @Override - public int compare(Queue q1, Queue q2) { + public int compare(CSQueue q1, CSQueue q2) { if (q1.getUtilization() < q2.getUtilization()) { return -1; } else if (q1.getUtilization() > q2.getUtilization()) { @@ -110,7 +109,7 @@ public int compare(SchedulerApp a1, SchedulerApp a2) { private ContainerTokenSecretManager containerTokenSecretManager; private RMContext rmContext; - private Map queues = new ConcurrentHashMap(); + private Map queues = new ConcurrentHashMap(); private Map nodes = new ConcurrentHashMap(); @@ -127,7 +126,9 @@ public int compare(SchedulerApp a1, SchedulerApp a2) { private boolean initialized = false; - public Queue getRootQueue() { + public CapacityScheduler() {} + + public CSQueue getRootQueue() { return root; } @@ -207,7 +208,7 @@ public synchronized void reinitialize(Configuration conf, CapacitySchedulerConfiguration.PREFIX + ROOT; static class QueueHook { - public Queue hook(Queue queue) { + public CSQueue hook(CSQueue queue) { return queue; } } @@ -225,8 +226,8 @@ private void initializeQueues(CapacitySchedulerConfiguration conf) { private void reinitializeQueues(CapacitySchedulerConfiguration conf) throws IOException { // Parse new queues - Map newQueues = new HashMap(); - Queue newRoot = + Map newQueues = new HashMap(); + CSQueue newRoot = parseQueue(this, conf, null, ROOT, newQueues, queues, queueComparator, applicationComparator, noop); @@ -247,7 +248,7 @@ private void reinitializeQueues(CapacitySchedulerConfiguration conf) */ @Lock(CapacityScheduler.class) private void validateExistingQueues( - Map queues, Map newQueues) + Map queues, Map newQueues) throws IOException { for (String queue : queues.keySet()) { if (!newQueues.containsKey(queue)) { @@ -264,11 +265,11 @@ private void validateExistingQueues( */ @Lock(CapacityScheduler.class) private void addNewQueues( - Map queues, Map newQueues) + Map queues, Map newQueues) { - for (Map.Entry e : newQueues.entrySet()) { + for (Map.Entry e : newQueues.entrySet()) { String queueName = e.getKey(); - Queue queue = e.getValue(); + CSQueue queue = e.getValue(); if (!queues.containsKey(queueName)) { queues.put(queueName, queue); } @@ -276,15 +277,15 @@ private void addNewQueues( } @Lock(CapacityScheduler.class) - static Queue parseQueue( + static CSQueue parseQueue( CapacitySchedulerContext csContext, CapacitySchedulerConfiguration conf, - Queue parent, String queueName, Map queues, - Map oldQueues, - Comparator queueComparator, + CSQueue parent, String queueName, Map queues, + Map oldQueues, + Comparator queueComparator, Comparator applicationComparator, QueueHook hook) { - Queue queue; + CSQueue queue; String[] childQueueNames = conf.getQueues((parent == null) ? queueName : (parent.getQueuePath()+"."+queueName)); @@ -306,9 +307,9 @@ static Queue parseQueue( // Used only for unit tests queue = hook.hook(parentQueue); - List childQueues = new ArrayList(); + List childQueues = new ArrayList(); for (String childQueueName : childQueueNames) { - Queue childQueue = + CSQueue childQueue = parseQueue(csContext, conf, queue, childQueueName, queues, oldQueues, queueComparator, applicationComparator, hook); childQueues.add(childQueue); @@ -322,7 +323,7 @@ static Queue parseQueue( return queue; } - synchronized Queue getQueue(String queueName) { + synchronized CSQueue getQueue(String queueName) { return queues.get(queueName); } @@ -331,7 +332,7 @@ synchronized Queue getQueue(String queueName) { String queueName, String user) { // Sanity checks - Queue queue = getQueue(queueName); + CSQueue queue = getQueue(queueName); if (queue == null) { String message = "Application " + applicationAttemptId + " submitted by user " + user + " to unknown queue: " + queueName; @@ -392,12 +393,20 @@ private synchronized void doneApplication( // Release all the running containers for (RMContainer rmContainer : application.getLiveContainers()) { - completedContainer(rmContainer, RMContainerEventType.KILL); + completedContainer(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + rmContainer.getContainerId(), + SchedulerUtils.COMPLETED_APPLICATION), + RMContainerEventType.KILL); } // Release all reserved containers for (RMContainer rmContainer : application.getAllReservedContainers()) { - completedContainer(rmContainer, RMContainerEventType.KILL); + completedContainer(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + rmContainer.getContainerId(), + "Application Complete"), + RMContainerEventType.KILL); } // Clean up pending requests, metrics etc. @@ -405,7 +414,7 @@ private synchronized void doneApplication( // Inform the queue String queueName = application.getQueue().getQueueName(); - Queue queue = queues.get(queueName); + CSQueue queue = queues.get(queueName); if (!(queue instanceof LeafQueue)) { LOG.error("Cannot finish application " + "from non-leaf queue: " + queueName); @@ -445,7 +454,11 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId, "Trying to release container not owned by app or with invalid id", application.getApplicationId(), releasedContainerId); } - completedContainer(rmContainer, RMContainerEventType.RELEASED); + completedContainer(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + releasedContainerId, + SchedulerUtils.RELEASED_CONTAINER), + RMContainerEventType.RELEASED); } synchronized (application) { @@ -479,7 +492,7 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId, public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues, boolean recursive) throws IOException { - Queue queue = null; + CSQueue queue = null; synchronized (this) { queue = this.queues.get(queueName); @@ -521,22 +534,23 @@ private void normalizeRequest(ResourceRequest ask) { } private synchronized void nodeUpdate(RMNode nm, - Map> containers ) { + List newlyLaunchedContainers, + List completedContainers) { LOG.info("nodeUpdate: " + nm + " clusterResources: " + clusterResource); SchedulerNode node = getNode(nm.getNodeID()); - // Processing the current containers running/finished on node - for (List appContainers : containers.values()) { - for (Container container : appContainers) { - if (container.getState() == ContainerState.RUNNING) { - containerLaunchedOnNode(container, node); - } else { // has to be 'COMPLETE' - LOG.info("DEBUG --- Container FINISHED: " + container.getId()); - completedContainer(getRMContainer(container.getId()), - RMContainerEventType.FINISHED); - } - } + // Processing the newly launched containers + for (ContainerStatus launchedContainer : newlyLaunchedContainers) { + containerLaunchedOnNode(launchedContainer.getContainerId(), node); + } + + // Process completed containers + for (ContainerStatus completedContainer : completedContainers) { + ContainerId containerId = completedContainer.getContainerId(); + LOG.info("DEBUG --- Container FINISHED: " + containerId); + completedContainer(getRMContainer(containerId), + completedContainer, RMContainerEventType.FINISHED); } // Now node data structures are upto date and ready for scheduling. @@ -566,23 +580,24 @@ private synchronized void nodeUpdate(RMNode nm, } else { LOG.info("Skipping scheduling since node " + nm + " is reserved by application " + - node.getReservedContainer().getContainerId().getAppId()); + node.getReservedContainer().getContainerId().getApplicationAttemptId() + ); } } - private void containerLaunchedOnNode(Container container, SchedulerNode node) { + private void containerLaunchedOnNode(ContainerId containerId, SchedulerNode node) { // Get the application for the finished container - ApplicationAttemptId applicationAttemptId = container.getId().getAppAttemptId(); + ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); SchedulerApp application = getApplication(applicationAttemptId); if (application == null) { LOG.info("Unknown application: " + applicationAttemptId + - " launched container " + container.getId() + + " launched container " + containerId + " on node: " + node); return; } - application.containerLaunchedOnNode(container.getId()); + application.containerLaunchedOnNode(containerId); } @Override @@ -604,7 +619,8 @@ public void handle(SchedulerEvent event) { { NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event; nodeUpdate(nodeUpdatedEvent.getRMNode(), - nodeUpdatedEvent.getContainers()); + nodeUpdatedEvent.getNewlyLaunchedContainers(), + nodeUpdatedEvent.getCompletedContainers()); } break; case APP_ADDED: @@ -625,7 +641,11 @@ public void handle(SchedulerEvent event) { { ContainerExpiredSchedulerEvent containerExpiredEvent = (ContainerExpiredSchedulerEvent) event; - completedContainer(getRMContainer(containerExpiredEvent.getContainerId()), + ContainerId containerId = containerExpiredEvent.getContainerId(); + completedContainer(getRMContainer(containerId), + SchedulerUtils.createAbnormalContainerStatus( + containerId, + SchedulerUtils.EXPIRED_CONTAINER), RMContainerEventType.EXPIRE); } break; @@ -652,13 +672,21 @@ private synchronized void removeNode(RMNode nodeInfo) { // Remove running containers List runningContainers = node.getRunningContainers(); for (RMContainer container : runningContainers) { - completedContainer(container, RMContainerEventType.KILL); + completedContainer(container, + SchedulerUtils.createAbnormalContainerStatus( + container.getContainerId(), + SchedulerUtils.LOST_CONTAINER), + RMContainerEventType.KILL); } // Remove reservations, if any RMContainer reservedContainer = node.getReservedContainer(); if (reservedContainer != null) { - completedContainer(reservedContainer, RMContainerEventType.KILL); + completedContainer(reservedContainer, + SchedulerUtils.createAbnormalContainerStatus( + reservedContainer.getContainerId(), + SchedulerUtils.LOST_CONTAINER), + RMContainerEventType.KILL); } this.nodes.remove(nodeInfo.getNodeID()); @@ -667,8 +695,8 @@ private synchronized void removeNode(RMNode nodeInfo) { } @Lock(CapacityScheduler.class) - private synchronized void completedContainer(RMContainer rmContainer, - RMContainerEventType event) { + private synchronized void completedContainer(RMContainer rmContainer, + ContainerStatus containerStatus, RMContainerEventType event) { if (rmContainer == null) { LOG.info("Null container completed..."); return; @@ -677,7 +705,7 @@ private synchronized void completedContainer(RMContainer rmContainer, Container container = rmContainer.getContainer(); // Get the application for the finished container - ApplicationAttemptId applicationAttemptId = container.getId().getAppAttemptId(); + ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId(); SchedulerApp application = getApplication(applicationAttemptId); if (application == null) { LOG.info("Container " + container + " of" + @@ -692,7 +720,7 @@ private synchronized void completedContainer(RMContainer rmContainer, // Inform the queue LeafQueue queue = (LeafQueue)application.getQueue(); queue.completedContainer(clusterResource, application, node, - rmContainer, event); + rmContainer, containerStatus, event); LOG.info("Application " + applicationAttemptId + " released container " + container.getId() + @@ -712,7 +740,7 @@ SchedulerNode getNode(NodeId nodeId) { private RMContainer getRMContainer(ContainerId containerId) { SchedulerApp application = - getApplication(containerId.getAppAttemptId()); + getApplication(containerId.getApplicationAttemptId()); return (application == null) ? null : application.getRMContainer(containerId); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java index 345381651c..f24307d5b8 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java @@ -40,7 +40,7 @@ public class CapacitySchedulerConfiguration extends Configuration { private static final String CS_CONFIGURATION_FILE = "capacity-scheduler.xml"; @Private - public static final String PREFIX = "yarn.capacity-scheduler."; + public static final String PREFIX = "yarn.scheduler.capacity."; @Private public static final String DOT = "."; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index 3d3ac1265e..2038e2d871 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -39,6 +39,7 @@ import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerToken; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.QueueACL; @@ -59,16 +60,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.util.BuilderUtils; @Private @Unstable -public class LeafQueue implements Queue { +public class LeafQueue implements CSQueue { private static final Log LOG = LogFactory.getLog(LeafQueue.class); private final String queueName; - private Queue parent; + private CSQueue parent; private float capacity; private float absoluteCapacity; private float maximumCapacity; @@ -119,8 +121,8 @@ public class LeafQueue implements Queue { final static int DEFAULT_AM_RESOURCE = 2 * 1024; public LeafQueue(CapacitySchedulerContext cs, - String queueName, Queue parent, - Comparator applicationComparator, Queue old) { + String queueName, CSQueue parent, + Comparator applicationComparator, CSQueue old) { this.scheduler = cs; this.queueName = queueName; this.parent = parent; @@ -192,7 +194,7 @@ private int computeMaxActiveApplications(Resource clusterResource, float maxAMResourcePercent, float absoluteCapacity) { return Math.max( - (int)((clusterResource.getMemory() / DEFAULT_AM_RESOURCE) * + (int)((clusterResource.getMemory() / (float)DEFAULT_AM_RESOURCE) * maxAMResourcePercent * absoluteCapacity), 1); } @@ -271,7 +273,7 @@ public synchronized float getAbsoluteMaximumCapacity() { } @Override - public Queue getParent() { + public CSQueue getParent() { return parent; } @@ -313,15 +315,15 @@ public int getMaxApplications() { return maxApplications; } - public int getMaxApplicationsPerUser() { + public synchronized int getMaxApplicationsPerUser() { return maxApplicationsPerUser; } - public int getMaximumActiveApplications() { + public synchronized int getMaximumActiveApplications() { return maxActiveApplications; } - public int getMaximumActiveApplicationsPerUser() { + public synchronized int getMaximumActiveApplicationsPerUser() { return maxActiveApplicationsPerUser; } @@ -341,7 +343,7 @@ public synchronized float getUtilization() { } @Override - public List getChildQueues() { + public List getChildQueues() { return null; } @@ -381,7 +383,7 @@ synchronized void setUserLimitFactor(int userLimitFactor) { this.userLimitFactor = userLimitFactor; } - synchronized void setParentQueue(Queue parent) { + synchronized void setParentQueue(CSQueue parent) { this.parent = parent; } @@ -423,12 +425,12 @@ public synchronized QueueState getState() { } @Private - public int getUserLimit() { + public synchronized int getUserLimit() { return userLimit; } @Private - public float getUserLimitFactor() { + public synchronized float getUserLimitFactor() { return userLimitFactor; } @@ -480,7 +482,7 @@ private synchronized User getUser(String userName) { } @Override - public synchronized void reinitialize(Queue queue, Resource clusterResource) + public synchronized void reinitialize(CSQueue queue, Resource clusterResource) throws IOException { // Sanity check if (!(queue instanceof LeafQueue) || @@ -493,9 +495,10 @@ public synchronized void reinitialize(Queue queue, Resource clusterResource) setupQueueConfigs(leafQueue.capacity, leafQueue.absoluteCapacity, leafQueue.maximumCapacity, leafQueue.absoluteMaxCapacity, leafQueue.userLimit, leafQueue.userLimitFactor, - leafQueue.maxApplications, leafQueue.maxApplicationsPerUser, - leafQueue.maxActiveApplications, - leafQueue.maxActiveApplicationsPerUser, + leafQueue.maxApplications, + leafQueue.getMaxApplicationsPerUser(), + leafQueue.getMaximumActiveApplications(), + leafQueue.getMaximumActiveApplicationsPerUser(), leafQueue.state, leafQueue.acls); updateResource(clusterResource); @@ -761,7 +764,11 @@ private synchronized Resource assignReservedContainer(SchedulerApp application, // Release Container container = rmContainer.getContainer(); completedContainer(clusterResource, application, node, - rmContainer, RMContainerEventType.RELEASED); + rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + container.getId(), + SchedulerUtils.UNRESERVED_CONTAINER), + RMContainerEventType.RELEASED); return container.getResource(); } @@ -900,7 +907,7 @@ boolean needContainers(SchedulerApp application, Priority priority, Resource req // Protect against corner case where you need the whole node with // Math.min(nodeFactor, minimumAllocationFactor) starvation = - (int)((application.getReReservations(priority) / reservedContainers) * + (int)((application.getReReservations(priority) / (float)reservedContainers) * (1.0f - (Math.min(nodeFactor, getMinimumAllocationFactor()))) ); @@ -1174,7 +1181,7 @@ private void unreserve(SchedulerApp application, Priority priority, @Override public void completedContainer(Resource clusterResource, SchedulerApp application, SchedulerNode node, RMContainer rmContainer, - RMContainerEventType event) { + ContainerStatus containerStatus, RMContainerEventType event) { if (application != null) { // Careful! Locking order is important! synchronized (this) { @@ -1189,7 +1196,7 @@ public void completedContainer(Resource clusterResource, application.unreserve(node, rmContainer.getReservedPriority()); node.unreserveResource(application); } else { - application.containerCompleted(rmContainer, event); + application.containerCompleted(rmContainer, containerStatus, event); node.releaseContainer(container); } @@ -1209,7 +1216,7 @@ public void completedContainer(Resource clusterResource, // Inform the parent queue parent.completedContainer(clusterResource, application, - node, rmContainer, event); + node, rmContainer, null, event); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java index 4be8522c5e..6aa282798c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java @@ -37,6 +37,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueInfo; import org.apache.hadoop.yarn.api.records.QueueState; @@ -53,11 +54,11 @@ @Private @Evolving -public class ParentQueue implements Queue { +public class ParentQueue implements CSQueue { private static final Log LOG = LogFactory.getLog(ParentQueue.class); - private final Queue parent; + private final CSQueue parent; private final String queueName; private float capacity; @@ -68,8 +69,8 @@ public class ParentQueue implements Queue { private float usedCapacity = 0.0f; private float utilization = 0.0f; - private final Set childQueues; - private final Comparator queueComparator; + private final Set childQueues; + private final Comparator queueComparator; private Resource usedResources = Resources.createResource(0); @@ -94,7 +95,7 @@ public class ParentQueue implements Queue { RecordFactoryProvider.getRecordFactory(null); public ParentQueue(CapacitySchedulerContext cs, - String queueName, Comparator comparator, Queue parent, Queue old) { + String queueName, Comparator comparator, CSQueue parent, CSQueue old) { minimumAllocation = cs.getMinimumResourceCapability(); this.parent = parent; @@ -140,7 +141,7 @@ public ParentQueue(CapacitySchedulerContext cs, maximumCapacity, absoluteMaxCapacity, state, acls); this.queueComparator = comparator; - this.childQueues = new TreeSet(comparator); + this.childQueues = new TreeSet(comparator); LOG.info("Initialized parent-queue " + queueName + " name=" + queueName + @@ -180,11 +181,11 @@ private synchronized void setupQueueConfigs( } private static float PRECISION = 0.005f; // 0.05% precision - void setChildQueues(Collection childQueues) { + void setChildQueues(Collection childQueues) { // Validate float childCapacities = 0; - for (Queue queue : childQueues) { + for (CSQueue queue : childQueues) { childCapacities += queue.getCapacity(); } float delta = Math.abs(1.0f - childCapacities); // crude way to check @@ -200,7 +201,7 @@ void setChildQueues(Collection childQueues) { } @Override - public Queue getParent() { + public CSQueue getParent() { return parent; } @@ -251,8 +252,8 @@ public synchronized float getUtilization() { } @Override - public synchronized List getChildQueues() { - return new ArrayList(childQueues); + public synchronized List getChildQueues() { + return new ArrayList(childQueues); } public synchronized int getNumContainers() { @@ -280,7 +281,7 @@ public synchronized QueueInfo getQueueInfo( List childQueuesInfo = new ArrayList(); if (includeChildQueues) { - for (Queue child : childQueues) { + for (CSQueue child : childQueues) { // Get queue information recursively? childQueuesInfo.add( child.getQueueInfo(recursive, recursive)); @@ -319,7 +320,7 @@ public synchronized List getQueueUserAclInfo( userAcls.add(getUserAclInfo(user)); // Add children queue acls - for (Queue child : childQueues) { + for (CSQueue child : childQueues) { userAcls.addAll(child.getQueueUserAclInfo(user)); } return userAcls; @@ -333,7 +334,7 @@ public String toString() { } @Override - public synchronized void reinitialize(Queue queue, Resource clusterResource) + public synchronized void reinitialize(CSQueue queue, Resource clusterResource) throws IOException { // Sanity check if (!(queue instanceof ParentQueue) || @@ -346,13 +347,13 @@ public synchronized void reinitialize(Queue queue, Resource clusterResource) // Re-configure existing child queues and add new ones // The CS has already checked to ensure all existing child queues are present! - Map currentChildQueues = getQueues(childQueues); - Map newChildQueues = getQueues(parentQueue.childQueues); - for (Map.Entry e : newChildQueues.entrySet()) { + Map currentChildQueues = getQueues(childQueues); + Map newChildQueues = getQueues(parentQueue.childQueues); + for (Map.Entry e : newChildQueues.entrySet()) { String newChildQueueName = e.getKey(); - Queue newChildQueue = e.getValue(); + CSQueue newChildQueue = e.getValue(); - Queue childQueue = currentChildQueues.get(newChildQueueName); + CSQueue childQueue = currentChildQueues.get(newChildQueueName); if (childQueue != null){ childQueue.reinitialize(newChildQueue, clusterResource); LOG.info(getQueueName() + ": re-configured queue: " + childQueue); @@ -375,9 +376,9 @@ public synchronized void reinitialize(Queue queue, Resource clusterResource) updateResource(clusterResource); } - Map getQueues(Set queues) { - Map queuesMap = new HashMap(); - for (Queue queue : queues) { + Map getQueues(Set queues) { + Map queuesMap = new HashMap(); + for (CSQueue queue : queues) { queuesMap.put(queue.getQueueName(), queue); } return queuesMap; @@ -568,8 +569,8 @@ synchronized Resource assignContainersToChildQueues(Resource cluster, printChildQueues(); // Try to assign to most 'under-served' sub-queue - for (Iterator iter=childQueues.iterator(); iter.hasNext();) { - Queue childQueue = iter.next(); + for (Iterator iter=childQueues.iterator(); iter.hasNext();) { + CSQueue childQueue = iter.next(); LOG.info("DEBUG --- Trying to assign to" + " queue: " + childQueue.getQueuePath() + " stats: " + childQueue); @@ -595,7 +596,7 @@ synchronized Resource assignContainersToChildQueues(Resource cluster, String getChildQueuesToPrint() { StringBuilder sb = new StringBuilder(); - for (Queue q : childQueues) { + for (CSQueue q : childQueues) { sb.append(q.getQueuePath() + "(" + q.getUtilization() + "), "); } return sb.toString(); @@ -608,7 +609,7 @@ void printChildQueues() { @Override public void completedContainer(Resource clusterResource, SchedulerApp application, SchedulerNode node, - RMContainer rmContainer, RMContainerEventType event) { + RMContainer rmContainer, ContainerStatus containerStatus, RMContainerEventType event) { if (application != null) { // Careful! Locking order is important! // Book keeping @@ -626,7 +627,7 @@ public void completedContainer(Resource clusterResource, // Inform the parent if (parent != null) { parent.completedContainer(clusterResource, application, - node, rmContainer, event); + node, rmContainer, null, event); } } } @@ -648,7 +649,7 @@ synchronized void releaseResource(Resource clusterResource, @Override public synchronized void updateClusterResource(Resource clusterResource) { // Update all children - for (Queue childQueue : childQueues) { + for (CSQueue childQueue : childQueues) { childQueue.updateClusterResource(clusterResource); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeUpdateSchedulerEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeUpdateSchedulerEvent.java index d0f07f17e5..9f3bc1cce7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeUpdateSchedulerEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeUpdateSchedulerEvent.java @@ -23,26 +23,33 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; public class NodeUpdateSchedulerEvent extends SchedulerEvent { private final RMNode rmNode; - private final Map> containers; + private final List newlyLaunchedContainers; + private final List completedContainersStatuses; public NodeUpdateSchedulerEvent(RMNode rmNode, - Map> containers) { + List newlyLaunchedContainers, + List completedContainers) { super(SchedulerEventType.NODE_UPDATE); this.rmNode = rmNode; - this.containers = containers; + this.newlyLaunchedContainers = newlyLaunchedContainers; + this.completedContainersStatuses = completedContainers; } public RMNode getRMNode() { return rmNode; } - public Map> getContainers() { - return containers; + public List getNewlyLaunchedContainers() { + return newlyLaunchedContainers; } + public List getCompletedContainers() { + return completedContainersStatuses; + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index 06aea2c9a4..9b4b3169ff 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -39,10 +39,9 @@ import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.yarn.Lock; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerToken; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; @@ -51,7 +50,6 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; @@ -67,7 +65,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; @@ -75,6 +72,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; @@ -84,6 +82,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.util.BuilderUtils; +import org.apache.hadoop.yarn.api.records.QueueState; @LimitedPrivate("yarn") @Evolving @@ -91,7 +90,7 @@ public class FifoScheduler implements ResourceScheduler { private static final Log LOG = LogFactory.getLog(FifoScheduler.class); - private final RecordFactory recordFactory = + private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); Configuration conf; @@ -105,8 +104,7 @@ public class FifoScheduler implements ResourceScheduler { private static final int MINIMUM_MEMORY = 1024; - private static final String FIFO_PREFIX = - YarnConfiguration.RM_PREFIX + "fifo."; + private static final String FIFO_PREFIX = "yarn.scheduler.fifo."; @Private public static final String MINIMUM_ALLOCATION = FIFO_PREFIX + "minimum-allocation-mb"; @@ -147,6 +145,7 @@ public QueueInfo getQueueInfo( queueInfo.setCapacity(100.0f); queueInfo.setMaximumCapacity(100.0f); queueInfo.setChildQueues(new ArrayList()); + queueInfo.setQueueState(QueueState.RUNNING); return queueInfo; } @@ -235,7 +234,11 @@ public Allocation allocate( "Trying to release container not owned by app or with invalid id", application.getApplicationId(), releasedContainer); } - containerCompleted(rmContainer, RMContainerEventType.RELEASED); + containerCompleted(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + releasedContainer, + SchedulerUtils.RELEASED_CONTAINER), + RMContainerEventType.RELEASED); } if (!ask.isEmpty()) { @@ -313,7 +316,11 @@ private synchronized void doneApplication( // Kill all 'live' containers for (RMContainer container : application.getLiveContainers()) { - containerCompleted(container, RMContainerEventType.KILL); + containerCompleted(container, + SchedulerUtils.createAbnormalContainerStatus( + container.getContainerId(), + SchedulerUtils.COMPLETED_APPLICATION), + RMContainerEventType.KILL); } // Clean up pending requests, metrics etc. @@ -543,25 +550,22 @@ private int assignContainer(SchedulerNode node, SchedulerApp application, return assignedContainers; } - private synchronized void nodeUpdate(RMNode rmNode, - Map> remoteContainers) { + private synchronized void nodeUpdate(RMNode rmNode, + List newlyLaunchedContainers, + List completedContainers) { SchedulerNode node = getNode(rmNode.getNodeID()); - for (List appContainers : remoteContainers.values()) { - for (Container container : appContainers) { - /* make sure the scheduler hasnt already removed the applications */ - if (getApplication(container.getId().getAppAttemptId()) != null) { - if (container.getState() == ContainerState.RUNNING) { - containerLaunchedOnNode(container, node); - } else { // has to COMPLETE - containerCompleted(getRMContainer(container.getId()), - RMContainerEventType.FINISHED); - } - } - else { - LOG.warn("Scheduler not tracking application " + container.getId().getAppAttemptId()); - } - } + // Processing the newly launched containers + for (ContainerStatus launchedContainer : newlyLaunchedContainers) { + containerLaunchedOnNode(launchedContainer.getContainerId(), node); + } + + // Process completed containers + for (ContainerStatus completedContainer : completedContainers) { + ContainerId containerId = completedContainer.getContainerId(); + LOG.info("DEBUG --- Container FINISHED: " + containerId); + containerCompleted(getRMContainer(containerId), + completedContainer, RMContainerEventType.FINISHED); } if (Resources.greaterThanOrEqual(node.getAvailableResource(), @@ -599,7 +603,8 @@ public void handle(SchedulerEvent event) { NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event; nodeUpdate(nodeUpdatedEvent.getRMNode(), - nodeUpdatedEvent.getContainers()); + nodeUpdatedEvent.getNewlyLaunchedContainers(), + nodeUpdatedEvent.getCompletedContainers()); } break; case APP_ADDED: @@ -625,7 +630,11 @@ public void handle(SchedulerEvent event) { { ContainerExpiredSchedulerEvent containerExpiredEvent = (ContainerExpiredSchedulerEvent) event; - containerCompleted(getRMContainer(containerExpiredEvent.getContainerId()), + ContainerId containerid = containerExpiredEvent.getContainerId(); + containerCompleted(getRMContainer(containerid), + SchedulerUtils.createAbnormalContainerStatus( + containerid, + SchedulerUtils.EXPIRED_CONTAINER), RMContainerEventType.EXPIRE); } break; @@ -634,23 +643,23 @@ public void handle(SchedulerEvent event) { } } - private void containerLaunchedOnNode(Container container, SchedulerNode node) { + private void containerLaunchedOnNode(ContainerId containerId, SchedulerNode node) { // Get the application for the finished container - ApplicationAttemptId applicationAttemptId = container.getId().getAppAttemptId(); + ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); SchedulerApp application = getApplication(applicationAttemptId); if (application == null) { LOG.info("Unknown application: " + applicationAttemptId + - " launched container " + container.getId() + + " launched container " + containerId + " on node: " + node); return; } - application.containerLaunchedOnNode(container.getId()); + application.containerLaunchedOnNode(containerId); } @Lock(FifoScheduler.class) private synchronized void containerCompleted(RMContainer rmContainer, - RMContainerEventType event) { + ContainerStatus containerStatus, RMContainerEventType event) { if (rmContainer == null) { LOG.info("Null container completed..."); return; @@ -658,7 +667,7 @@ private synchronized void containerCompleted(RMContainer rmContainer, // Get the application for the finished container Container container = rmContainer.getContainer(); - ApplicationAttemptId applicationAttemptId = container.getId().getAppAttemptId(); + ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId(); SchedulerApp application = getApplication(applicationAttemptId); // Get the node on which the container was allocated @@ -673,7 +682,7 @@ private synchronized void containerCompleted(RMContainer rmContainer, } // Inform the application - application.containerCompleted(rmContainer, event); + application.containerCompleted(rmContainer, containerStatus, event); // Inform the node node.releaseContainer(container); @@ -692,7 +701,11 @@ private synchronized void removeNode(RMNode nodeInfo) { SchedulerNode node = getNode(nodeInfo.getNodeID()); // Kill running containers for(RMContainer container : node.getRunningContainers()) { - containerCompleted(container, RMContainerEventType.KILL); + containerCompleted(container, + SchedulerUtils.createAbnormalContainerStatus( + container.getContainerId(), + SchedulerUtils.LOST_CONTAINER), + RMContainerEventType.KILL); } //Remove the node @@ -738,7 +751,7 @@ public synchronized SchedulerNodeReport getNodeReport(NodeId nodeId) { private RMContainer getRMContainer(ContainerId containerId) { SchedulerApp application = - getApplication(containerId.getAppAttemptId()); + getApplication(containerId.getApplicationAttemptId()); return (application == null) ? null : application.getRMContainer(containerId); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/tools/RMAdmin.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/tools/RMAdmin.java index 0f1bd7ce6b..3c367a6a6b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/tools/RMAdmin.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/tools/RMAdmin.java @@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.admin.AdminSecurityInfo; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocol; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsRequest; import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesRequest; @@ -146,8 +145,8 @@ private RMAdminProtocol createAdminProtocol() throws IOException { // Create the client final String adminAddress = - conf.get(RMConfig.ADMIN_ADDRESS, - RMConfig.DEFAULT_ADMIN_BIND_ADDRESS); + conf.get(YarnConfiguration.RM_ADMIN_ADDRESS, + YarnConfiguration.RM_ADMIN_ADDRESS); final YarnRPC rpc = YarnRPC.create(conf); if (UserGroupInformation.isSecurityEnabled()) { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java index cf4d94b1ee..94649923cb 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java @@ -56,7 +56,7 @@ class AppsBlock extends HtmlBlock { tbody(); int i = 0; for (RMApp app : list.apps.values()) { - String appId = Apps.toString(app.getApplicationId()); + String appId = app.getApplicationId().toString(); String trackingUrl = app.getTrackingUrl(); String ui = trackingUrl == null || trackingUrl.isEmpty() ? "UNASSIGNED" : (app.getFinishTime() == 0 ? "ApplicationMaster" : "JobHistory"); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsList.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsList.java index a0dfa783a6..ba6b0ea231 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsList.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsList.java @@ -60,7 +60,7 @@ void toDataTableArrays(PrintWriter out) { } else { out.append(",\n"); } - String appID = Apps.toString(app.getApplicationId()); + String appID = app.getApplicationId().toString(); String trackingUrl = app.getTrackingUrl(); String ui = trackingUrl == null ? "UNASSIGNED" : (app.getFinishTime() == 0 ? "ApplicationMaster" : "JobHistory"); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java index 76b0f0eb0c..f36e181502 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java @@ -24,7 +24,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Queue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue; import org.apache.hadoop.yarn.webapp.SubView; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.*; @@ -43,7 +43,7 @@ class CapacitySchedulerPage extends RmView { @RequestScoped static class Parent { - Queue queue; + CSQueue queue; } public static class QueueBlock extends HtmlBlock { @@ -56,8 +56,8 @@ public static class QueueBlock extends HtmlBlock { @Override public void render(Block html) { UL ul = html.ul(); - Queue parentQueue = parent.queue; - for (Queue queue : parentQueue.getChildQueues()) { + CSQueue parentQueue = parent.queue; + for (CSQueue queue : parentQueue.getChildQueues()) { float used = queue.getUsedCapacity(); float set = queue.getCapacity(); float delta = Math.abs(set - used) + 0.001f; @@ -109,7 +109,7 @@ public void render(Block html) { span().$style(Q_END)._("100% ")._(). span(".q", "default")._()._(); } else { - Queue root = cs.getRootQueue(); + CSQueue root = cs.getRootQueue(); parent.queue = root; float used = root.getUsedCapacity(); float set = root.getCapacity(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java index a0a3030be3..a232e5bcc8 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java @@ -18,20 +18,177 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; -import org.apache.hadoop.yarn.webapp.SubView; -import org.apache.hadoop.yarn.webapp.view.HtmlBlock; +import com.google.inject.Inject; +import com.google.inject.servlet.RequestScoped; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.*; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.QueueState; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.webapp.view.InfoBlock; + +import static org.apache.hadoop.yarn.util.StringHelper.*; import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*; class DefaultSchedulerPage extends RmView { + static final String _Q = ".ui-state-default.ui-corner-all"; + static final float WIDTH_F = 0.8f; + static final String Q_END = "left:101%"; + static final String OVER = "font-size:1px;background:rgba(255, 140, 0, 0.8)"; + static final String UNDER = "font-size:1px;background:rgba(50, 205, 50, 0.8)"; + static final float EPSILON = 1e-8f; + + static class QueueInfoBlock extends HtmlBlock { + final RMContext rmContext; + final FifoScheduler fs; + final String qName; + final QueueInfo qInfo; + + @Inject QueueInfoBlock(RMContext context, ViewContext ctx, ResourceManager rm) { + super(ctx); + this.rmContext = context; + + fs = (FifoScheduler) rm.getResourceScheduler(); + qName = fs.getQueueInfo("",false,false).getQueueName(); + qInfo = fs.getQueueInfo(qName,true,true); + } - static class QueueBlock extends HtmlBlock { @Override public void render(Block html) { - html.h2("Under construction"); + String minmemoryresource = + Integer.toString(fs.getMinimumResourceCapability().getMemory()); + String maxmemoryresource = + Integer.toString(fs.getMaximumResourceCapability().getMemory()); + String qstate = (qInfo.getQueueState() == QueueState.RUNNING) ? + "Running" : + (qInfo.getQueueState() == QueueState.STOPPED) ? + "Stopped" : "Unknown"; + + int usedNodeMem = 0; + int availNodeMem = 0; + int totNodeMem = 0; + int nodeContainers = 0; + + for (RMNode ni : this.rmContext.getRMNodes().values()) { + usedNodeMem += fs.getUsedResource(ni.getNodeID()).getMemory(); + availNodeMem += fs.getAvailableResource(ni.getNodeID()).getMemory(); + totNodeMem += ni.getTotalCapability().getMemory(); + nodeContainers += fs.getNodeReport(ni.getNodeID()).getNumContainers(); + } + + info("\'" + qName + "\' Queue Status"). + _("Queue State:" , qstate). + _("Minimum Queue Memory Capacity:" , minmemoryresource). + _("Maximum Queue Memory Capacity:" , maxmemoryresource). + _("Number of Nodes:" , Integer.toString(this.rmContext.getRMNodes().size())). + _("Used Node Capacity:" , Integer.toString(usedNodeMem)). + _("Available Node Capacity:" , Integer.toString(availNodeMem)). + _("Total Node Capacity:" , Integer.toString(totNodeMem)). + _("Number of Node Containers:" , Integer.toString(nodeContainers)); + + html._(InfoBlock.class); } } + static class QueuesBlock extends HtmlBlock { + final FifoScheduler fs; + final String qName; + final QueueInfo qInfo; + + @Inject QueuesBlock(ResourceManager rm) { + fs = (FifoScheduler) rm.getResourceScheduler(); + qName = fs.getQueueInfo("",false,false).getQueueName(); + qInfo = fs.getQueueInfo(qName,false,false); + } + + @Override + public void render(Block html) { + UL>> ul = html. + div("#cs-wrapper.ui-widget"). + div(".ui-widget-header.ui-corner-top"). + _("FifoScheduler Queue")._(). + div("#cs.ui-widget-content.ui-corner-bottom"). + ul(); + + if (fs == null) { + ul. + li(). + a(_Q).$style(width(WIDTH_F)). + span().$style(Q_END)._("100% ")._(). + span(".q", "default")._()._(); + } else { + float used = qInfo.getCurrentCapacity() / 100.0f; + float set = qInfo.getCapacity() / 100.0f; + float delta = Math.abs(set - used) + 0.001f; + ul. + li(). + a(_Q).$style(width(WIDTH_F)). + $title(join("used:", percent(used))). + span().$style(Q_END)._("100%")._(). + span().$style(join(width(delta), ';', used > set ? OVER : UNDER, + ';', used > set ? left(set) : left(used)))._(".")._(). + span(".q", qName)._(). + _(QueueInfoBlock.class)._(); + } + + ul._()._(). + script().$type("text/javascript"). + _("$('#cs').hide();")._()._(). + _(AppsBlock.class); + } + } + + + @Override protected void postHead(Page.HTML<_> html) { + html. + style().$type("text/css"). + _("#cs { padding: 0.5em 0 1em 0; margin-bottom: 1em; position: relative }", + "#cs ul { list-style: none }", + "#cs a { font-weight: normal; margin: 2px; position: relative }", + "#cs a span { font-weight: normal; font-size: 80% }", + "#cs-wrapper .ui-widget-header { padding: 0.2em 0.5em }", + "table.info tr th {width: 50%}")._(). // to center info table + script("/static/jt/jquery.jstree.js"). + script().$type("text/javascript"). + _("$(function() {", + " $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');", + " $('#cs').bind('loaded.jstree', function (e, data) {", + " data.inst.open_all(); }).", + " jstree({", + " core: { animation: 188, html_titles: true },", + " plugins: ['themeroller', 'html_data', 'ui'],", + " themeroller: { item_open: 'ui-icon-minus',", + " item_clsd: 'ui-icon-plus', item_leaf: 'ui-icon-gear'", + " }", + " });", + " $('#cs').bind('select_node.jstree', function(e, data) {", + " var q = $('.q', data.rslt.obj).first().text();", + " if (q == 'root') q = '';", + " $('#apps').dataTable().fnFilter(q, 3);", + " });", + " $('#cs').show();", + "});")._(); + } + @Override protected Class content() { - return QueueBlock.class; + return QueuesBlock.class; + } + + static String percent(float f) { + return String.format("%.1f%%", f * 100); + } + + static String width(float f) { + return String.format("width:%.1f%%", f * 100); + } + + static String left(float f) { + return String.format("left:%.1f%%", f * 100); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java index 4231c4c838..a621cc1047 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java @@ -52,8 +52,8 @@ protected void render(Block html) { thead(). tr(). th(".rack", "Rack"). - th(".nodeid", "Node ID"). - th(".host", "Host"). + th(".nodeaddress", "Node Address"). + th(".nodehttpaddress", "Node HTTP Address"). th(".healthStatus", "Health-status"). th(".lastHealthUpdate", "Last health-update"). th(".healthReport", "Health-report"). diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/capacity-scheduler.xml b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/capacity-scheduler.xml index 43a0437b9d..6db99098ec 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/capacity-scheduler.xml +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/capacity-scheduler.xml @@ -1,58 +1,79 @@ - yarn.capacity-scheduler.maximum-applications + yarn.scheduler.capacity.maximum-applications 10000 + Maximum number of applications that can be running. + - yarn.capacity-scheduler.maximum-am-resource-percent + yarn.scheduler.capacity.maximum-am-resource-percent 0.1 - yarn.capacity-scheduler.root.queues + yarn.scheduler.capacity.root.queues default + The queues at the this level (root is the root queue). + - yarn.capacity-scheduler.root.capacity + yarn.scheduler.capacity.root.capacity 100 + The total capacity as a percentage out of 100 for this queue. + If it has child queues then this includes their capacity as well. + The child queues capacity should add up to their parent queue's capacity + or less. - yarn.capacity-scheduler.root.acl_administer_queues + yarn.scheduler.capacity.root.acl_administer_queues * + The ACL for who can administer this queue. i.e. + change sub queue allocations. - yarn.capacity-scheduler.root.default.capacity + yarn.scheduler.capacity.root.default.capacity 100 + default queue target capacity. - yarn.capacity-scheduler.root.default.user-limit-factor + yarn.scheduler.capacity.root.default.user-limit-factor 1 + default queue user limit a percantage from 0.0 to 1.0. + - yarn.capacity-scheduler.root.default.maximum-capacity + yarn.scheduler.capacity.root.default.maximum-capacity -1 + the maximum capacity of the default queue -1 disables. + - yarn.capacity-scheduler.root.default.state + yarn.scheduler.capacity.root.default.state RUNNING + The state of the default queue. can be RUNNING or STOPPED + - yarn.capacity-scheduler.root.default.acl_submit_jobs + yarn.scheduler.capacity.root.default.acl_submit_jobs * + The ACL of who can submit jobs to the default queue. + - yarn.capacity-scheduler.root.default.acl_administer_jobs + yarn.scheduler.capacity.root.default.acl_administer_jobs * + The ACL of who can administer jobs on the default queue. + diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java index e7178d013e..7536857b2c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java @@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; @@ -53,9 +54,10 @@ public NodeId getNodeId() { } public void containerStatus(Container container) throws Exception { - Map> conts = new HashMap>(); - conts.put(container.getId().getAppId(), Arrays - .asList(new Container[] { container })); + Map> conts = + new HashMap>(); + conts.put(container.getId().getApplicationAttemptId().getApplicationId(), + Arrays.asList(new ContainerStatus[] { container.getContainerStatus() })); nodeHeartbeat(conts, true); } @@ -76,16 +78,16 @@ public NodeId registerNode() throws Exception { } public HeartbeatResponse nodeHeartbeat(boolean b) throws Exception { - return nodeHeartbeat(new HashMap>(), b); + return nodeHeartbeat(new HashMap>(), b); } public HeartbeatResponse nodeHeartbeat(Map> conts, boolean isHealthy) throws Exception { + List> conts, boolean isHealthy) throws Exception { NodeHeartbeatRequest req = Records.newRecord(NodeHeartbeatRequest.class); NodeStatus status = Records.newRecord(NodeStatus.class); status.setNodeId(nodeId); - for (Map.Entry> entry : conts.entrySet()) { - status.setContainers(entry.getKey(), entry.getValue()); + for (Map.Entry> entry : conts.entrySet()) { + status.setContainersStatuses(entry.getValue()); } NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class); healthStatus.setHealthReport(""); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java index 59da09652a..901948fab7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java @@ -60,13 +60,9 @@ public MockRM(Configuration conf) { public void waitForState(ApplicationId appId, RMAppState finalState) throws Exception { + RMApp app = getRMContext().getRMApps().get(appId); + Assert.assertNotNull("app shouldn't be null", app); int timeoutSecs = 0; - RMApp app = null; - while ((app == null) && timeoutSecs++ < 20) { - app = getRMContext().getRMApps().get(appId); - Thread.sleep(500); - } - timeoutSecs = 0; while (!finalState.equals(app.getState()) && timeoutSecs++ < 20) { System.out.println("App State is : " + app.getState() + @@ -95,6 +91,7 @@ public RMApp submitApp(int masterMemory) throws Exception { req.setApplicationSubmissionContext(sub); client.submitApplication(req); + // make sure app is immediately available after submit waitForState(appId, RMAppState.ACCEPTED); return getRMContext().getRMApps().get(appId); } @@ -131,7 +128,7 @@ public void sendAMLaunchFailed(ApplicationAttemptId appAttemptId) throws Excepti @Override protected ClientRMService createClientRMService() { - return new ClientRMService(getRMContext(), getResourceScheduler()) { + return new ClientRMService(getRMContext(), getResourceScheduler(), rmAppManager) { @Override public void start() { //override to not start rpc handler diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java index 23871a7c49..a7b5d02c91 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java @@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; @@ -53,9 +54,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse; import org.apache.hadoop.yarn.server.api.records.NodeStatus; -import org.apache.hadoop.yarn.server.api.records.RegistrationResponse; import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; -import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.util.BuilderUtils; @@ -133,10 +132,19 @@ public Resource getUsed() { int responseID = 0; + private List getContainerStatuses(Map> containers) { + List containerStatuses = new ArrayList(); + for (List appContainers : containers.values()) { + for (Container container : appContainers) { + containerStatuses.add(container.getContainerStatus()); + } + } + return containerStatuses; + } public void heartbeat() throws IOException { NodeStatus nodeStatus = org.apache.hadoop.yarn.server.resourcemanager.NodeManager.createNodeStatus( - nodeId, containers); + nodeId, getContainerStatuses(containers)); nodeStatus.setResponseId(responseID); NodeHeartbeatRequest request = recordFactory .newRecordInstance(NodeHeartbeatRequest.class); @@ -147,11 +155,15 @@ public void heartbeat() throws IOException { } @Override - synchronized public StartContainerResponse startContainer(StartContainerRequest request) throws YarnRemoteException { - ContainerLaunchContext containerLaunchContext = request.getContainerLaunchContext(); + synchronized public StartContainerResponse startContainer( + StartContainerRequest request) + throws YarnRemoteException { + ContainerLaunchContext containerLaunchContext = + request.getContainerLaunchContext(); - ApplicationId applicationId = containerLaunchContext.getContainerId() - .getAppId(); + ApplicationId applicationId = + containerLaunchContext.getContainerId().getApplicationAttemptId(). + getApplicationId(); List applicationContainers = containers.get(applicationId); if (applicationContainers == null) { @@ -161,7 +173,8 @@ synchronized public StartContainerResponse startContainer(StartContainerRequest // Sanity check for (Container container : applicationContainers) { - if (container.getId().compareTo(containerLaunchContext.getContainerId()) == 0) { + if (container.getId().compareTo(containerLaunchContext.getContainerId()) + == 0) { throw new IllegalStateException( "Container " + containerLaunchContext.getContainerId() + " already setup on node " + containerManagerAddress); @@ -201,7 +214,8 @@ synchronized public void checkResourceUsage() { synchronized public StopContainerResponse stopContainer(StopContainerRequest request) throws YarnRemoteException { ContainerId containerID = request.getContainerId(); - String applicationId = String.valueOf(containerID.getAppId().getId()); + String applicationId = String.valueOf( + containerID.getApplicationAttemptId().getApplicationId().getId()); // Mark the container as COMPLETE List applicationContainers = containers.get(applicationId); @@ -250,17 +264,31 @@ synchronized public StopContainerResponse stopContainer(StopContainerRequest req @Override synchronized public GetContainerStatusResponse getContainerStatus(GetContainerStatusRequest request) throws YarnRemoteException { - ContainerId containerID = request.getContainerId(); - GetContainerStatusResponse response = recordFactory.newRecordInstance(GetContainerStatusResponse.class); + ContainerId containerId = request.getContainerId(); + List appContainers = + containers.get( + containerId.getApplicationAttemptId().getApplicationId()); + Container container = null; + for (Container c : appContainers) { + if (c.getId().equals(containerId)) { + container = c; + } + } + GetContainerStatusResponse response = + recordFactory.newRecordInstance(GetContainerStatusResponse.class); + if (container != null && container.getContainerStatus() != null) { + response.setStatus(container.getContainerStatus()); + } return response; } - public static org.apache.hadoop.yarn.server.api.records.NodeStatus createNodeStatus( - NodeId nodeId, Map> containers) { + public static org.apache.hadoop.yarn.server.api.records.NodeStatus + createNodeStatus(NodeId nodeId, List containers) { RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); - org.apache.hadoop.yarn.server.api.records.NodeStatus nodeStatus = recordFactory.newRecordInstance(org.apache.hadoop.yarn.server.api.records.NodeStatus.class); + org.apache.hadoop.yarn.server.api.records.NodeStatus nodeStatus = + recordFactory.newRecordInstance(org.apache.hadoop.yarn.server.api.records.NodeStatus.class); nodeStatus.setNodeId(nodeId); - nodeStatus.addAllContainers(containers); + nodeStatus.setContainersStatuses(containers); NodeHealthStatus nodeHealthStatus = recordFactory.newRecordInstance(NodeHealthStatus.class); nodeHealthStatus.setIsNodeHealthy(true); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index 3109198d97..bd66a6337f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -32,6 +32,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +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.api.records.ApplicationId; @@ -47,7 +48,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManager; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp; @@ -153,14 +153,14 @@ public class TestRMAppManager extends RMAppManager { public TestRMAppManager(RMContext context, Configuration conf) { super(context, null, null, null, conf); - setCompletedAppsMax(RMConfig.DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX); + setCompletedAppsMax(YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS); } public TestRMAppManager(RMContext context, ClientToAMSecretManager clientToAMSecretManager, YarnScheduler scheduler, ApplicationMasterService masterService, Configuration conf) { super(context, clientToAMSecretManager, scheduler, masterService, conf); - setCompletedAppsMax(RMConfig.DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX); + setCompletedAppsMax(YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS); } public void checkAppNumCompletedLimit() { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java index 1da7733cee..79320b6eb7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java @@ -66,11 +66,11 @@ public void testAppCleanup() throws Exception { //kick the scheduler nm1.nodeHeartbeat(true); List conts = am.allocate(new ArrayList(), - new ArrayList()).getNewContainerList(); + new ArrayList()).getAllocatedContainers(); int contReceived = conts.size(); while (contReceived < request) { conts = am.allocate(new ArrayList(), - new ArrayList()).getNewContainerList(); + new ArrayList()).getAllocatedContainers(); contReceived += conts.size(); Log.info("Got " + contReceived + " containers. Waiting to get " + request); Thread.sleep(2000); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java index 0fa897ac52..605a0f363b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java @@ -92,12 +92,12 @@ public void test() throws Exception { // kick the scheduler, 1 GB and 3 GB given to AM1 and AM2, remaining 0 nm1.nodeHeartbeat(true); - while (am1Response.getNewContainerCount() < 1) { + while (am1Response.getAllocatedContainers().size() < 1) { LOG.info("Waiting for containers to be created for app 1..."); Thread.sleep(1000); am1Response = am1.schedule(); } - while (am2Response.getNewContainerCount() < 1) { + while (am2Response.getAllocatedContainers().size() < 1) { LOG.info("Waiting for containers to be created for app 2..."); Thread.sleep(1000); am2Response = am2.schedule(); @@ -105,12 +105,12 @@ public void test() throws Exception { // kick the scheduler, nothing given remaining 2 GB. nm2.nodeHeartbeat(true); - List allocated1 = am1Response.getNewContainerList(); + List allocated1 = am1Response.getAllocatedContainers(); Assert.assertEquals(1, allocated1.size()); Assert.assertEquals(1 * GB, allocated1.get(0).getResource().getMemory()); Assert.assertEquals(nm1.getNodeId(), allocated1.get(0).getNodeId()); - List allocated2 = am2Response.getNewContainerList(); + List allocated2 = am2Response.getAllocatedContainers(); Assert.assertEquals(1, allocated2.size()); Assert.assertEquals(3 * GB, allocated2.get(0).getResource().getMemory()); Assert.assertEquals(nm1.getNodeId(), allocated2.get(0).getNodeId()); @@ -137,7 +137,7 @@ public void test() throws Exception { Thread.sleep(1000); } Assert.assertEquals(1, attempt1.getJustFinishedContainers().size()); - Assert.assertEquals(1, am1.schedule().getFinishedContainerList().size()); + Assert.assertEquals(1, am1.schedule().getCompletedContainersStatuses().size()); Assert.assertEquals(5 * GB, rm.getResourceScheduler().getUsedResource( nm1.getNodeId()).getMemory()); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java index dbce71e24d..03941e3625 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java @@ -86,11 +86,11 @@ public void testAppOnMultiNode() throws Exception { //kick the scheduler nm1.nodeHeartbeat(true); List conts = am.allocate(new ArrayList(), - new ArrayList()).getNewContainerList(); + new ArrayList()).getAllocatedContainers(); int contReceived = conts.size(); while (contReceived < 3) {//only 3 containers are available on node1 conts.addAll(am.allocate(new ArrayList(), - new ArrayList()).getNewContainerList()); + new ArrayList()).getAllocatedContainers()); contReceived = conts.size(); LOG.info("Got " + contReceived + " containers. Waiting to get " + 3); Thread.sleep(2000); @@ -100,11 +100,11 @@ public void testAppOnMultiNode() throws Exception { //send node2 heartbeat nm2.nodeHeartbeat(true); conts = am.allocate(new ArrayList(), - new ArrayList()).getNewContainerList(); + new ArrayList()).getAllocatedContainers(); contReceived = conts.size(); while (contReceived < 10) { conts.addAll(am.allocate(new ArrayList(), - new ArrayList()).getNewContainerList()); + new ArrayList()).getAllocatedContainers()); contReceived = conts.size(); LOG.info("Got " + contReceived + " containers. Waiting to get " + 10); Thread.sleep(2000); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java index 9291b49aba..4f6aaddb4b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.Keys; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.net.NetUtils; @@ -228,7 +229,8 @@ public void ping() { public void testRMAuditLoggerWithIP() throws Exception { Configuration conf = new Configuration(); // start the IPC server - Server server = RPC.getServer(new MyTestRPCServer(), "0.0.0.0", 0, conf); + Server server = RPC.getServer(TestProtocol.class, + new MyTestRPCServer(), "0.0.0.0", 0, 5, true, conf, null); server.start(); InetSocketAddress addr = NetUtils.getConnectAddress(server); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java index 95690f0f9b..fcc2c08344 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java @@ -27,19 +27,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationState; import org.apache.hadoop.yarn.api.records.ApplicationStatus; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore; -import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; -import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.util.Records; import com.google.common.collect.Lists; @@ -218,10 +212,10 @@ public void handle(RMAppEvent event) { } public static RMApp newApplication(int i) { - final ApplicationId id = newAppID(i); + final ApplicationAttemptId appAttemptId = newAppAttemptID(newAppID(i), 0); final Container masterContainer = Records.newRecord(Container.class); ContainerId containerId = Records.newRecord(ContainerId.class); - containerId.setAppId(id); + containerId.setApplicationAttemptId(appAttemptId); masterContainer.setId(containerId); masterContainer.setNodeHttpAddress("node:port"); final String user = newUserName(); @@ -233,7 +227,7 @@ public static RMApp newApplication(int i) { return new ApplicationBase() { @Override public ApplicationId getApplicationId() { - return id; + return appAttemptId.getApplicationId(); } @Override public String getUser() { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java index f8ec9f4764..85d8432535 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java @@ -46,7 +46,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java index 2116499b4e..61e3821857 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java @@ -50,7 +50,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java index 5885d95ace..8bbfd105c5 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; @@ -36,7 +37,6 @@ import org.apache.hadoop.yarn.server.api.records.RegistrationResponse; import org.apache.hadoop.yarn.server.resourcemanager.NMLivelinessMonitor; import org.apache.hadoop.yarn.server.resourcemanager.NodesListManager; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService; @@ -65,7 +65,7 @@ public TestNmLivelinessMonitor(Dispatcher dispatcher) { @Override public void init(Configuration conf) { - conf.setLong(RMConfig.NM_EXPIRY_INTERVAL, 1000); + conf.setLong(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 1000); super.init(conf); } @Override diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java index 7d261ed201..56bac77209 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java @@ -117,7 +117,7 @@ protected RMApp createNewTestApp() { String queue = MockApps.newQueue(); Configuration conf = new YarnConfiguration(); // ensure max retries set to known value - conf.setInt("yarn.server.resourcemanager.application.max.retries", maxRetries); + conf.setInt(YarnConfiguration.RM_AM_MAX_RETRIES, maxRetries); ApplicationSubmissionContext submissionContext = null; String clientTokenStr = "bogusstring"; ApplicationStore appStore = mock(ApplicationStore.class); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java index fe9b15b64f..4dc277ea1b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java @@ -38,8 +38,8 @@ public void setUp() { when(csContext.getMaximumResourceCapability()).thenReturn(Resources.createResource(16*GB)); when(csContext.getClusterResources()).thenReturn(Resources.createResource(10 * 16 * GB)); - Map queues = new HashMap(); - Queue root = + Map queues = new HashMap(); + CSQueue root = CapacityScheduler.parseQueue(csContext, csConf, null, "root", queues, queues, CapacityScheduler.queueComparator, @@ -108,8 +108,8 @@ public void testLimitsComputation() throws Exception { Resource clusterResource = Resources.createResource(100 * 16 * GB); when(csContext.getClusterResources()).thenReturn(clusterResource); - Map queues = new HashMap(); - Queue root = + Map queues = new HashMap(); + CSQueue root = CapacityScheduler.parseQueue(csContext, csConf, null, "root", queues, queues, CapacityScheduler.queueComparator, diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java index 1f4a19b387..3c110b2130 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java @@ -30,9 +30,9 @@ import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.server.resourcemanager.Application; -import org.apache.hadoop.yarn.server.resourcemanager.RMConfig; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.Task; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store; @@ -55,7 +55,7 @@ public void setUp() throws Exception { resourceManager = new ResourceManager(store); CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setClass(RMConfig.RESOURCE_SCHEDULER, + csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); setupQueueConfiguration(csConf); resourceManager.init(csConf); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java index a3ac403306..01acd1162f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -65,8 +66,8 @@ public class TestLeafQueue { CapacitySchedulerConfiguration csConf; CapacitySchedulerContext csContext; - Queue root; - Map queues = new HashMap(); + CSQueue root; + Map queues = new HashMap(); final static int GB = 1024; final static String DEFAULT_RACK = "/default"; @@ -145,10 +146,11 @@ public Container answer(InvocationOnMock invocation) any(Resource.class)); // 2. Stub out LeafQueue.parent.completedContainer - Queue parent = queue.getParent(); + CSQueue parent = queue.getParent(); doNothing().when(parent).completedContainer( any(Resource.class), any(SchedulerApp.class), any(SchedulerNode.class), - any(RMContainer.class), any(RMContainerEventType.class)); + any(RMContainer.class), any(ContainerStatus.class), + any(RMContainerEventType.class)); return queue; } @@ -238,7 +240,7 @@ public void testSingleQueueWithOneUser() throws Exception { // Release each container from app_0 for (RMContainer rmContainer : app_0.getLiveContainers()) { a.completedContainer(clusterResource, app_0, node_0, rmContainer, - RMContainerEventType.KILL); + null, RMContainerEventType.KILL); } assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); @@ -247,7 +249,7 @@ public void testSingleQueueWithOneUser() throws Exception { // Release each container from app_1 for (RMContainer rmContainer : app_1.getLiveContainers()) { a.completedContainer(clusterResource, app_1, node_0, rmContainer, - RMContainerEventType.KILL); + null, RMContainerEventType.KILL); } assertEquals(0*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); @@ -392,7 +394,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // 8. Release each container from app_0 for (RMContainer rmContainer : app_0.getLiveContainers()) { a.completedContainer(clusterResource, app_0, node_0, rmContainer, - RMContainerEventType.KILL); + null, RMContainerEventType.KILL); } assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); @@ -403,7 +405,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // 9. Release each container from app_2 for (RMContainer rmContainer : app_2.getLiveContainers()) { a.completedContainer(clusterResource, app_2, node_0, rmContainer, - RMContainerEventType.KILL); + null, RMContainerEventType.KILL); } assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); @@ -414,7 +416,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // 10. Release each container from app_3 for (RMContainer rmContainer : app_3.getLiveContainers()) { a.completedContainer(clusterResource, app_3, node_0, rmContainer, - RMContainerEventType.KILL); + null, RMContainerEventType.KILL); } assertEquals(0*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); @@ -489,7 +491,7 @@ public void testReservation() throws Exception { // Now free 1 container from app_0 i.e. 1G a.completedContainer(clusterResource, app_0, node_0, - app_0.getLiveContainers().iterator().next(), RMContainerEventType.KILL); + app_0.getLiveContainers().iterator().next(), null, RMContainerEventType.KILL); a.assignContainers(clusterResource, node_0); assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); @@ -499,7 +501,7 @@ public void testReservation() throws Exception { // Now finish another container from app_0 and fulfill the reservation a.completedContainer(clusterResource, app_0, node_0, - app_0.getLiveContainers().iterator().next(), RMContainerEventType.KILL); + app_0.getLiveContainers().iterator().next(), null, RMContainerEventType.KILL); a.assignContainers(clusterResource, node_0); assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); @@ -582,7 +584,7 @@ public void testReservationExchange() throws Exception { // Now free 1 container from app_0 i.e. 1G, and re-reserve it a.completedContainer(clusterResource, app_0, node_0, - app_0.getLiveContainers().iterator().next(), RMContainerEventType.KILL); + app_0.getLiveContainers().iterator().next(), null, RMContainerEventType.KILL); a.assignContainers(clusterResource, node_0); assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); @@ -613,7 +615,7 @@ public void testReservationExchange() throws Exception { // Now finish another container from app_0 and see the reservation cancelled a.completedContainer(clusterResource, app_0, node_0, - app_0.getLiveContainers().iterator().next(), RMContainerEventType.KILL); + app_0.getLiveContainers().iterator().next(), null, RMContainerEventType.KILL); a.assignContainers(clusterResource, node_0); assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java index ea635270e0..5a82afa1d5 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java @@ -81,7 +81,7 @@ private void setupSingleLevelQueues(CapacitySchedulerConfiguration conf) { LOG.info("Setup top-level queues a and b"); } - private void stubQueueAllocation(final Queue queue, + private void stubQueueAllocation(final CSQueue queue, final Resource clusterResource, final SchedulerNode node, final int allocation) { @@ -121,7 +121,7 @@ public Resource answer(InvocationOnMock invocation) throws Throwable { when(queue).assignContainers(eq(clusterResource), eq(node)); } - private float computeQueueUtilization(Queue queue, + private float computeQueueUtilization(CSQueue queue, int expectedMemory, Resource clusterResource) { return (expectedMemory / (clusterResource.getMemory() * queue.getAbsoluteCapacity())); @@ -132,8 +132,8 @@ public void testSingleLevelQueues() throws Exception { // Setup queue configs setupSingleLevelQueues(csConf); - Map queues = new HashMap(); - Queue root = + Map queues = new HashMap(); + CSQueue root = CapacityScheduler.parseQueue(csContext, csConf, null, CapacityScheduler.ROOT, queues, queues, CapacityScheduler.queueComparator, @@ -270,8 +270,8 @@ public void testMultiLevelQueues() throws Exception { // Setup queue configs setupMultiLevelQueues(csConf); - Map queues = new HashMap(); - Queue root = + Map queues = new HashMap(); + CSQueue root = CapacityScheduler.parseQueue(csContext, csConf, null, CapacityScheduler.ROOT, queues, queues, CapacityScheduler.queueComparator, @@ -294,17 +294,17 @@ public void testMultiLevelQueues() throws Exception { when(csContext.getNumClusterNodes()).thenReturn(numNodes); // Start testing - Queue a = queues.get(A); - Queue b = queues.get(B); - Queue c = queues.get(C); - Queue d = queues.get(D); + CSQueue a = queues.get(A); + CSQueue b = queues.get(B); + CSQueue c = queues.get(C); + CSQueue d = queues.get(D); - Queue a1 = queues.get(A1); - Queue a2 = queues.get(A2); + CSQueue a1 = queues.get(A1); + CSQueue a2 = queues.get(A2); - Queue b1 = queues.get(B1); - Queue b2 = queues.get(B2); - Queue b3 = queues.get(B3); + CSQueue b1 = queues.get(B1); + CSQueue b2 = queues.get(B2); + CSQueue b3 = queues.get(B3); final float delta = 0.0001f; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java index 0d59711578..84dbbac867 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java @@ -85,7 +85,7 @@ public EventHandler getEventHandler() { */ static class SpyHook extends CapacityScheduler.QueueHook { @Override - public Queue hook(Queue queue) { + public CSQueue hook(CSQueue queue) { return spy(queue); } } @@ -154,8 +154,8 @@ public static SchedulerNode getMockNode( public static ContainerId getMockContainerId(SchedulerApp application) { ContainerId containerId = mock(ContainerId.class); - doReturn(application.getApplicationAttemptId()).when(containerId).getAppAttemptId(); - doReturn(application.getApplicationId()).when(containerId).getAppId(); + doReturn(application.getApplicationAttemptId()). + when(containerId).getApplicationAttemptId(); doReturn(application.getNewContainerId()).when(containerId).getId(); return containerId; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java index 8ea9e80762..3dfa301e34 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java @@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.webapp.WebApps; import org.apache.hadoop.yarn.webapp.test.WebAppTests; import org.junit.Test; @@ -168,9 +169,38 @@ static void setupQueueConfiguration(CapacitySchedulerConfiguration conf) { conf.setCapacity(C13, 40); } + public static ResourceManager mockFifoRm(int apps, int racks, int nodes, + int mbsPerNode) + throws Exception { + ResourceManager rm = mock(ResourceManager.class); + RMContext rmContext = mockRMContext(apps, racks, nodes, + mbsPerNode); + ResourceScheduler rs = mockFifoScheduler(); + when(rm.getResourceScheduler()).thenReturn(rs); + when(rm.getRMContext()).thenReturn(rmContext); + return rm; + } + + public static FifoScheduler mockFifoScheduler() throws Exception { + CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); + setupFifoQueueConfiguration(conf); + + FifoScheduler fs = new FifoScheduler(); + fs.reinitialize(conf, null, null); + return fs; + } + + static void setupFifoQueueConfiguration(CapacitySchedulerConfiguration conf) { + // Define default queue + conf.setQueues("default", new String[] {"default"}); + conf.setCapacity("default", 100); + } + public static void main(String[] args) throws Exception { // For manual testing WebApps.$for("yarn", new TestRMWebApp()).at(8888).inDevMode(). start(new RMWebApp(mockRm(101, 8, 8, 8*GiB))).joinThread(); + WebApps.$for("yarn", new TestRMWebApp()).at(8888).inDevMode(). + start(new RMWebApp(mockFifoRm(10, 1, 4, 8*GiB))).joinThread(); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java index ef4f73a5c2..ead8675fe3 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.yarn.YarnException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.factories.RecordFactory; @@ -39,7 +40,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.nodemanager.Context; -import org.apache.hadoop.yarn.server.nodemanager.NMConfig; import org.apache.hadoop.yarn.server.nodemanager.NodeManager; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl; @@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store; import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.service.CompositeService; @@ -154,7 +155,7 @@ public synchronized void start() { new File(testWorkDir, MiniYARNCluster.this.getName() + "-localDir"); localDir.mkdir(); LOG.info("Created localDir in " + localDir.getAbsolutePath()); - getConfig().set(NMConfig.NM_LOCAL_DIR, localDir.getAbsolutePath()); + getConfig().set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath()); File logDir = new File(testWorkDir, MiniYARNCluster.this.getName() + "-logDir"); @@ -164,10 +165,10 @@ public synchronized void start() { logDir.mkdir(); remoteLogDir.mkdir(); LOG.info("Created logDir in " + logDir.getAbsolutePath()); - getConfig().set(NMConfig.NM_LOG_DIR, logDir.getAbsolutePath()); - getConfig().set(NMConfig.REMOTE_USER_LOG_DIR, + getConfig().set(YarnConfiguration.NM_LOG_DIRS, logDir.getAbsolutePath()); + getConfig().set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath()); - getConfig().setInt(NMConfig.NM_VMEM_GB, 4); // By default AM + 2 containers + getConfig().setInt(YarnConfiguration.NM_VMEM_GB, 4); // By default AM + 2 containers nodeManager = new NodeManager() { @Override @@ -177,9 +178,10 @@ protected void doSecureLogin() throws IOException { @Override protected NodeStatusUpdater createNodeStatusUpdater(Context context, - Dispatcher dispatcher, NodeHealthCheckerService healthChecker) { + Dispatcher dispatcher, NodeHealthCheckerService healthChecker, + ContainerTokenSecretManager containerTokenSecretManager) { return new NodeStatusUpdaterImpl(context, dispatcher, - healthChecker, metrics) { + healthChecker, metrics, containerTokenSecretManager) { @Override protected ResourceTracker getRMClient() { final ResourceTrackerService rt = resourceManager diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerTokenSecretManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerTokenSecretManager.java index d64983dcc4..3214898277 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerTokenSecretManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerTokenSecretManager.java @@ -82,6 +82,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.util.ConverterUtils; import org.junit.BeforeClass; +import org.junit.AfterClass; import org.junit.Test; public class TestContainerTokenSecretManager { @@ -94,6 +95,7 @@ public class TestContainerTokenSecretManager { private static final File localDir = new File("target", TestContainerTokenSecretManager.class.getName() + "-localDir") .getAbsoluteFile(); + private static MiniYARNCluster yarnCluster; @BeforeClass public static void setup() throws AccessControlException, @@ -103,6 +105,12 @@ public static void setup() throws AccessControlException, localDir.mkdir(); } + @AfterClass + public static void teardown() { + yarnCluster.stop(); + } + + @Test public void test() throws IOException, InterruptedException { @@ -114,9 +122,9 @@ public void test() throws IOException, InterruptedException { conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); // Set AM expiry interval to be very long. - conf.setLong(YarnConfiguration.AM_EXPIRY_INTERVAL, 100000L); + conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 100000L); UserGroupInformation.setConfiguration(conf); - MiniYARNCluster yarnCluster = + yarnCluster = new MiniYARNCluster(TestContainerTokenSecretManager.class.getName()); yarnCluster.init(conf); yarnCluster.start(); @@ -183,8 +191,8 @@ public void test() throws IOException, InterruptedException { // Ask for a container from the RM String schedulerAddressString = - conf.get(YarnConfiguration.SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_SCHEDULER_BIND_ADDRESS); + conf.get(YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS); final InetSocketAddress schedulerAddr = NetUtils.createSocketAddr(schedulerAddressString); ApplicationTokenIdentifier appTokenIdentifier = @@ -216,8 +224,6 @@ public AMRMProtocol run() { RegisterApplicationMasterRequest request = recordFactory .newRecordInstance(RegisterApplicationMasterRequest.class); - ApplicationMaster applicationMaster = recordFactory - .newRecordInstance(ApplicationMaster.class); request.setApplicationAttemptId(resourceManager.getRMContext() .getRMApps().get(appID).getCurrentAppAttempt().getAppAttemptId()); scheduler.registerApplicationMaster(request); @@ -241,7 +247,7 @@ public AMRMProtocol run() { allocateRequest.addAllAsks(ask); allocateRequest.addAllReleases(release); List allocatedContainers = scheduler.allocate(allocateRequest) - .getAMResponse().getNewContainerList(); + .getAMResponse().getAllocatedContainers(); waitCounter = 0; while ((allocatedContainers == null || allocatedContainers.size() == 0) @@ -251,7 +257,7 @@ public AMRMProtocol run() { allocateRequest.setResponseId(allocateRequest.getResponseId() + 1); allocatedContainers = scheduler.allocate(allocateRequest).getAMResponse() - .getNewContainerList(); + .getAllocatedContainers(); } Assert.assertNotNull("Container is not allocted!", allocatedContainers); @@ -285,12 +291,13 @@ public Void run() { .newRecordInstance(GetContainerStatusRequest.class); ContainerId containerID = recordFactory.newRecordInstance(ContainerId.class); - ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class); + ApplicationAttemptId appAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); appAttemptId.setApplicationId(appID); appAttemptId.setAttemptId(1); - containerID.setAppId(appID); + appAttemptId.setApplicationId(appID); + containerID.setApplicationAttemptId(appAttemptId); containerID.setId(1); - containerID.setAppAttemptId(appAttemptId); request.setContainerId(containerID); client.getContainerStatus(request); } catch (YarnRemoteException e) { @@ -339,9 +346,9 @@ public Void run() { ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class); appAttemptId.setApplicationId(appID); appAttemptId.setAttemptId(1); - containerID.setAppId(appID); + appAttemptId.setApplicationId(appID); + containerID.setApplicationAttemptId(appAttemptId); containerID.setId(1); - containerID.setAppAttemptId(appAttemptId); request.setContainerId(containerID); try { client.getContainerStatus(request); diff --git a/hadoop-mapreduce-project/src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/server/HadoopServer.java b/hadoop-mapreduce-project/src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/server/HadoopServer.java index 1454c42831..f7252d504b 100644 --- a/hadoop-mapreduce-project/src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/server/HadoopServer.java +++ b/hadoop-mapreduce-project/src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/server/HadoopServer.java @@ -36,6 +36,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.eclipse.Activator; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobID; @@ -420,8 +421,14 @@ public void setLocationName(String newName) { */ public void storeSettingsToFile(File file) throws IOException { FileOutputStream fos = new FileOutputStream(file); - this.conf.writeXml(fos); - fos.close(); + try { + this.conf.writeXml(fos); + fos.close(); + fos = null; + } finally { + IOUtils.closeStream(fos); + } + } /* @inheritDoc */ diff --git a/hadoop-mapreduce-project/src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/servers/RunOnHadoopWizard.java b/hadoop-mapreduce-project/src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/servers/RunOnHadoopWizard.java index 097507f357..2df29e9c16 100644 --- a/hadoop-mapreduce-project/src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/servers/RunOnHadoopWizard.java +++ b/hadoop-mapreduce-project/src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/servers/RunOnHadoopWizard.java @@ -28,6 +28,7 @@ import org.apache.hadoop.eclipse.ErrorMessageDialog; import org.apache.hadoop.eclipse.server.HadoopServer; import org.apache.hadoop.eclipse.server.JarModule; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.mapred.JobConf; import org.eclipse.core.resources.IFile; import org.eclipse.core.runtime.CoreException; @@ -164,8 +165,13 @@ public boolean performFinish() { // confDir); File confFile = new File(confDir, "core-site.xml"); FileOutputStream fos = new FileOutputStream(confFile); - conf.writeXml(fos); - fos.close(); + try { + conf.writeXml(fos); + fos.close(); + fos = null; + } finally { + IOUtils.closeStream(fos); + } } catch (IOException ioe) { ioe.printStackTrace(); diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ExecutionSummarizer.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ExecutionSummarizer.java index f59bf9e66c..fc362c5643 100644 --- a/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ExecutionSummarizer.java +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ExecutionSummarizer.java @@ -149,10 +149,15 @@ void finalize(JobFactory factory, String inputPath, long dataSize, throws IOException { numJobsInInputTrace = factory.numJobsInTrace; endTime = System.currentTimeMillis(); - Path inputTracePath = new Path(inputPath); - FileSystem fs = inputTracePath.getFileSystem(conf); - inputTraceLocation = fs.makeQualified(inputTracePath).toString(); - inputTraceSignature = getTraceSignature(inputTraceLocation); + if ("-".equals(inputPath)) { + inputTraceLocation = Summarizer.NA; + inputTraceSignature = Summarizer.NA; + } else { + Path inputTracePath = new Path(inputPath); + FileSystem fs = inputTracePath.getFileSystem(conf); + inputTraceLocation = fs.makeQualified(inputTracePath).toString(); + inputTraceSignature = getTraceSignature(inputPath); + } jobSubmissionPolicy = Gridmix.getJobSubmissionPolicy(conf).name(); resolver = userResolver.getClass().getName(); if (dataSize > 0) { diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java index 9b8f476487..fcc6939c4e 100644 --- a/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java @@ -314,9 +314,13 @@ public Integer run() throws Exception { } }); - // print the run summary - System.out.print("\n\n"); - System.out.println(summarizer.toString()); + // print the gridmix summary if the run was successful + if (val == 0) { + // print the run summary + System.out.print("\n\n"); + System.out.println(summarizer.toString()); + } + return val; } diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ResourceUsageMatcher.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ResourceUsageMatcher.java index 10d6e733f1..917cd09372 100644 --- a/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ResourceUsageMatcher.java +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ResourceUsageMatcher.java @@ -52,15 +52,23 @@ public class ResourceUsageMatcher { @SuppressWarnings("unchecked") public void configure(Configuration conf, ResourceCalculatorPlugin monitor, ResourceUsageMetrics metrics, Progressive progress) { - Class[] plugins = - conf.getClasses(RESOURCE_USAGE_EMULATION_PLUGINS, - ResourceUsageEmulatorPlugin.class); + Class[] plugins = conf.getClasses(RESOURCE_USAGE_EMULATION_PLUGINS); if (plugins == null) { System.out.println("No resource usage emulator plugins configured."); } else { - for (Class plugin : plugins) { - if (plugin != null) { - emulationPlugins.add(ReflectionUtils.newInstance(plugin, conf)); + for (Class clazz : plugins) { + if (clazz != null) { + if (ResourceUsageEmulatorPlugin.class.isAssignableFrom(clazz)) { + ResourceUsageEmulatorPlugin plugin = + (ResourceUsageEmulatorPlugin) ReflectionUtils.newInstance(clazz, + conf); + emulationPlugins.add(plugin); + } else { + throw new RuntimeException("Misconfigured resource usage plugins. " + + "Class " + clazz.getClass().getName() + " is not a resource " + + "usage plugin as it does not extend " + + ResourceUsageEmulatorPlugin.class.getName()); + } } } } diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java index 694ca2e7e3..3199c42efd 100644 --- a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java @@ -159,7 +159,7 @@ public void update(Object item) { @Override protected Thread createReaderThread() { - return null; + return new Thread(); } } @@ -243,7 +243,7 @@ public void testExecutionSummarizer() throws IOException { tid, es.getInputTraceSignature()); // test trace location Path qPath = fs.makeQualified(testTraceFile); - assertEquals("Mismatch in trace signature", + assertEquals("Mismatch in trace filename", qPath.toString(), es.getInputTraceLocation()); // test expected data size assertEquals("Mismatch in expected data size", @@ -275,7 +275,7 @@ public void testExecutionSummarizer() throws IOException { es.finalize(factory, testTraceFile.toString(), 0L, resolver, dataStats, conf); // test missing expected data size - assertEquals("Mismatch in trace signature", + assertEquals("Mismatch in trace data size", Summarizer.NA, es.getExpectedDataSize()); assertFalse("Mismatch in trace signature", tid.equals(es.getInputTraceSignature())); @@ -295,6 +295,12 @@ public void testExecutionSummarizer() throws IOException { assertEquals("Mismatch in trace signature", tid, es.getInputTraceSignature()); + // finalize trace identifier '-' input + es.finalize(factory, "-", 0L, resolver, dataStats, conf); + assertEquals("Mismatch in trace signature", + Summarizer.NA, es.getInputTraceSignature()); + assertEquals("Mismatch in trace file location", + Summarizer.NA, es.getInputTraceLocation()); } // test the ExecutionSummarizer diff --git a/hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyRaid.java b/hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyRaid.java index 07331ef8d8..dd9d012493 100644 --- a/hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyRaid.java +++ b/hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyRaid.java @@ -26,8 +26,6 @@ import java.util.Map; import java.util.Set; -import junit.framework.Assert; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -43,475 +41,479 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRaid.CachedFullPathNames; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRaid.CachedLocatedBlocks; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRaid.FileType; -import org.apache.hadoop.hdfs.server.namenode.*; +import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.namenode.INodeFile; +import org.apache.hadoop.hdfs.server.namenode.NameNodeRaidTestUtil; +import org.apache.hadoop.hdfs.server.namenode.NameNodeRaidUtil; +import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.raid.RaidNode; +import org.junit.Assert; import org.junit.Test; public class TestBlockPlacementPolicyRaid { - @Test - public void testFoo() { + private Configuration conf = null; + private MiniDFSCluster cluster = null; + private FSNamesystem namesystem = null; + private BlockManager blockManager; + private NetworkTopology networktopology; + private BlockPlacementPolicyRaid policy = null; + private FileSystem fs = null; + String[] rack1 = {"/rack1"}; + String[] rack2 = {"/rack2"}; + String[] host1 = {"host1.rack1.com"}; + String[] host2 = {"host2.rack2.com"}; + String xorPrefix = null; + String raidTempPrefix = null; + String raidrsTempPrefix = null; + String raidrsHarTempPrefix = null; + + final static Log LOG = + LogFactory.getLog(TestBlockPlacementPolicyRaid.class); + + protected void setupCluster() throws IOException { + conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L); + conf.set("dfs.replication.pending.timeout.sec", "2"); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1L); + conf.set("dfs.block.replicator.classname", + BlockPlacementPolicyRaid.class.getName()); + conf.set(RaidNode.STRIPE_LENGTH_KEY, "2"); + conf.set(RaidNode.RS_PARITY_LENGTH_KEY, "3"); + conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1); + // start the cluster with one datanode first + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1). + format(true).racks(rack1).hosts(host1).build(); + cluster.waitActive(); + namesystem = cluster.getNameNode().getNamesystem(); + blockManager = namesystem.getBlockManager(); + networktopology = blockManager.getDatanodeManager().getNetworkTopology(); + + Assert.assertTrue("BlockPlacementPolicy type is not correct.", + blockManager.getBlockPlacementPolicy() instanceof BlockPlacementPolicyRaid); + policy = (BlockPlacementPolicyRaid)blockManager.getBlockPlacementPolicy(); + fs = cluster.getFileSystem(); + xorPrefix = RaidNode.xorDestinationPath(conf).toUri().getPath(); + raidTempPrefix = RaidNode.xorTempPrefix(conf); + raidrsTempPrefix = RaidNode.rsTempPrefix(conf); + raidrsHarTempPrefix = RaidNode.rsHarTempPrefix(conf); + } + + /** + * Test that the parity files will be placed at the good locations when we + * create them. + */ + @Test + public void testChooseTargetForRaidFile() throws IOException { + setupCluster(); + try { + String src = "/dir/file"; + String parity = raidrsTempPrefix + src; + DFSTestUtil.createFile(fs, new Path(src), 4, (short)1, 0L); + DFSTestUtil.waitReplication(fs, new Path(src), (short)1); + refreshPolicy(); + setBlockPlacementPolicy(namesystem, policy); + // start 3 more datanodes + String[] racks = {"/rack2", "/rack2", "/rack2", + "/rack2", "/rack2", "/rack2"}; + String[] hosts = + {"host2.rack2.com", "host3.rack2.com", "host4.rack2.com", + "host5.rack2.com", "host6.rack2.com", "host7.rack2.com"}; + cluster.startDataNodes(conf, 6, true, null, racks, hosts, null); + int numBlocks = 6; + DFSTestUtil.createFile(fs, new Path(parity), numBlocks, (short)2, 0L); + DFSTestUtil.waitReplication(fs, new Path(parity), (short)2); + FileStatus srcStat = fs.getFileStatus(new Path(src)); + BlockLocation[] srcLoc = + fs.getFileBlockLocations(srcStat, 0, srcStat.getLen()); + FileStatus parityStat = fs.getFileStatus(new Path(parity)); + BlockLocation[] parityLoc = + fs.getFileBlockLocations(parityStat, 0, parityStat.getLen()); + int parityLen = RaidNode.rsParityLength(conf); + for (int i = 0; i < numBlocks / parityLen; i++) { + Set locations = new HashSet(); + for (int j = 0; j < srcLoc.length; j++) { + String [] names = srcLoc[j].getNames(); + for (int k = 0; k < names.length; k++) { + LOG.info("Source block location: " + names[k]); + locations.add(names[k]); + } + } + for (int j = 0 ; j < parityLen; j++) { + String[] names = parityLoc[j + i * parityLen].getNames(); + for (int k = 0; k < names.length; k++) { + LOG.info("Parity block location: " + names[k]); + Assert.assertTrue(locations.add(names[k])); + } + } + } + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + /** + * Test that the har parity files will be placed at the good locations when we + * create them. + */ + @Test + public void testChooseTargetForHarRaidFile() throws IOException { + setupCluster(); + try { + String[] racks = {"/rack2", "/rack2", "/rack2", + "/rack2", "/rack2", "/rack2"}; + String[] hosts = + {"host2.rack2.com", "host3.rack2.com", "host4.rack2.com", + "host5.rack2.com", "host6.rack2.com", "host7.rack2.com"}; + cluster.startDataNodes(conf, 6, true, null, racks, hosts, null); + String harParity = raidrsHarTempPrefix + "/dir/file"; + int numBlocks = 11; + DFSTestUtil.createFile(fs, new Path(harParity), numBlocks, (short)1, 0L); + DFSTestUtil.waitReplication(fs, new Path(harParity), (short)1); + FileStatus stat = fs.getFileStatus(new Path(harParity)); + BlockLocation[] loc = fs.getFileBlockLocations(stat, 0, stat.getLen()); + int rsParityLength = RaidNode.rsParityLength(conf); + for (int i = 0; i < numBlocks - rsParityLength; i++) { + Set locations = new HashSet(); + for (int j = 0; j < rsParityLength; j++) { + for (int k = 0; k < loc[i + j].getNames().length; k++) { + // verify that every adjacent 4 blocks are on differnt nodes + String name = loc[i + j].getNames()[k]; + LOG.info("Har Raid block location: " + name); + Assert.assertTrue(locations.add(name)); + } + } + } + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + /** + * Test BlockPlacementPolicyRaid.CachedLocatedBlocks + * Verify that the results obtained from cache is the same as + * the results obtained directly + */ + @Test + public void testCachedBlocks() throws IOException { + setupCluster(); + try { + String file1 = "/dir/file1"; + String file2 = "/dir/file2"; + DFSTestUtil.createFile(fs, new Path(file1), 3, (short)1, 0L); + DFSTestUtil.createFile(fs, new Path(file2), 4, (short)1, 0L); + // test blocks cache + CachedLocatedBlocks cachedBlocks = new CachedLocatedBlocks(namesystem); + verifyCachedBlocksResult(cachedBlocks, namesystem, file1); + verifyCachedBlocksResult(cachedBlocks, namesystem, file1); + verifyCachedBlocksResult(cachedBlocks, namesystem, file2); + verifyCachedBlocksResult(cachedBlocks, namesystem, file2); + try { + Thread.sleep(1200L); + } catch (InterruptedException e) { + } + verifyCachedBlocksResult(cachedBlocks, namesystem, file2); + verifyCachedBlocksResult(cachedBlocks, namesystem, file1); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + /** + * Test BlockPlacementPolicyRaid.CachedFullPathNames + * Verify that the results obtained from cache is the same as + * the results obtained directly + */ + @Test + public void testCachedPathNames() throws IOException { + setupCluster(); + try { + String file1 = "/dir/file1"; + String file2 = "/dir/file2"; + DFSTestUtil.createFile(fs, new Path(file1), 3, (short)1, 0L); + DFSTestUtil.createFile(fs, new Path(file2), 4, (short)1, 0L); + // test full path cache + CachedFullPathNames cachedFullPathNames = + new CachedFullPathNames(namesystem); + final FSInodeInfo[] inodes = NameNodeRaidTestUtil.getFSInodeInfo( + namesystem, file1, file2); + + verifyCachedFullPathNameResult(cachedFullPathNames, inodes[0]); + verifyCachedFullPathNameResult(cachedFullPathNames, inodes[0]); + verifyCachedFullPathNameResult(cachedFullPathNames, inodes[1]); + verifyCachedFullPathNameResult(cachedFullPathNames, inodes[1]); + try { + Thread.sleep(1200L); + } catch (InterruptedException e) { + } + verifyCachedFullPathNameResult(cachedFullPathNames, inodes[1]); + verifyCachedFullPathNameResult(cachedFullPathNames, inodes[0]); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + /** + * Test the result of getCompanionBlocks() on the unraided files + */ + @Test + public void testGetCompanionBLocks() throws IOException { + setupCluster(); + try { + String file1 = "/dir/file1"; + String file2 = "/raid/dir/file2"; + String file3 = "/raidrs/dir/file3"; + // Set the policy to default policy to place the block in the default way + setBlockPlacementPolicy(namesystem, new BlockPlacementPolicyDefault( + conf, namesystem, networktopology)); + DFSTestUtil.createFile(fs, new Path(file1), 3, (short)1, 0L); + DFSTestUtil.createFile(fs, new Path(file2), 4, (short)1, 0L); + DFSTestUtil.createFile(fs, new Path(file3), 8, (short)1, 0L); + Collection companionBlocks; + + companionBlocks = getCompanionBlocks( + namesystem, policy, getBlocks(namesystem, file1).get(0).getBlock()); + Assert.assertTrue(companionBlocks == null || companionBlocks.size() == 0); + + companionBlocks = getCompanionBlocks( + namesystem, policy, getBlocks(namesystem, file1).get(2).getBlock()); + Assert.assertTrue(companionBlocks == null || companionBlocks.size() == 0); + + companionBlocks = getCompanionBlocks( + namesystem, policy, getBlocks(namesystem, file2).get(0).getBlock()); + Assert.assertEquals(1, companionBlocks.size()); + + companionBlocks = getCompanionBlocks( + namesystem, policy, getBlocks(namesystem, file2).get(3).getBlock()); + Assert.assertEquals(1, companionBlocks.size()); + + int rsParityLength = RaidNode.rsParityLength(conf); + companionBlocks = getCompanionBlocks( + namesystem, policy, getBlocks(namesystem, file3).get(0).getBlock()); + Assert.assertEquals(rsParityLength, companionBlocks.size()); + + companionBlocks = getCompanionBlocks( + namesystem, policy, getBlocks(namesystem, file3).get(4).getBlock()); + Assert.assertEquals(rsParityLength, companionBlocks.size()); + + companionBlocks = getCompanionBlocks( + namesystem, policy, getBlocks(namesystem, file3).get(6).getBlock()); + Assert.assertEquals(2, companionBlocks.size()); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + static void setBlockPlacementPolicy( + FSNamesystem namesystem, BlockPlacementPolicy policy) { + namesystem.writeLock(); + try { + namesystem.getBlockManager().setBlockPlacementPolicy(policy); + } finally { + namesystem.writeUnlock(); + } + } + + /** + * Test BlockPlacementPolicyRaid actually deletes the correct replica. + * Start 2 datanodes and create 1 source file and its parity file. + * 1) Start host1, create the parity file with replication 1 + * 2) Start host2, create the source file with replication 2 + * 3) Set repliation of source file to 1 + * Verify that the policy should delete the block with more companion blocks. + */ + @Test + public void testDeleteReplica() throws IOException { + setupCluster(); + try { + // Set the policy to default policy to place the block in the default way + setBlockPlacementPolicy(namesystem, new BlockPlacementPolicyDefault( + conf, namesystem, networktopology)); + DatanodeDescriptor datanode1 = blockManager.getDatanodeManager( + ).getDatanodeCyclicIteration("").iterator().next().getValue(); + String source = "/dir/file"; + String parity = xorPrefix + source; + + final Path parityPath = new Path(parity); + DFSTestUtil.createFile(fs, parityPath, 3, (short)1, 0L); + DFSTestUtil.waitReplication(fs, parityPath, (short)1); + + // start one more datanode + cluster.startDataNodes(conf, 1, true, null, rack2, host2, null); + DatanodeDescriptor datanode2 = null; + for(Map.Entry e : blockManager.getDatanodeManager( + ).getDatanodeCyclicIteration("")) { + final DatanodeDescriptor d = e.getValue(); + if (!d.getName().equals(datanode1.getName())) { + datanode2 = d; + } + } + Assert.assertTrue(datanode2 != null); + cluster.waitActive(); + final Path sourcePath = new Path(source); + DFSTestUtil.createFile(fs, sourcePath, 5, (short)2, 0L); + DFSTestUtil.waitReplication(fs, sourcePath, (short)2); + + refreshPolicy(); + Assert.assertEquals(parity, + policy.getParityFile(source)); + Assert.assertEquals(source, + policy.getSourceFile(parity, xorPrefix)); + + List sourceBlocks = getBlocks(namesystem, source); + List parityBlocks = getBlocks(namesystem, parity); + Assert.assertEquals(5, sourceBlocks.size()); + Assert.assertEquals(3, parityBlocks.size()); + + // verify the result of getCompanionBlocks() + Collection companionBlocks; + companionBlocks = getCompanionBlocks( + namesystem, policy, sourceBlocks.get(0).getBlock()); + verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, + new int[]{0, 1}, new int[]{0}); + + companionBlocks = getCompanionBlocks( + namesystem, policy, sourceBlocks.get(1).getBlock()); + verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, + new int[]{0, 1}, new int[]{0}); + + companionBlocks = getCompanionBlocks( + namesystem, policy, sourceBlocks.get(2).getBlock()); + verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, + new int[]{2, 3}, new int[]{1}); + + companionBlocks = getCompanionBlocks( + namesystem, policy, sourceBlocks.get(3).getBlock()); + verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, + new int[]{2, 3}, new int[]{1}); + + companionBlocks = getCompanionBlocks( + namesystem, policy, sourceBlocks.get(4).getBlock()); + verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, + new int[]{4}, new int[]{2}); + + companionBlocks = getCompanionBlocks( + namesystem, policy, parityBlocks.get(0).getBlock()); + verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, + new int[]{0, 1}, new int[]{0}); + + companionBlocks = getCompanionBlocks( + namesystem, policy, parityBlocks.get(1).getBlock()); + verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, + new int[]{2, 3}, new int[]{1}); + + companionBlocks = getCompanionBlocks( + namesystem, policy, parityBlocks.get(2).getBlock()); + verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, + new int[]{4}, new int[]{2}); + + // Set the policy back to raid policy. We have to create a new object + // here to clear the block location cache + refreshPolicy(); + setBlockPlacementPolicy(namesystem, policy); + // verify policy deletes the correct blocks. companion blocks should be + // evenly distributed. + fs.setReplication(sourcePath, (short)1); + DFSTestUtil.waitReplication(fs, sourcePath, (short)1); + Map counters = new HashMap(); + refreshPolicy(); + for (int i = 0; i < parityBlocks.size(); i++) { + companionBlocks = getCompanionBlocks( + namesystem, policy, parityBlocks.get(i).getBlock()); + + counters = BlockPlacementPolicyRaid.countCompanionBlocks( + companionBlocks, false); + Assert.assertTrue(counters.get(datanode1.getName()) >= 1 && + counters.get(datanode1.getName()) <= 2); + Assert.assertTrue(counters.get(datanode1.getName()) + + counters.get(datanode2.getName()) == + companionBlocks.size()); + + counters = BlockPlacementPolicyRaid.countCompanionBlocks( + companionBlocks, true); + Assert.assertTrue(counters.get(datanode1.getParent().getName()) >= 1 && + counters.get(datanode1.getParent().getName()) <= 2); + Assert.assertTrue(counters.get(datanode1.getParent().getName()) + + counters.get(datanode2.getParent().getName()) == + companionBlocks.size()); + } + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + // create a new BlockPlacementPolicyRaid to clear the cache + private void refreshPolicy() { + policy = new BlockPlacementPolicyRaid(); + policy.initialize(conf, namesystem, networktopology); + } + + private void verifyCompanionBlocks(Collection companionBlocks, + List sourceBlocks, List parityBlocks, + int[] sourceBlockIndexes, int[] parityBlockIndexes) { + Set blockSet = new HashSet(); + for (LocatedBlock b : companionBlocks) { + blockSet.add(b.getBlock()); + } + Assert.assertEquals(sourceBlockIndexes.length + parityBlockIndexes.length, + blockSet.size()); + for (int index : sourceBlockIndexes) { + Assert.assertTrue(blockSet.contains(sourceBlocks.get(index).getBlock())); + } + for (int index : parityBlockIndexes) { + Assert.assertTrue(blockSet.contains(parityBlocks.get(index).getBlock())); + } + } + + private void verifyCachedFullPathNameResult( + CachedFullPathNames cachedFullPathNames, FSInodeInfo inode) + throws IOException { + String res1 = inode.getFullPathName(); + String res2 = cachedFullPathNames.get(inode); + LOG.info("Actual path name: " + res1); + LOG.info("Cached path name: " + res2); + Assert.assertEquals(cachedFullPathNames.get(inode), + inode.getFullPathName()); + } + + private void verifyCachedBlocksResult(CachedLocatedBlocks cachedBlocks, + FSNamesystem namesystem, String file) throws IOException{ + long len = NameNodeRaidUtil.getFileInfo(namesystem, file, true).getLen(); + List res1 = NameNodeRaidUtil.getBlockLocations(namesystem, + file, 0L, len, false, false).getLocatedBlocks(); + List res2 = cachedBlocks.get(file); + for (int i = 0; i < res1.size(); i++) { + LOG.info("Actual block: " + res1.get(i).getBlock()); + LOG.info("Cached block: " + res2.get(i).getBlock()); + Assert.assertEquals(res1.get(i).getBlock(), res2.get(i).getBlock()); + } + } + + private Collection getCompanionBlocks( + FSNamesystem namesystem, BlockPlacementPolicyRaid policy, + ExtendedBlock block) throws IOException { + INodeFile inode = blockManager.blocksMap.getINode(block + .getLocalBlock()); + FileType type = policy.getFileType(inode.getFullPathName()); + return policy.getCompanionBlocks(inode.getFullPathName(), type, + block.getLocalBlock()); + } + + private List getBlocks(FSNamesystem namesystem, String file) + throws IOException { + long len = NameNodeRaidUtil.getFileInfo(namesystem, file, true).getLen(); + return NameNodeRaidUtil.getBlockLocations(namesystem, + file, 0, len, false, false).getLocatedBlocks(); } -// private Configuration conf = null; -// private MiniDFSCluster cluster = null; -// private FSNamesystem namesystem = null; -// private BlockPlacementPolicyRaid policy = null; -// private FileSystem fs = null; -// String[] rack1 = {"/rack1"}; -// String[] rack2 = {"/rack2"}; -// String[] host1 = {"host1.rack1.com"}; -// String[] host2 = {"host2.rack2.com"}; -// String xorPrefix = null; -// String raidTempPrefix = null; -// String raidrsTempPrefix = null; -// String raidrsHarTempPrefix = null; -// -// final static Log LOG = -// LogFactory.getLog(TestBlockPlacementPolicyRaid.class); -// -// protected void setupCluster() throws IOException { -// conf = new Configuration(); -// conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L); -// conf.set("dfs.replication.pending.timeout.sec", "2"); -// conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1L); -// conf.set("dfs.block.replicator.classname", -// "org.apache.hadoop.hdfs.server.namenode.BlockPlacementPolicyRaid"); -// conf.set(RaidNode.STRIPE_LENGTH_KEY, "2"); -// conf.set(RaidNode.RS_PARITY_LENGTH_KEY, "3"); -// conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1); -// // start the cluster with one datanode first -// cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1). -// format(true).racks(rack1).hosts(host1).build(); -// cluster.waitActive(); -// namesystem = cluster.getNameNode().getNamesystem(); -// Assert.assertTrue("BlockPlacementPolicy type is not correct.", -// namesystem.blockManager.replicator instanceof BlockPlacementPolicyRaid); -// policy = (BlockPlacementPolicyRaid) namesystem.blockManager.replicator; -// fs = cluster.getFileSystem(); -// xorPrefix = RaidNode.xorDestinationPath(conf).toUri().getPath(); -// raidTempPrefix = RaidNode.xorTempPrefix(conf); -// raidrsTempPrefix = RaidNode.rsTempPrefix(conf); -// raidrsHarTempPrefix = RaidNode.rsHarTempPrefix(conf); -// } -// -// /** -// * Test that the parity files will be placed at the good locations when we -// * create them. -// */ -// @Test -// public void testChooseTargetForRaidFile() throws IOException { -// setupCluster(); -// try { -// String src = "/dir/file"; -// String parity = raidrsTempPrefix + src; -// DFSTestUtil.createFile(fs, new Path(src), 4, (short)1, 0L); -// DFSTestUtil.waitReplication(fs, new Path(src), (short)1); -// refreshPolicy(); -// setBlockPlacementPolicy(namesystem, policy); -// // start 3 more datanodes -// String[] racks = {"/rack2", "/rack2", "/rack2", -// "/rack2", "/rack2", "/rack2"}; -// String[] hosts = -// {"host2.rack2.com", "host3.rack2.com", "host4.rack2.com", -// "host5.rack2.com", "host6.rack2.com", "host7.rack2.com"}; -// cluster.startDataNodes(conf, 6, true, null, racks, hosts, null); -// int numBlocks = 6; -// DFSTestUtil.createFile(fs, new Path(parity), numBlocks, (short)2, 0L); -// DFSTestUtil.waitReplication(fs, new Path(parity), (short)2); -// FileStatus srcStat = fs.getFileStatus(new Path(src)); -// BlockLocation[] srcLoc = -// fs.getFileBlockLocations(srcStat, 0, srcStat.getLen()); -// FileStatus parityStat = fs.getFileStatus(new Path(parity)); -// BlockLocation[] parityLoc = -// fs.getFileBlockLocations(parityStat, 0, parityStat.getLen()); -// int parityLen = RaidNode.rsParityLength(conf); -// for (int i = 0; i < numBlocks / parityLen; i++) { -// Set locations = new HashSet(); -// for (int j = 0; j < srcLoc.length; j++) { -// String [] names = srcLoc[j].getNames(); -// for (int k = 0; k < names.length; k++) { -// LOG.info("Source block location: " + names[k]); -// locations.add(names[k]); -// } -// } -// for (int j = 0 ; j < parityLen; j++) { -// String[] names = parityLoc[j + i * parityLen].getNames(); -// for (int k = 0; k < names.length; k++) { -// LOG.info("Parity block location: " + names[k]); -// Assert.assertTrue(locations.add(names[k])); -// } -// } -// } -// } finally { -// if (cluster != null) { -// cluster.shutdown(); -// } -// } -// } -// -// /** -// * Test that the har parity files will be placed at the good locations when we -// * create them. -// */ -// @Test -// public void testChooseTargetForHarRaidFile() throws IOException { -// setupCluster(); -// try { -// String[] racks = {"/rack2", "/rack2", "/rack2", -// "/rack2", "/rack2", "/rack2"}; -// String[] hosts = -// {"host2.rack2.com", "host3.rack2.com", "host4.rack2.com", -// "host5.rack2.com", "host6.rack2.com", "host7.rack2.com"}; -// cluster.startDataNodes(conf, 6, true, null, racks, hosts, null); -// String harParity = raidrsHarTempPrefix + "/dir/file"; -// int numBlocks = 11; -// DFSTestUtil.createFile(fs, new Path(harParity), numBlocks, (short)1, 0L); -// DFSTestUtil.waitReplication(fs, new Path(harParity), (short)1); -// FileStatus stat = fs.getFileStatus(new Path(harParity)); -// BlockLocation[] loc = fs.getFileBlockLocations(stat, 0, stat.getLen()); -// int rsParityLength = RaidNode.rsParityLength(conf); -// for (int i = 0; i < numBlocks - rsParityLength; i++) { -// Set locations = new HashSet(); -// for (int j = 0; j < rsParityLength; j++) { -// for (int k = 0; k < loc[i + j].getNames().length; k++) { -// // verify that every adjacent 4 blocks are on differnt nodes -// String name = loc[i + j].getNames()[k]; -// LOG.info("Har Raid block location: " + name); -// Assert.assertTrue(locations.add(name)); -// } -// } -// } -// } finally { -// if (cluster != null) { -// cluster.shutdown(); -// } -// } -// } -// -// /** -// * Test BlockPlacementPolicyRaid.CachedLocatedBlocks -// * Verify that the results obtained from cache is the same as -// * the results obtained directly -// */ -// @Test -// public void testCachedBlocks() throws IOException { -// setupCluster(); -// try { -// String file1 = "/dir/file1"; -// String file2 = "/dir/file2"; -// DFSTestUtil.createFile(fs, new Path(file1), 3, (short)1, 0L); -// DFSTestUtil.createFile(fs, new Path(file2), 4, (short)1, 0L); -// // test blocks cache -// CachedLocatedBlocks cachedBlocks = new CachedLocatedBlocks(namesystem); -// verifyCachedBlocksResult(cachedBlocks, namesystem, file1); -// verifyCachedBlocksResult(cachedBlocks, namesystem, file1); -// verifyCachedBlocksResult(cachedBlocks, namesystem, file2); -// verifyCachedBlocksResult(cachedBlocks, namesystem, file2); -// try { -// Thread.sleep(1200L); -// } catch (InterruptedException e) { -// } -// verifyCachedBlocksResult(cachedBlocks, namesystem, file2); -// verifyCachedBlocksResult(cachedBlocks, namesystem, file1); -// } finally { -// if (cluster != null) { -// cluster.shutdown(); -// } -// } -// } -// -// /** -// * Test BlockPlacementPolicyRaid.CachedFullPathNames -// * Verify that the results obtained from cache is the same as -// * the results obtained directly -// */ -// @Test -// public void testCachedPathNames() throws IOException { -// setupCluster(); -// try { -// String file1 = "/dir/file1"; -// String file2 = "/dir/file2"; -// DFSTestUtil.createFile(fs, new Path(file1), 3, (short)1, 0L); -// DFSTestUtil.createFile(fs, new Path(file2), 4, (short)1, 0L); -// // test full path cache -// CachedFullPathNames cachedFullPathNames = -// new CachedFullPathNames(namesystem); -// FSInodeInfo inode1 = null; -// FSInodeInfo inode2 = null; -// NameNodeRaidTestUtil.readLock(namesystem.dir); -// try { -// inode1 = NameNodeRaidTestUtil.getNode(namesystem.dir, file1, true); -// inode2 = NameNodeRaidTestUtil.getNode(namesystem.dir, file2, true); -// } finally { -// NameNodeRaidTestUtil.readUnLock(namesystem.dir); -// } -// verifyCachedFullPathNameResult(cachedFullPathNames, inode1); -// verifyCachedFullPathNameResult(cachedFullPathNames, inode1); -// verifyCachedFullPathNameResult(cachedFullPathNames, inode2); -// verifyCachedFullPathNameResult(cachedFullPathNames, inode2); -// try { -// Thread.sleep(1200L); -// } catch (InterruptedException e) { -// } -// verifyCachedFullPathNameResult(cachedFullPathNames, inode2); -// verifyCachedFullPathNameResult(cachedFullPathNames, inode1); -// } finally { -// if (cluster != null) { -// cluster.shutdown(); -// } -// } -// } -// /** -// * Test the result of getCompanionBlocks() on the unraided files -// */ -// @Test -// public void testGetCompanionBLocks() throws IOException { -// setupCluster(); -// try { -// String file1 = "/dir/file1"; -// String file2 = "/raid/dir/file2"; -// String file3 = "/raidrs/dir/file3"; -// // Set the policy to default policy to place the block in the default way -// setBlockPlacementPolicy(namesystem, new BlockPlacementPolicyDefault( -// conf, namesystem, namesystem.clusterMap)); -// DFSTestUtil.createFile(fs, new Path(file1), 3, (short)1, 0L); -// DFSTestUtil.createFile(fs, new Path(file2), 4, (short)1, 0L); -// DFSTestUtil.createFile(fs, new Path(file3), 8, (short)1, 0L); -// Collection companionBlocks; -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, getBlocks(namesystem, file1).get(0).getBlock()); -// Assert.assertTrue(companionBlocks == null || companionBlocks.size() == 0); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, getBlocks(namesystem, file1).get(2).getBlock()); -// Assert.assertTrue(companionBlocks == null || companionBlocks.size() == 0); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, getBlocks(namesystem, file2).get(0).getBlock()); -// Assert.assertEquals(1, companionBlocks.size()); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, getBlocks(namesystem, file2).get(3).getBlock()); -// Assert.assertEquals(1, companionBlocks.size()); -// -// int rsParityLength = RaidNode.rsParityLength(conf); -// companionBlocks = getCompanionBlocks( -// namesystem, policy, getBlocks(namesystem, file3).get(0).getBlock()); -// Assert.assertEquals(rsParityLength, companionBlocks.size()); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, getBlocks(namesystem, file3).get(4).getBlock()); -// Assert.assertEquals(rsParityLength, companionBlocks.size()); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, getBlocks(namesystem, file3).get(6).getBlock()); -// Assert.assertEquals(2, companionBlocks.size()); -// } finally { -// if (cluster != null) { -// cluster.shutdown(); -// } -// } -// } -// -// static void setBlockPlacementPolicy( -// FSNamesystem namesystem, BlockPlacementPolicy policy) { -// namesystem.writeLock(); -// try { -// namesystem.blockManager.replicator = policy; -// } finally { -// namesystem.writeUnlock(); -// } -// } -// -// /** -// * Test BlockPlacementPolicyRaid actually deletes the correct replica. -// * Start 2 datanodes and create 1 source file and its parity file. -// * 1) Start host1, create the parity file with replication 1 -// * 2) Start host2, create the source file with replication 2 -// * 3) Set repliation of source file to 1 -// * Verify that the policy should delete the block with more companion blocks. -// */ -// @Test -// public void testDeleteReplica() throws IOException { -// setupCluster(); -// try { -// // Set the policy to default policy to place the block in the default way -// setBlockPlacementPolicy(namesystem, new BlockPlacementPolicyDefault( -// conf, namesystem, namesystem.clusterMap)); -// DatanodeDescriptor datanode1 = -// NameNodeRaidTestUtil.getDatanodeMap(namesystem).values().iterator().next(); -// String source = "/dir/file"; -// String parity = xorPrefix + source; -// -// final Path parityPath = new Path(parity); -// DFSTestUtil.createFile(fs, parityPath, 3, (short)1, 0L); -// DFSTestUtil.waitReplication(fs, parityPath, (short)1); -// -// // start one more datanode -// cluster.startDataNodes(conf, 1, true, null, rack2, host2, null); -// DatanodeDescriptor datanode2 = null; -// for (DatanodeDescriptor d : NameNodeRaidTestUtil.getDatanodeMap(namesystem).values()) { -// if (!d.getName().equals(datanode1.getName())) { -// datanode2 = d; -// } -// } -// Assert.assertTrue(datanode2 != null); -// cluster.waitActive(); -// final Path sourcePath = new Path(source); -// DFSTestUtil.createFile(fs, sourcePath, 5, (short)2, 0L); -// DFSTestUtil.waitReplication(fs, sourcePath, (short)2); -// -// refreshPolicy(); -// Assert.assertEquals(parity, -// policy.getParityFile(source)); -// Assert.assertEquals(source, -// policy.getSourceFile(parity, xorPrefix)); -// -// List sourceBlocks = getBlocks(namesystem, source); -// List parityBlocks = getBlocks(namesystem, parity); -// Assert.assertEquals(5, sourceBlocks.size()); -// Assert.assertEquals(3, parityBlocks.size()); -// -// // verify the result of getCompanionBlocks() -// Collection companionBlocks; -// companionBlocks = getCompanionBlocks( -// namesystem, policy, sourceBlocks.get(0).getBlock()); -// verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, -// new int[]{0, 1}, new int[]{0}); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, sourceBlocks.get(1).getBlock()); -// verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, -// new int[]{0, 1}, new int[]{0}); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, sourceBlocks.get(2).getBlock()); -// verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, -// new int[]{2, 3}, new int[]{1}); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, sourceBlocks.get(3).getBlock()); -// verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, -// new int[]{2, 3}, new int[]{1}); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, sourceBlocks.get(4).getBlock()); -// verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, -// new int[]{4}, new int[]{2}); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, parityBlocks.get(0).getBlock()); -// verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, -// new int[]{0, 1}, new int[]{0}); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, parityBlocks.get(1).getBlock()); -// verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, -// new int[]{2, 3}, new int[]{1}); -// -// companionBlocks = getCompanionBlocks( -// namesystem, policy, parityBlocks.get(2).getBlock()); -// verifyCompanionBlocks(companionBlocks, sourceBlocks, parityBlocks, -// new int[]{4}, new int[]{2}); -// -// // Set the policy back to raid policy. We have to create a new object -// // here to clear the block location cache -// refreshPolicy(); -// setBlockPlacementPolicy(namesystem, policy); -// // verify policy deletes the correct blocks. companion blocks should be -// // evenly distributed. -// fs.setReplication(sourcePath, (short)1); -// DFSTestUtil.waitReplication(fs, sourcePath, (short)1); -// Map counters = new HashMap(); -// refreshPolicy(); -// for (int i = 0; i < parityBlocks.size(); i++) { -// companionBlocks = getCompanionBlocks( -// namesystem, policy, parityBlocks.get(i).getBlock()); -// -// counters = BlockPlacementPolicyRaid.countCompanionBlocks( -// companionBlocks, false); -// Assert.assertTrue(counters.get(datanode1.getName()) >= 1 && -// counters.get(datanode1.getName()) <= 2); -// Assert.assertTrue(counters.get(datanode1.getName()) + -// counters.get(datanode2.getName()) == -// companionBlocks.size()); -// -// counters = BlockPlacementPolicyRaid.countCompanionBlocks( -// companionBlocks, true); -// Assert.assertTrue(counters.get(datanode1.getParent().getName()) >= 1 && -// counters.get(datanode1.getParent().getName()) <= 2); -// Assert.assertTrue(counters.get(datanode1.getParent().getName()) + -// counters.get(datanode2.getParent().getName()) == -// companionBlocks.size()); -// } -// } finally { -// if (cluster != null) { -// cluster.shutdown(); -// } -// } -// } -// -// // create a new BlockPlacementPolicyRaid to clear the cache -// private void refreshPolicy() { -// policy = new BlockPlacementPolicyRaid(); -// policy.initialize(conf, namesystem, namesystem.clusterMap); -// } -// -// private void verifyCompanionBlocks(Collection companionBlocks, -// List sourceBlocks, List parityBlocks, -// int[] sourceBlockIndexes, int[] parityBlockIndexes) { -// Set blockSet = new HashSet(); -// for (LocatedBlock b : companionBlocks) { -// blockSet.add(b.getBlock()); -// } -// Assert.assertEquals(sourceBlockIndexes.length + parityBlockIndexes.length, -// blockSet.size()); -// for (int index : sourceBlockIndexes) { -// Assert.assertTrue(blockSet.contains(sourceBlocks.get(index).getBlock())); -// } -// for (int index : parityBlockIndexes) { -// Assert.assertTrue(blockSet.contains(parityBlocks.get(index).getBlock())); -// } -// } -// -// private void verifyCachedFullPathNameResult( -// CachedFullPathNames cachedFullPathNames, FSInodeInfo inode) -// throws IOException { -// String res1 = inode.getFullPathName(); -// String res2 = cachedFullPathNames.get(inode); -// LOG.info("Actual path name: " + res1); -// LOG.info("Cached path name: " + res2); -// Assert.assertEquals(cachedFullPathNames.get(inode), -// inode.getFullPathName()); -// } -// -// private void verifyCachedBlocksResult(CachedLocatedBlocks cachedBlocks, -// FSNamesystem namesystem, String file) throws IOException{ -// long len = NameNodeRaidUtil.getFileInfo(namesystem, file, true).getLen(); -// List res1 = NameNodeRaidUtil.getBlockLocations(namesystem, -// file, 0L, len, false, false).getLocatedBlocks(); -// List res2 = cachedBlocks.get(file); -// for (int i = 0; i < res1.size(); i++) { -// LOG.info("Actual block: " + res1.get(i).getBlock()); -// LOG.info("Cached block: " + res2.get(i).getBlock()); -// Assert.assertEquals(res1.get(i).getBlock(), res2.get(i).getBlock()); -// } -// } -// -// private Collection getCompanionBlocks( -// FSNamesystem namesystem, BlockPlacementPolicyRaid policy, -// ExtendedBlock block) throws IOException { -// INodeFile inode = namesystem.blockManager.blocksMap.getINode(block -// .getLocalBlock()); -// FileType type = policy.getFileType(inode.getFullPathName()); -// return policy.getCompanionBlocks(inode.getFullPathName(), type, -// block.getLocalBlock()); -// } -// -// private List getBlocks(FSNamesystem namesystem, String file) -// throws IOException { -// long len = NameNodeRaidUtil.getFileInfo(namesystem, file, true).getLen(); -// return NameNodeRaidUtil.getBlockLocations(namesystem, -// file, 0, len, false, false).getLocatedBlocks(); -// } } diff --git a/hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/namenode/NameNodeRaidTestUtil.java b/hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/namenode/NameNodeRaidTestUtil.java index 0cb2ac7544..41960f8a17 100644 --- a/hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/namenode/NameNodeRaidTestUtil.java +++ b/hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/namenode/NameNodeRaidTestUtil.java @@ -17,33 +17,21 @@ */ package org.apache.hadoop.hdfs.server.namenode; -import java.io.*; -import java.util.*; - -import org.apache.hadoop.classification.*; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.hdfs.protocol.*; -import org.apache.hadoop.hdfs.server.blockmanagement.*; -import org.apache.hadoop.security.AccessControlException; +import org.apache.hadoop.fs.UnresolvedLinkException; public class NameNodeRaidTestUtil { - public static void readLock(final FSDirectory dir) { + public static FSInodeInfo[] getFSInodeInfo(final FSNamesystem namesystem, + final String... files) throws UnresolvedLinkException { + final FSInodeInfo[] inodes = new FSInodeInfo[files.length]; + final FSDirectory dir = namesystem.dir; dir.readLock(); + try { + for(int i = 0; i < files.length; i++) { + inodes[i] = dir.rootDir.getNode(files[i], true); + } + return inodes; + } finally { + dir.readUnlock(); + } } - - public static void readUnLock(final FSDirectory dir) { - dir.readUnlock(); - } - - public static FSInodeInfo getNode(final FSDirectory dir, - final String src, final boolean resolveLink - ) throws UnresolvedLinkException { - return dir.rootDir.getNode(src, resolveLink); - } - -// public static NavigableMap getDatanodeMap( -// final FSNamesystem namesystem) { -// return namesystem.datanodeMap; -// } } - diff --git a/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/Environment.java b/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/Environment.java index f8b45f3d06..bd76c31998 100644 --- a/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/Environment.java +++ b/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/Environment.java @@ -23,6 +23,7 @@ import java.util.*; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.io.IOUtils; /** * This is a class used to get the current environment @@ -62,17 +63,24 @@ public Environment() throws IOException { Process pid = Runtime.getRuntime().exec(command); BufferedReader in = new BufferedReader(new InputStreamReader(pid.getInputStream())); - while (true) { - String line = in.readLine(); - if (line == null) break; - int p = line.indexOf("="); - if (p != -1) { - String name = line.substring(0, p); - String value = line.substring(p + 1); - setProperty(name, value); + try { + while (true) { + String line = in.readLine(); + if (line == null) + break; + int p = line.indexOf("="); + if (p != -1) { + String name = line.substring(0, p); + String value = line.substring(p + 1); + setProperty(name, value); + } } + in.close(); + in = null; + } finally { + IOUtils.closeStream(in); } - in.close(); + try { pid.waitFor(); } catch (InterruptedException e) { diff --git a/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordMean.java b/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordMean.java new file mode 100644 index 0000000000..bc2d658b23 --- /dev/null +++ b/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordMean.java @@ -0,0 +1,196 @@ +package org.apache.hadoop.examples; + +/** + * 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. + */ + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.StringTokenizer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +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.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +public class WordMean extends Configured implements Tool { + + private double mean = 0; + + private final static Text COUNT = new Text("count"); + private final static Text LENGTH = new Text("length"); + private final static LongWritable ONE = new LongWritable(1); + + /** + * Maps words from line of text into 2 key-value pairs; one key-value pair for + * counting the word, another for counting its length. + */ + public static class WordMeanMapper extends + Mapper { + + private LongWritable wordLen = new LongWritable(); + + /** + * Emits 2 key-value pairs for counting the word and its length. Outputs are + * (Text, LongWritable). + * + * @param value + * This will be a line of text coming in from our input file. + */ + public void map(Object key, Text value, Context context) + throws IOException, InterruptedException { + StringTokenizer itr = new StringTokenizer(value.toString()); + while (itr.hasMoreTokens()) { + String string = itr.nextToken(); + this.wordLen.set(string.length()); + context.write(LENGTH, this.wordLen); + context.write(COUNT, ONE); + } + } + } + + /** + * Performs integer summation of all the values for each key. + */ + public static class WordMeanReducer extends + Reducer { + + private LongWritable sum = new LongWritable(); + + /** + * Sums all the individual values within the iterator and writes them to the + * same key. + * + * @param key + * This will be one of 2 constants: LENGTH_STR or COUNT_STR. + * @param values + * This will be an iterator of all the values associated with that + * key. + */ + public void reduce(Text key, Iterable values, Context context) + throws IOException, InterruptedException { + + int theSum = 0; + for (LongWritable val : values) { + theSum += val.get(); + } + sum.set(theSum); + context.write(key, sum); + } + } + + /** + * Reads the output file and parses the summation of lengths, and the word + * count, to perform a quick calculation of the mean. + * + * @param path + * The path to find the output file in. Set in main to the output + * directory. + * @throws IOException + * If it cannot access the output directory, we throw an exception. + */ + private double readAndCalcMean(Path path, Configuration conf) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path file = new Path(path, "part-r-00000"); + + if (!fs.exists(file)) + throw new IOException("Output not found!"); + + BufferedReader br = null; + + // average = total sum / number of elements; + try { + br = new BufferedReader(new InputStreamReader(fs.open(file))); + + long count = 0; + long length = 0; + + String line; + while ((line = br.readLine()) != null) { + StringTokenizer st = new StringTokenizer(line); + + // grab type + String type = st.nextToken(); + + // differentiate + if (type.equals(COUNT.toString())) { + String countLit = st.nextToken(); + count = Long.parseLong(countLit); + } else if (type.equals(LENGTH.toString())) { + String lengthLit = st.nextToken(); + length = Long.parseLong(lengthLit); + } + } + + double theMean = (((double) length) / ((double) count)); + System.out.println("The mean is: " + theMean); + return theMean; + } finally { + br.close(); + } + } + + public static void main(String[] args) throws Exception { + ToolRunner.run(new Configuration(), new WordMean(), args); + } + + @Override + public int run(String[] args) throws Exception { + if (args.length != 2) { + System.err.println("Usage: wordmean "); + return 0; + } + + Configuration conf = getConf(); + + @SuppressWarnings("deprecation") + Job job = new Job(conf, "word mean"); + job.setJarByClass(WordMean.class); + job.setMapperClass(WordMeanMapper.class); + job.setCombinerClass(WordMeanReducer.class); + job.setReducerClass(WordMeanReducer.class); + job.setOutputKeyClass(Text.class); + job.setOutputValueClass(LongWritable.class); + FileInputFormat.addInputPath(job, new Path(args[0])); + Path outputpath = new Path(args[1]); + FileOutputFormat.setOutputPath(job, outputpath); + boolean result = job.waitForCompletion(true); + mean = readAndCalcMean(outputpath, conf); + + return (result ? 0 : 1); + } + + /** + * Only valuable after run() called. + * + * @return Returns the mean value. + */ + public double getMean() { + return mean; + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordMedian.java b/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordMedian.java new file mode 100644 index 0000000000..406d19ed4f --- /dev/null +++ b/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordMedian.java @@ -0,0 +1,208 @@ +package org.apache.hadoop.examples; + +/** + * 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. + */ + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.StringTokenizer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.TaskCounter; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +public class WordMedian extends Configured implements Tool { + + private double median = 0; + private final static IntWritable ONE = new IntWritable(1); + + /** + * Maps words from line of text into a key-value pair; the length of the word + * as the key, and 1 as the value. + */ + public static class WordMedianMapper extends + Mapper { + + private IntWritable length = new IntWritable(); + + /** + * Emits a key-value pair for counting the word. Outputs are (IntWritable, + * IntWritable). + * + * @param value + * This will be a line of text coming in from our input file. + */ + public void map(Object key, Text value, Context context) + throws IOException, InterruptedException { + StringTokenizer itr = new StringTokenizer(value.toString()); + while (itr.hasMoreTokens()) { + String string = itr.nextToken(); + length.set(string.length()); + context.write(length, ONE); + } + } + } + + /** + * Performs integer summation of all the values for each key. + */ + public static class WordMedianReducer extends + Reducer { + + private IntWritable val = new IntWritable(); + + /** + * Sums all the individual values within the iterator and writes them to the + * same key. + * + * @param key + * This will be a length of a word that was read. + * @param values + * This will be an iterator of all the values associated with that + * key. + */ + public void reduce(IntWritable key, Iterable values, + Context context) throws IOException, InterruptedException { + + int sum = 0; + for (IntWritable value : values) { + sum += value.get(); + } + val.set(sum); + context.write(key, val); + } + } + + /** + * This is a standard program to read and find a median value based on a file + * of word counts such as: 1 456, 2 132, 3 56... Where the first values are + * the word lengths and the following values are the number of times that + * words of that length appear. + * + * @param path + * The path to read the HDFS file from (part-r-00000...00001...etc). + * @param medianIndex1 + * The first length value to look for. + * @param medianIndex2 + * The second length value to look for (will be the same as the first + * if there are an even number of words total). + * @throws IOException + * If file cannot be found, we throw an exception. + * */ + private double readAndFindMedian(String path, int medianIndex1, + int medianIndex2, Configuration conf) throws IOException { + FileSystem fs = FileSystem.get(conf); + Path file = new Path(path, "part-r-00000"); + + if (!fs.exists(file)) + throw new IOException("Output not found!"); + + BufferedReader br = null; + + try { + br = new BufferedReader(new InputStreamReader(fs.open(file))); + int num = 0; + + String line; + while ((line = br.readLine()) != null) { + StringTokenizer st = new StringTokenizer(line); + + // grab length + String currLen = st.nextToken(); + + // grab count + String lengthFreq = st.nextToken(); + + int prevNum = num; + num += Integer.parseInt(lengthFreq); + + if (medianIndex2 >= prevNum && medianIndex1 <= num) { + System.out.println("The median is: " + currLen); + br.close(); + return Double.parseDouble(currLen); + } else if (medianIndex2 >= prevNum && medianIndex1 < num) { + String nextCurrLen = st.nextToken(); + double theMedian = (Integer.parseInt(currLen) + Integer + .parseInt(nextCurrLen)) / 2.0; + System.out.println("The median is: " + theMedian); + br.close(); + return theMedian; + } + } + } finally { + br.close(); + } + // error, no median found + return -1; + } + + public static void main(String[] args) throws Exception { + ToolRunner.run(new Configuration(), new WordMedian(), args); + } + + @Override + public int run(String[] args) throws Exception { + if (args.length != 2) { + System.err.println("Usage: wordmedian "); + return 0; + } + + setConf(new Configuration()); + Configuration conf = getConf(); + + @SuppressWarnings("deprecation") + Job job = new Job(conf, "word median"); + job.setJarByClass(WordMedian.class); + job.setMapperClass(WordMedianMapper.class); + job.setCombinerClass(WordMedianReducer.class); + job.setReducerClass(WordMedianReducer.class); + job.setOutputKeyClass(IntWritable.class); + job.setOutputValueClass(IntWritable.class); + FileInputFormat.addInputPath(job, new Path(args[0])); + FileOutputFormat.setOutputPath(job, new Path(args[1])); + boolean result = job.waitForCompletion(true); + + // Wait for JOB 1 -- get middle value to check for Median + + long totalWords = job.getCounters() + .getGroup(TaskCounter.class.getCanonicalName()) + .findCounter("MAP_OUTPUT_RECORDS", "Map output records").getValue(); + int medianIndex1 = (int) Math.ceil((totalWords / 2.0)); + int medianIndex2 = (int) Math.floor((totalWords / 2.0)); + + median = readAndFindMedian(args[1], medianIndex1, medianIndex2, conf); + + return (result ? 0 : 1); + } + + public double getMedian() { + return median; + } +} diff --git a/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordStandardDeviation.java b/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordStandardDeviation.java new file mode 100644 index 0000000000..6a122df652 --- /dev/null +++ b/hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/WordStandardDeviation.java @@ -0,0 +1,210 @@ +package org.apache.hadoop.examples; + +/** + * 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. + */ + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.StringTokenizer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +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.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +public class WordStandardDeviation extends Configured implements Tool { + + private double stddev = 0; + + private final static Text LENGTH = new Text("length"); + private final static Text SQUARE = new Text("square"); + private final static Text COUNT = new Text("count"); + private final static LongWritable ONE = new LongWritable(1); + + /** + * Maps words from line of text into 3 key-value pairs; one key-value pair for + * counting the word, one for counting its length, and one for counting the + * square of its length. + */ + public static class WordStandardDeviationMapper extends + Mapper { + + private LongWritable wordLen = new LongWritable(); + private LongWritable wordLenSq = new LongWritable(); + + /** + * Emits 3 key-value pairs for counting the word, its length, and the + * squares of its length. Outputs are (Text, LongWritable). + * + * @param value + * This will be a line of text coming in from our input file. + */ + public void map(Object key, Text value, Context context) + throws IOException, InterruptedException { + StringTokenizer itr = new StringTokenizer(value.toString()); + while (itr.hasMoreTokens()) { + String string = itr.nextToken(); + + this.wordLen.set(string.length()); + + // the square of an integer is an integer... + this.wordLenSq.set((long) Math.pow(string.length(), 2.0)); + + context.write(LENGTH, this.wordLen); + context.write(SQUARE, this.wordLenSq); + context.write(COUNT, ONE); + } + } + } + + /** + * Performs integer summation of all the values for each key. + */ + public static class WordStandardDeviationReducer extends + Reducer { + + private LongWritable val = new LongWritable(); + + /** + * Sums all the individual values within the iterator and writes them to the + * same key. + * + * @param key + * This will be one of 2 constants: LENGTH_STR, COUNT_STR, or + * SQUARE_STR. + * @param values + * This will be an iterator of all the values associated with that + * key. + */ + public void reduce(Text key, Iterable values, Context context) + throws IOException, InterruptedException { + + int sum = 0; + for (LongWritable value : values) { + sum += value.get(); + } + val.set(sum); + context.write(key, val); + } + } + + /** + * Reads the output file and parses the summation of lengths, the word count, + * and the lengths squared, to perform a quick calculation of the standard + * deviation. + * + * @param path + * The path to find the output file in. Set in main to the output + * directory. + * @throws IOException + * If it cannot access the output directory, we throw an exception. + */ + private double readAndCalcStdDev(Path path, Configuration conf) + throws IOException { + FileSystem fs = FileSystem.get(conf); + Path file = new Path(path, "part-r-00000"); + + if (!fs.exists(file)) + throw new IOException("Output not found!"); + + double stddev = 0; + BufferedReader br = null; + try { + br = new BufferedReader(new InputStreamReader(fs.open(file))); + long count = 0; + long length = 0; + long square = 0; + String line; + while ((line = br.readLine()) != null) { + StringTokenizer st = new StringTokenizer(line); + + // grab type + String type = st.nextToken(); + + // differentiate + if (type.equals(COUNT.toString())) { + String countLit = st.nextToken(); + count = Long.parseLong(countLit); + } else if (type.equals(LENGTH.toString())) { + String lengthLit = st.nextToken(); + length = Long.parseLong(lengthLit); + } else if (type.equals(SQUARE.toString())) { + String squareLit = st.nextToken(); + square = Long.parseLong(squareLit); + } + } + // average = total sum / number of elements; + double mean = (((double) length) / ((double) count)); + // standard deviation = sqrt((sum(lengths ^ 2)/count) - (mean ^ 2)) + mean = Math.pow(mean, 2.0); + double term = (((double) square / ((double) count))); + stddev = Math.sqrt((term - mean)); + System.out.println("The standard deviation is: " + stddev); + } finally { + br.close(); + } + return stddev; + } + + public static void main(String[] args) throws Exception { + ToolRunner.run(new Configuration(), new WordStandardDeviation(), + args); + } + + @Override + public int run(String[] args) throws Exception { + if (args.length != 2) { + System.err.println("Usage: wordstddev "); + return 0; + } + + Configuration conf = getConf(); + + @SuppressWarnings("deprecation") + Job job = new Job(conf, "word stddev"); + job.setJarByClass(WordStandardDeviation.class); + job.setMapperClass(WordStandardDeviationMapper.class); + job.setCombinerClass(WordStandardDeviationReducer.class); + job.setReducerClass(WordStandardDeviationReducer.class); + job.setOutputKeyClass(Text.class); + job.setOutputValueClass(LongWritable.class); + FileInputFormat.addInputPath(job, new Path(args[0])); + Path outputpath = new Path(args[1]); + FileOutputFormat.setOutputPath(job, outputpath); + boolean result = job.waitForCompletion(true); + + // read output and calculate standard deviation + stddev = readAndCalcStdDev(outputpath, conf); + + return (result ? 0 : 1); + } + + public double getStandardDeviation() { + return stddev; + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobQueueClient.java b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobQueueClient.java index 68f5fc4366..e4dc310077 100644 --- a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobQueueClient.java +++ b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobQueueClient.java @@ -170,10 +170,10 @@ List expandQueueList(JobQueueInfo[] rootQueues) { * registered with the {@link QueueManager}. Display of the Jobs is determine * by the boolean * - * @throws IOException + * @throws IOException, InterruptedException */ private void displayQueueInfo(String queue, boolean showJobs) - throws IOException { + throws IOException, InterruptedException { JobQueueInfo jobQueueInfo = jc.getQueueInfo(queue); if (jobQueueInfo == null) { diff --git a/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml b/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml index 3d576f4c21..28edfbfe36 100644 --- a/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml +++ b/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml @@ -388,9 +388,4 @@
- - - - - - + diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/examples/TestWordStats.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/examples/TestWordStats.java new file mode 100644 index 0000000000..a728386e62 --- /dev/null +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/examples/TestWordStats.java @@ -0,0 +1,272 @@ +package org.apache.hadoop.examples; + +import static org.junit.Assert.assertEquals; + +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.StringTokenizer; +import java.util.TreeMap; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.ToolRunner; +import org.junit.Before; +import org.junit.Test; + +public class TestWordStats { + + private final static String INPUT = "src/test/mapred/org/apache/hadoop/examples/pi/math"; + private final static String MEAN_OUTPUT = "build/data/mean_output"; + private final static String MEDIAN_OUTPUT = "build/data/median_output"; + private final static String STDDEV_OUTPUT = "build/data/stddev_output"; + + /** + * Modified internal test class that is designed to read all the files in the + * input directory, and find the standard deviation between all of the word + * lengths. + */ + public static class WordStdDevReader { + private long wordsRead = 0; + private long wordLengthsRead = 0; + private long wordLengthsReadSquared = 0; + + public WordStdDevReader() { + } + + public double read(String path) throws IOException { + FileSystem fs = FileSystem.get(new Configuration()); + FileStatus[] files = fs.listStatus(new Path(path)); + + for (FileStatus fileStat : files) { + if (!fileStat.isFile()) + continue; + + BufferedReader br = null; + + try { + br = new BufferedReader(new InputStreamReader(fs.open(fileStat.getPath()))); + + String line; + while ((line = br.readLine()) != null) { + StringTokenizer st = new StringTokenizer(line); + String word; + while (st.hasMoreTokens()) { + word = st.nextToken(); + this.wordsRead++; + this.wordLengthsRead += word.length(); + this.wordLengthsReadSquared += (long) Math.pow(word.length(), 2.0); + } + } + + } catch (IOException e) { + System.out.println("Output could not be read!"); + throw e; + } finally { + br.close(); + } + } + + double mean = (((double) this.wordLengthsRead) / ((double) this.wordsRead)); + mean = Math.pow(mean, 2.0); + double term = (((double) this.wordLengthsReadSquared / ((double) this.wordsRead))); + double stddev = Math.sqrt((term - mean)); + return stddev; + } + + } + + /** + * Modified internal test class that is designed to read all the files in the + * input directory, and find the median length of all the words. + */ + public static class WordMedianReader { + private long wordsRead = 0; + private TreeMap map = new TreeMap(); + + public WordMedianReader() { + } + + public double read(String path) throws IOException { + FileSystem fs = FileSystem.get(new Configuration()); + FileStatus[] files = fs.listStatus(new Path(path)); + + int num = 0; + + for (FileStatus fileStat : files) { + if (!fileStat.isFile()) + continue; + + BufferedReader br = null; + + try { + br = new BufferedReader(new InputStreamReader(fs.open(fileStat.getPath()))); + + String line; + while ((line = br.readLine()) != null) { + StringTokenizer st = new StringTokenizer(line); + String word; + while (st.hasMoreTokens()) { + word = st.nextToken(); + this.wordsRead++; + if (this.map.get(word.length()) == null) { + this.map.put(word.length(), 1); + } else { + int count = this.map.get(word.length()); + this.map.put(word.length(), count + 1); + } + } + } + } catch (IOException e) { + System.out.println("Output could not be read!"); + throw e; + } finally { + br.close(); + } + } + + int medianIndex1 = (int) Math.ceil((this.wordsRead / 2.0)); + int medianIndex2 = (int) Math.floor((this.wordsRead / 2.0)); + + for (Integer key : this.map.navigableKeySet()) { + int prevNum = num; + num += this.map.get(key); + + if (medianIndex2 >= prevNum && medianIndex1 <= num) { + return key; + } else if (medianIndex2 >= prevNum && medianIndex1 < num) { + Integer nextCurrLen = this.map.navigableKeySet().iterator().next(); + double median = (key + nextCurrLen) / 2.0; + return median; + } + } + return -1; + } + + } + + /** + * Modified internal test class that is designed to read all the files in the + * input directory, and find the mean length of all the words. + */ + public static class WordMeanReader { + private long wordsRead = 0; + private long wordLengthsRead = 0; + + public WordMeanReader() { + } + + public double read(String path) throws IOException { + FileSystem fs = FileSystem.get(new Configuration()); + FileStatus[] files = fs.listStatus(new Path(path)); + + for (FileStatus fileStat : files) { + if (!fileStat.isFile()) + continue; + + BufferedReader br = null; + + try { + br = new BufferedReader(new InputStreamReader(fs.open(fileStat.getPath()))); + + String line; + while ((line = br.readLine()) != null) { + StringTokenizer st = new StringTokenizer(line); + String word; + while (st.hasMoreTokens()) { + word = st.nextToken(); + this.wordsRead++; + this.wordLengthsRead += word.length(); + } + } + } catch (IOException e) { + System.out.println("Output could not be read!"); + throw e; + } finally { + br.close(); + } + } + + double mean = (((double) this.wordLengthsRead) / ((double) this.wordsRead)); + return mean; + } + + } + + /** + * Internal class designed to delete the output directory. Meant solely for + * use before and after the test is run; this is so next iterations of the + * test do not encounter a "file already exists" error. + * + * @param dir + * The directory to delete. + * @return Returns whether the deletion was successful or not. + */ + public static boolean deleteDir(File dir) { + if (dir.isDirectory()) { + String[] children = dir.list(); + for (int i = 0; i < children.length; i++) { + boolean success = deleteDir(new File(dir, children[i])); + if (!success) { + System.out.println("Could not delete directory after test!"); + return false; + } + } + } + + // The directory is now empty so delete it + return dir.delete(); + } + + @Before public void setup() throws Exception { + deleteDir(new File(MEAN_OUTPUT)); + deleteDir(new File(MEDIAN_OUTPUT)); + deleteDir(new File(STDDEV_OUTPUT)); + } + + @Test public void testGetTheMean() throws Exception { + String args[] = new String[2]; + args[0] = INPUT; + args[1] = MEAN_OUTPUT; + + WordMean wm = new WordMean(); + ToolRunner.run(new Configuration(), wm, args); + double mean = wm.getMean(); + + // outputs MUST match + WordMeanReader wr = new WordMeanReader(); + assertEquals(mean, wr.read(INPUT), 0.0); + } + + @Test public void testGetTheMedian() throws Exception { + String args[] = new String[2]; + args[0] = INPUT; + args[1] = MEDIAN_OUTPUT; + + WordMedian wm = new WordMedian(); + ToolRunner.run(new Configuration(), wm, args); + double median = wm.getMedian(); + + // outputs MUST match + WordMedianReader wr = new WordMedianReader(); + assertEquals(median, wr.read(INPUT), 0.0); + } + + @Test public void testGetTheStandardDeviation() throws Exception { + String args[] = new String[2]; + args[0] = INPUT; + args[1] = STDDEV_OUTPUT; + + WordStandardDeviation wsd = new WordStandardDeviation(); + ToolRunner.run(new Configuration(), wsd, args); + double stddev = wsd.getStandardDeviation(); + + // outputs MUST match + WordStdDevReader wr = new WordStdDevReader(); + assertEquals(stddev, wr.read(INPUT), 0.0); + } + +}