diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh index 6325e6a193b..76b020a01e3 100755 --- a/dev-support/test-patch.sh +++ b/dev-support/test-patch.sh @@ -598,8 +598,8 @@ runTests () { echo "" echo "" - echo "$MVN clean install test -Pnative -D${PROJECT_NAME}PatchProcess" - $MVN clean install test -Pnative -D${PROJECT_NAME}PatchProcess + 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 failed_tests=`find . -name 'TEST*.xml' | xargs $GREP -l -E "36000.

-

hadoop.http.authentication.signature.secret.file: The signature secret - file for signing the authentication tokens. If not set a random secret is generated at +

hadoop.http.authentication.signature.secret: The signature secret for + signing the authentication tokens. If not set a random secret is generated at startup time. The same secret should be used for all nodes in the cluster, JobTracker, - NameNode, DataNode and TastTracker. The default value is - ${user.home}/hadoop-http-auth-signature-secret. - IMPORTANT: This file should be readable only by the Unix user running the daemons. + NameNode, DataNode and TastTracker. The default value is a hadoop value.

hadoop.http.authentication.cookie.domain: The domain to use for the HTTP diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java index c310aa65e6c..4fb1d190663 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java @@ -1632,10 +1632,6 @@ public class Configuration implements Iterable>, try { doc = DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument(); - - // Allow a broader set of control characters to appear in job confs. - // cf https://issues.apache.org/jira/browse/MAPREDUCE-109 - doc.setXmlVersion( "1.1" ); } catch (ParserConfigurationException pe) { throw new IOException(pe); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java index d1eae086f90..71c82357577 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java @@ -264,15 +264,9 @@ public class LocalDirAllocator { Path tmpDir = new Path(localDirs[i]); if(localFS.mkdirs(tmpDir)|| localFS.exists(tmpDir)) { try { - - File tmpFile = tmpDir.isAbsolute() - ? new File(localFS.makeQualified(tmpDir).toUri()) - : new File(localDirs[i]); - - DiskChecker.checkDir(tmpFile); - dirs.add(tmpFile.getPath()); - dfList.add(new DF(tmpFile, 30000)); - + DiskChecker.checkDir(new File(localDirs[i])); + dirs.add(localDirs[i]); + dfList.add(new DF(new File(localDirs[i]), 30000)); } catch (DiskErrorException de) { LOG.warn( localDirs[i] + " is not writable\n", de); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java index c526e102865..00cdf32746f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java @@ -20,7 +20,6 @@ package org.apache.hadoop.http; import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintWriter; -import java.io.InterruptedIOException; import java.net.BindException; import java.net.InetSocketAddress; import java.net.URL; @@ -125,29 +124,6 @@ public class HttpServer implements FilterContainer { boolean findPort, Configuration conf, Connector connector) throws IOException { this(name, bindAddress, port, findPort, conf, null, connector); } - - /** - * Create a status server on the given port. Allows you to specify the - * path specifications that this server will be serving so that they will be - * added to the filters properly. - * - * @param name The name of the server - * @param bindAddress The address for this server - * @param port The port to use on the server - * @param findPort whether the server should start at the given port and - * increment by 1 until it finds a free port. - * @param conf Configuration - * @param pathSpecs Path specifications that this httpserver will be serving. - * These will be added to any filters. - */ - public HttpServer(String name, String bindAddress, int port, - boolean findPort, Configuration conf, String[] pathSpecs) throws IOException { - this(name, bindAddress, port, findPort, conf, null, null); - for (String path : pathSpecs) { - LOG.info("adding path spec: " + path); - addFilterPathMapping(path, webAppContext); - } - } /** * Create a status server on the given port. @@ -283,7 +259,7 @@ public class HttpServer implements FilterContainer { if (logDir != null) { Context logContext = new Context(parent, "/logs"); logContext.setResourceBase(logDir); - logContext.addServlet(AdminAuthorizedServlet.class, "/*"); + logContext.addServlet(AdminAuthorizedServlet.class, "/"); logContext.setDisplayName("logs"); setContextAttributes(logContext, conf); defaultContexts.put(logContext, true); @@ -684,9 +660,6 @@ public class HttpServer implements FilterContainer { } } catch (IOException e) { throw e; - } catch (InterruptedException e) { - throw (IOException) new InterruptedIOException( - "Interrupted while starting HTTP server").initCause(e); } catch (Exception e) { throw new IOException("Problem starting http server", e); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java index 04d08c5142d..a055a7fd46a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java @@ -199,7 +199,7 @@ public class ProtocolSignature implements Writable { * @param protocol protocol * @return the server's protocol signature */ - public static ProtocolSignature getProtocolSignature( + static ProtocolSignature getProtocolSignature( int clientMethodsHashCode, long serverVersion, Class protocol) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java index d94b69f1836..b22aaa009c1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java @@ -516,25 +516,4 @@ public class NetUtils { } catch (UnknownHostException ignore) { } return addr; } - - /** - * Given an InetAddress, checks to see if the address is a local address, by - * comparing the address with all the interfaces on the node. - * @param addr address to check if it is local node's address - * @return true if the address corresponds to the local node - */ - public static boolean isLocalAddress(InetAddress addr) { - // Check if the address is any local or loop back - boolean local = addr.isAnyLocalAddress() || addr.isLoopbackAddress(); - - // Check if the address is defined on any interface - if (!local) { - try { - local = NetworkInterface.getByInetAddress(addr) != null; - } catch (SocketException e) { - local = false; - } - } - return local; - } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java index 666632d5bfa..cd6ab7b3260 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java @@ -22,9 +22,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.http.FilterContainer; import org.apache.hadoop.http.FilterInitializer; -import java.io.FileReader; -import java.io.IOException; -import java.io.Reader; import java.util.HashMap; import java.util.Map; @@ -43,10 +40,8 @@ import java.util.Map; */ public class AuthenticationFilterInitializer extends FilterInitializer { - static final String PREFIX = "hadoop.http.authentication."; + private static final String PREFIX = "hadoop.http.authentication."; - static final String SIGNATURE_SECRET_FILE = AuthenticationFilter.SIGNATURE_SECRET + ".file"; - /** * Initializes Alfredo AuthenticationFilter. *

@@ -72,25 +67,6 @@ public class AuthenticationFilterInitializer extends FilterInitializer { } } - String signatureSecretFile = filterConfig.get(SIGNATURE_SECRET_FILE); - if (signatureSecretFile == null) { - throw new RuntimeException("Undefined property: " + SIGNATURE_SECRET_FILE); - } - - try { - StringBuilder secret = new StringBuilder(); - Reader reader = new FileReader(signatureSecretFile); - int c = reader.read(); - while (c > -1) { - secret.append((char)c); - c = reader.read(); - } - reader.close(); - filterConfig.put(AuthenticationFilter.SIGNATURE_SECRET, secret.toString()); - } catch (IOException ex) { - throw new RuntimeException("Could not read HTTP signature secret file: " + signatureSecretFile); - } - container.addFilter("authentication", AuthenticationFilter.class.getName(), filterConfig); 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 96a989fc39f..8e903cf308d 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 @@ -475,10 +475,7 @@ else 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/common/templates/conf/capacity-scheduler.xml ${HADOOP_CONF_DIR}/capacity-scheduler.xml - fi - if [ ! -e ${HADOOP_CONF_DIR}/hadoop-metrics2.properties ]; then - cp ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-metrics2.properties ${HADOOP_CONF_DIR}/hadoop-metrics2.properties + template_generator ${HADOOP_PREFIX}/share/hadoop/templates/conf/capacity-scheduler.xml ${HADOOP_CONF_DIR}/capacity-scheduler.xml fi if [ ! -e ${HADOOP_CONF_DIR}/log4j.properties ]; then cp ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/log4j.properties ${HADOOP_CONF_DIR}/log4j.properties diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-metrics2.properties b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-metrics2.properties deleted file mode 100644 index 4a1019385c0..00000000000 --- a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-metrics2.properties +++ /dev/null @@ -1,20 +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. - -# syntax: [prefix].[source|sink|jmx].[instance].[options] -# See package.html for org.apache.hadoop.metrics2 for details - -*.period=60 - 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 index ffec60355ae..69e078380c1 100644 --- 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 @@ -144,26 +144,6 @@ - - dfs.web.authentication.kerberos.principal - HTTP/_HOST@${local.realm} - - The HTTP Kerberos principal used by Hadoop-Auth in the HTTP endpoint. - - The HTTP Kerberos principal MUST start with 'HTTP/' per Kerberos - HTTP SPENGO specification. - - - - - dfs.web.authentication.kerberos.keytab - /etc/security/keytabs/nn.service.keytab - - The Kerberos keytab file with the credentials for the - HTTP Kerberos principal used by Hadoop-Auth in the HTTP endpoint. - - - dfs.namenode.keytab.file /etc/security/keytabs/nn.service.keytab diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/log4j.properties b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/log4j.properties deleted file mode 100644 index 16c6aa6890e..00000000000 --- a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/log4j.properties +++ /dev/null @@ -1,213 +0,0 @@ -# Copyright 2011 The Apache Software Foundation -# -# 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. - -# Define some default values that can be overridden by system properties -hadoop.root.logger=INFO,console -hadoop.log.dir=. -hadoop.log.file=hadoop.log - -# -# Job Summary Appender -# -# Use following logger to send summary to separate file defined by -# hadoop.mapreduce.jobsummary.log.file rolled daily: -# hadoop.mapreduce.jobsummary.logger=INFO,JSA -# -hadoop.mapreduce.jobsummary.logger=${hadoop.root.logger} -hadoop.mapreduce.jobsummary.log.file=hadoop-mapreduce.jobsummary.log - -# Define the root logger to the system property "hadoop.root.logger". -log4j.rootLogger=${hadoop.root.logger}, EventCounter - -# Logging Threshold -log4j.threshold=ALL - -# -# Daily Rolling File Appender -# - -log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender -log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file} - -# Rollver at midnight -log4j.appender.DRFA.DatePattern=.yyyy-MM-dd - -# 30-day backup -#log4j.appender.DRFA.MaxBackupIndex=30 -log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout - -# Pattern format: Date LogLevel LoggerName LogMessage -log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n -# Debugging Pattern format -#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n - - -# -# console -# Add "console" to rootlogger above if you want to use this -# - -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n - -# -# TaskLog Appender -# - -#Default values -hadoop.tasklog.taskid=null -hadoop.tasklog.iscleanup=false -hadoop.tasklog.noKeepSplits=4 -hadoop.tasklog.totalLogFileSize=100 -hadoop.tasklog.purgeLogSplits=true -hadoop.tasklog.logsRetainHours=12 - -log4j.appender.TLA=org.apache.hadoop.mapred.TaskLogAppender -log4j.appender.TLA.taskId=${hadoop.tasklog.taskid} -log4j.appender.TLA.isCleanup=${hadoop.tasklog.iscleanup} -log4j.appender.TLA.totalLogFileSize=${hadoop.tasklog.totalLogFileSize} - -log4j.appender.TLA.layout=org.apache.log4j.PatternLayout -log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n - -# -#Security appender -# -hadoop.security.log.file=SecurityAuth.audit -log4j.appender.DRFAS=org.apache.log4j.DailyRollingFileAppender -log4j.appender.DRFAS.File=${hadoop.log.dir}/${hadoop.security.log.file} - -log4j.appender.DRFAS.layout=org.apache.log4j.PatternLayout -log4j.appender.DRFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n -#new logger -# Define some default values that can be overridden by system properties -hadoop.security.logger=INFO,console -log4j.category.SecurityLogger=${hadoop.security.logger} - -# hdfs audit logging - -hdfs.audit.logger=INFO,console -log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=${hdfs.audit.logger} -log4j.additivity.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=false -log4j.appender.DRFAAUDIT=org.apache.log4j.DailyRollingFileAppender -log4j.appender.DRFAAUDIT.File=${hadoop.log.dir}/hdfs-audit.log -log4j.appender.DRFAAUDIT.layout=org.apache.log4j.PatternLayout -log4j.appender.DRFAAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n -log4j.appender.DRFAAUDIT.DatePattern=.yyyy-MM-dd - -# mapred audit logging - -mapred.audit.logger=INFO,console -log4j.logger.org.apache.hadoop.mapred.AuditLogger=${mapred.audit.logger} -log4j.additivity.org.apache.hadoop.mapred.AuditLogger=false -log4j.appender.MRAUDIT=org.apache.log4j.DailyRollingFileAppender -log4j.appender.MRAUDIT.File=${hadoop.log.dir}/mapred-audit.log -log4j.appender.MRAUDIT.layout=org.apache.log4j.PatternLayout -log4j.appender.MRAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n -log4j.appender.MRAUDIT.DatePattern=.yyyy-MM-dd - -# -# Rolling File Appender -# - -#log4j.appender.RFA=org.apache.log4j.RollingFileAppender -#log4j.appender.RFA.File=${hadoop.log.dir}/${hadoop.log.file} - -# Logfile size and and 30-day backups -#log4j.appender.RFA.MaxFileSize=1MB -#log4j.appender.RFA.MaxBackupIndex=30 - -#log4j.appender.RFA.layout=org.apache.log4j.PatternLayout -#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} - %m%n -#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n - -# -# FSNamesystem Audit logging -# All audit events are logged at INFO level -# -log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=WARN - -# Custom Logging levels - -#log4j.logger.org.apache.hadoop.mapred.JobTracker=DEBUG -#log4j.logger.org.apache.hadoop.mapred.TaskTracker=DEBUG -#log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=DEBUG - -# Jets3t library -log4j.logger.org.jets3t.service.impl.rest.httpclient.RestS3Service=ERROR - -# -# Event Counter Appender -# Sends counts of logging messages at different severity levels to Hadoop Metrics. -# -log4j.appender.EventCounter=org.apache.hadoop.log.metrics.EventCounter - -# -# Job Summary Appender -# -log4j.appender.JSA=org.apache.log4j.DailyRollingFileAppender -log4j.appender.JSA.File=${hadoop.log.dir}/${hadoop.mapreduce.jobsummary.log.file} -log4j.appender.JSA.layout=org.apache.log4j.PatternLayout -log4j.appender.JSA.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n -log4j.appender.JSA.DatePattern=.yyyy-MM-dd -log4j.logger.org.apache.hadoop.mapred.JobInProgress$JobSummary=${hadoop.mapreduce.jobsummary.logger} -log4j.additivity.org.apache.hadoop.mapred.JobInProgress$JobSummary=false - -# -# MapReduce Audit Log Appender -# - -# Set the MapReduce audit log filename -#hadoop.mapreduce.audit.log.file=hadoop-mapreduce.audit.log - -# Appender for AuditLogger. -# Requires the following system properties to be set -# - hadoop.log.dir (Hadoop Log directory) -# - hadoop.mapreduce.audit.log.file (MapReduce audit log filename) - -#log4j.logger.org.apache.hadoop.mapred.AuditLogger=INFO,MRAUDIT -#log4j.additivity.org.apache.hadoop.mapred.AuditLogger=false -#log4j.appender.MRAUDIT=org.apache.log4j.DailyRollingFileAppender -#log4j.appender.MRAUDIT.File=${hadoop.log.dir}/${hadoop.mapreduce.audit.log.file} -#log4j.appender.MRAUDIT.DatePattern=.yyyy-MM-dd -#log4j.appender.MRAUDIT.layout=org.apache.log4j.PatternLayout -#log4j.appender.MRAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n - -# -# Yarn ResourceManager Application Summary Log -# -# Set the ResourceManager summary log filename -#yarn.server.resourcemanager.appsummary.log.file=rm-appsummary.log -# Set the ResourceManager summary log level and appender -#yarn.server.resourcemanager.appsummary.logger=INFO,RMSUMMARY - -# Appender for ResourceManager Application Summary Log - rolled daily -# Requires the following properties to be set -# - hadoop.log.dir (Hadoop Log directory) -# - yarn.server.resourcemanager.appsummary.log.file (resource manager app summary log filename) -# - yarn.server.resourcemanager.appsummary.logger (resource manager app summary log level and appender) - -#log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.RMAppManager$ApplicationSummary=${yarn.server.resourcemanager.appsummary.logger} -#log4j.additivity.org.apache.hadoop.yarn.server.resourcemanager.RMAppManager$ApplicationSummary=false -#log4j.appender.RMSUMMARY=org.apache.log4j.DailyRollingFileAppender -#log4j.appender.RMSUMMARY.File=${hadoop.log.dir}/${yarn.server.resourcemanager.appsummary.log.file} -#log4j.appender.RMSUMMARY.layout=org.apache.log4j.PatternLayout -#log4j.appender.RMSUMMARY.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n -#log4j.appender.RMSUMMARY.DatePattern=.yyyy-MM-dd diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index e34c2023738..d4b40305592 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -808,8 +808,8 @@ - hadoop.http.authentication.signature.secret.file - ${user.home}/hadoop-http-auth-signature-secret + hadoop.http.authentication.signature.secret + hadoop The signature secret for signing the authentication tokens. If not set a random secret is generated at startup time. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java index 5842db199de..f9f14fb8480 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java @@ -58,7 +58,7 @@ public class TestConfiguration extends TestCase { } private void startConfig() throws IOException{ - out.write("\n"); + out.write("\n"); out.write("\n"); } @@ -221,18 +221,6 @@ public class TestConfiguration extends TestCase { assertEquals("this contains a comment", conf.get("my.comment")); } - public void testControlAInValue() throws IOException { - out = new BufferedWriter(new FileWriter(CONFIG)); - startConfig(); - appendProperty("my.char", ""); - appendProperty("my.string", "somestring"); - endConfig(); - Path fileResource = new Path(CONFIG); - conf.addResource(fileResource); - assertEquals("\u0001", conf.get("my.char")); - assertEquals("some\u0001string", conf.get("my.string")); - } - public void testTrim() throws IOException { out=new BufferedWriter(new FileWriter(CONFIG)); startConfig(); @@ -310,7 +298,7 @@ public class TestConfiguration extends TestCase { conf.writeXml(baos); String result = baos.toString(); assertTrue("Result has proper header", result.startsWith( - "")); + "")); assertTrue("Result has proper footer", result.endsWith("")); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java index e87f2d122bf..1e22a73bbac 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java @@ -20,48 +20,40 @@ package org.apache.hadoop.fs; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; -import java.util.Arrays; -import java.util.Collection; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.Shell; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; -import org.junit.Test; - -import static org.junit.Assert.*; +import junit.framework.TestCase; /** This test LocalDirAllocator works correctly; - * Every test case uses different buffer dirs to + * Every test case uses different buffer dirs to * enforce the AllocatorPerContext initialization. * This test does not run on Cygwin because under Cygwin * a directory can be created in a read-only directory * which breaks this test. - */ -@RunWith(Parameterized.class) -public class TestLocalDirAllocator { + */ +public class TestLocalDirAllocator extends TestCase { final static private Configuration conf = new Configuration(); final static private String BUFFER_DIR_ROOT = "build/test/temp"; - final static private String ABSOLUTE_DIR_ROOT; - final static private String QUALIFIED_DIR_ROOT; final static private Path BUFFER_PATH_ROOT = new Path(BUFFER_DIR_ROOT); final static private File BUFFER_ROOT = new File(BUFFER_DIR_ROOT); - final static private String CONTEXT = "fs.client.buffer.dir"; + final static private String BUFFER_DIR[] = new String[] { + BUFFER_DIR_ROOT+"/tmp0", BUFFER_DIR_ROOT+"/tmp1", BUFFER_DIR_ROOT+"/tmp2", + BUFFER_DIR_ROOT+"/tmp3", BUFFER_DIR_ROOT+"/tmp4", BUFFER_DIR_ROOT+"/tmp5", + BUFFER_DIR_ROOT+"/tmp6"}; + final static private Path BUFFER_PATH[] = new Path[] { + new Path(BUFFER_DIR[0]), new Path(BUFFER_DIR[1]), new Path(BUFFER_DIR[2]), + new Path(BUFFER_DIR[3]), new Path(BUFFER_DIR[4]), new Path(BUFFER_DIR[5]), + new Path(BUFFER_DIR[6])}; + final static private String CONTEXT = "dfs.client.buffer.dir"; final static private String FILENAME = "block"; - final static private LocalDirAllocator dirAllocator = + final static private LocalDirAllocator dirAllocator = new LocalDirAllocator(CONTEXT); static LocalFileSystem localFs; final static private boolean isWindows = System.getProperty("os.name").startsWith("Windows"); final static int SMALL_FILE_SIZE = 100; - final static private String RELATIVE = "/RELATIVE"; - final static private String ABSOLUTE = "/ABSOLUTE"; - final static private String QUALIFIED = "/QUALIFIED"; - final private String ROOT; - final private String PREFIX; - static { try { localFs = FileSystem.getLocal(conf); @@ -71,214 +63,170 @@ public class TestLocalDirAllocator { e.printStackTrace(); System.exit(-1); } - - ABSOLUTE_DIR_ROOT = new Path(localFs.getWorkingDirectory(), - BUFFER_DIR_ROOT).toUri().getPath(); - QUALIFIED_DIR_ROOT = new Path(localFs.getWorkingDirectory(), - BUFFER_DIR_ROOT).toUri().toString(); - } - - public TestLocalDirAllocator(String root, String prefix) { - ROOT = root; - PREFIX = prefix; - } - - @Parameters - public static Collection params() { - Object [][] data = new Object[][] { - { BUFFER_DIR_ROOT, RELATIVE }, - { ABSOLUTE_DIR_ROOT, ABSOLUTE }, - { QUALIFIED_DIR_ROOT, QUALIFIED } - }; - - return Arrays.asList(data); } private static void rmBufferDirs() throws IOException { assertTrue(!localFs.exists(BUFFER_PATH_ROOT) || localFs.delete(BUFFER_PATH_ROOT, true)); } - - private static void validateTempDirCreation(String dir) throws IOException { + + private void validateTempDirCreation(int i) throws IOException { File result = createTempFile(SMALL_FILE_SIZE); - assertTrue("Checking for " + dir + " in " + result + " - FAILED!", - result.getPath().startsWith(new Path(dir, FILENAME).toUri().getPath())); + assertTrue("Checking for " + BUFFER_DIR[i] + " in " + result + " - FAILED!", + result.getPath().startsWith(new File(BUFFER_DIR[i], FILENAME).getPath())); } - - private static File createTempFile() throws IOException { - return createTempFile(-1); + + private File createTempFile() throws IOException { + File result = dirAllocator.createTmpFileForWrite(FILENAME, -1, conf); + result.delete(); + return result; } - - private static File createTempFile(long size) throws IOException { + + private File createTempFile(long size) throws IOException { File result = dirAllocator.createTmpFileForWrite(FILENAME, size, conf); result.delete(); return result; } - - private String buildBufferDir(String dir, int i) { - return dir + PREFIX + i; - } - - /** Two buffer dirs. The first dir does not exist & is on a read-only disk; + + /** Two buffer dirs. The first dir does not exist & is on a read-only disk; * The second dir exists & is RW * @throws Exception */ - @Test public void test0() throws Exception { if (isWindows) return; - String dir0 = buildBufferDir(ROOT, 0); - String dir1 = buildBufferDir(ROOT, 1); try { - conf.set(CONTEXT, dir0 + "," + dir1); - assertTrue(localFs.mkdirs(new Path(dir1))); + conf.set(CONTEXT, BUFFER_DIR[0]+","+BUFFER_DIR[1]); + assertTrue(localFs.mkdirs(BUFFER_PATH[1])); BUFFER_ROOT.setReadOnly(); - validateTempDirCreation(dir1); - validateTempDirCreation(dir1); + validateTempDirCreation(1); + validateTempDirCreation(1); } finally { Shell.execCommand(new String[]{"chmod", "u+w", BUFFER_DIR_ROOT}); rmBufferDirs(); } } - - /** Two buffer dirs. The first dir exists & is on a read-only disk; + + /** Two buffer dirs. The first dir exists & is on a read-only disk; * The second dir exists & is RW * @throws Exception */ - @Test public void test1() throws Exception { if (isWindows) return; - String dir1 = buildBufferDir(ROOT, 1); - String dir2 = buildBufferDir(ROOT, 2); try { - conf.set(CONTEXT, dir1 + "," + dir2); - assertTrue(localFs.mkdirs(new Path(dir2))); + conf.set(CONTEXT, BUFFER_DIR[1]+","+BUFFER_DIR[2]); + assertTrue(localFs.mkdirs(BUFFER_PATH[2])); BUFFER_ROOT.setReadOnly(); - validateTempDirCreation(dir2); - validateTempDirCreation(dir2); + validateTempDirCreation(2); + validateTempDirCreation(2); } finally { Shell.execCommand(new String[]{"chmod", "u+w", BUFFER_DIR_ROOT}); rmBufferDirs(); } } /** Two buffer dirs. Both do not exist but on a RW disk. - * Check if tmp dirs are allocated in a round-robin + * Check if tmp dirs are allocated in a round-robin */ - @Test public void test2() throws Exception { if (isWindows) return; - String dir2 = buildBufferDir(ROOT, 2); - String dir3 = buildBufferDir(ROOT, 3); try { - conf.set(CONTEXT, dir2 + "," + dir3); + conf.set(CONTEXT, BUFFER_DIR[2]+","+BUFFER_DIR[3]); // create the first file, and then figure the round-robin sequence createTempFile(SMALL_FILE_SIZE); int firstDirIdx = (dirAllocator.getCurrentDirectoryIndex() == 0) ? 2 : 3; int secondDirIdx = (firstDirIdx == 2) ? 3 : 2; - + // check if tmp dirs are allocated in a round-robin manner - validateTempDirCreation(buildBufferDir(ROOT, firstDirIdx)); - validateTempDirCreation(buildBufferDir(ROOT, secondDirIdx)); - validateTempDirCreation(buildBufferDir(ROOT, firstDirIdx)); + validateTempDirCreation(firstDirIdx); + validateTempDirCreation(secondDirIdx); + validateTempDirCreation(firstDirIdx); } finally { rmBufferDirs(); } } - /** Two buffer dirs. Both exists and on a R/W disk. + /** Two buffer dirs. Both exists and on a R/W disk. * Later disk1 becomes read-only. * @throws Exception */ - @Test public void test3() throws Exception { if (isWindows) return; - String dir3 = buildBufferDir(ROOT, 3); - String dir4 = buildBufferDir(ROOT, 4); try { - conf.set(CONTEXT, dir3 + "," + dir4); - assertTrue(localFs.mkdirs(new Path(dir3))); - assertTrue(localFs.mkdirs(new Path(dir4))); - - // Create the first small file + conf.set(CONTEXT, BUFFER_DIR[3]+","+BUFFER_DIR[4]); + assertTrue(localFs.mkdirs(BUFFER_PATH[3])); + assertTrue(localFs.mkdirs(BUFFER_PATH[4])); + + // create the first file with size, and then figure the round-robin sequence createTempFile(SMALL_FILE_SIZE); - // Determine the round-robin sequence int nextDirIdx = (dirAllocator.getCurrentDirectoryIndex() == 0) ? 3 : 4; - validateTempDirCreation(buildBufferDir(ROOT, nextDirIdx)); + validateTempDirCreation(nextDirIdx); // change buffer directory 2 to be read only - new File(new Path(dir4).toUri().getPath()).setReadOnly(); - validateTempDirCreation(dir3); - validateTempDirCreation(dir3); + new File(BUFFER_DIR[4]).setReadOnly(); + validateTempDirCreation(3); + validateTempDirCreation(3); } finally { rmBufferDirs(); } } - + /** * Two buffer dirs, on read-write disk. - * + * * Try to create a whole bunch of files. * Verify that they do indeed all get created where they should. - * + * * Would ideally check statistical properties of distribution, but * we don't have the nerve to risk false-positives here. - * + * * @throws Exception */ static final int TRIALS = 100; - @Test public void test4() throws Exception { if (isWindows) return; - String dir5 = buildBufferDir(ROOT, 5); - String dir6 = buildBufferDir(ROOT, 6); try { - conf.set(CONTEXT, dir5 + "," + dir6); - assertTrue(localFs.mkdirs(new Path(dir5))); - assertTrue(localFs.mkdirs(new Path(dir6))); - + conf.set(CONTEXT, BUFFER_DIR[5]+","+BUFFER_DIR[6]); + assertTrue(localFs.mkdirs(BUFFER_PATH[5])); + assertTrue(localFs.mkdirs(BUFFER_PATH[6])); + int inDir5=0, inDir6=0; for(int i = 0; i < TRIALS; ++i) { File result = createTempFile(); - if(result.getPath().startsWith( - new Path(dir5, FILENAME).toUri().getPath())) { + if(result.getPath().startsWith(new File(BUFFER_DIR[5], FILENAME).getPath())) { inDir5++; - } else if(result.getPath().startsWith( - new Path(dir6, FILENAME).toUri().getPath())) { + } else if(result.getPath().startsWith(new File(BUFFER_DIR[6], FILENAME).getPath())) { inDir6++; } result.delete(); } - - assertTrue(inDir5 + inDir6 == TRIALS); - + + assertTrue( inDir5 + inDir6 == TRIALS); + } finally { rmBufferDirs(); } } - - /** Two buffer dirs. The first dir does not exist & is on a read-only disk; + + /** Two buffer dirs. The first dir does not exist & is on a read-only disk; * The second dir exists & is RW * getLocalPathForWrite with checkAccess set to false should create a parent * directory. With checkAccess true, the directory should not be created. * @throws Exception */ - @Test public void testLocalPathForWriteDirCreation() throws IOException { - String dir0 = buildBufferDir(ROOT, 0); - String dir1 = buildBufferDir(ROOT, 1); try { - conf.set(CONTEXT, dir0 + "," + dir1); - assertTrue(localFs.mkdirs(new Path(dir1))); + conf.set(CONTEXT, BUFFER_DIR[0] + "," + BUFFER_DIR[1]); + assertTrue(localFs.mkdirs(BUFFER_PATH[1])); BUFFER_ROOT.setReadOnly(); Path p1 = - dirAllocator.getLocalPathForWrite("p1/x", SMALL_FILE_SIZE, conf); + dirAllocator.getLocalPathForWrite("p1/x", SMALL_FILE_SIZE, conf); assertTrue(localFs.getFileStatus(p1.getParent()).isDirectory()); Path p2 = - dirAllocator.getLocalPathForWrite("p2/x", SMALL_FILE_SIZE, conf, - false); + dirAllocator.getLocalPathForWrite("p2/x", SMALL_FILE_SIZE, conf, + false); try { localFs.getFileStatus(p2.getParent()); } catch (Exception e) { @@ -289,26 +237,5 @@ public class TestLocalDirAllocator { rmBufferDirs(); } } - - /** Test no side effect files are left over. After creating a temp - * temp file, remove both the temp file and its parent. Verify that - * no files or directories are left over as can happen when File objects - * are mistakenly created from fully qualified path strings. - * @throws IOException - */ - @Test - public void testNoSideEffects() throws IOException { - if (isWindows) return; - String dir = buildBufferDir(ROOT, 0); - try { - conf.set(CONTEXT, dir); - File result = dirAllocator.createTmpFileForWrite(FILENAME, -1, conf); - assertTrue(result.delete()); - assertTrue(result.getParentFile().delete()); - assertFalse(new File(dir).exists()); - } finally { - Shell.execCommand(new String[]{"chmod", "u+w", BUFFER_DIR_ROOT}); - rmBufferDirs(); - } - } + } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java index 782e4e41674..3d739a07d8b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java @@ -486,9 +486,6 @@ public class TestTrash extends TestCase { conf.set(FS_TRASH_INTERVAL_KEY, "0.2"); // 12 seconds conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class); conf.set(FS_TRASH_CHECKPOINT_INTERVAL_KEY, "0.1"); // 6 seconds - FileSystem fs = FileSystem.getLocal(conf); - conf.set("fs.default.name", fs.getUri().toString()); - Trash trash = new Trash(conf); // Start Emptier in background @@ -496,6 +493,8 @@ public class TestTrash extends TestCase { Thread emptierThread = new Thread(emptier); emptierThread.start(); + FileSystem fs = FileSystem.getLocal(conf); + conf.set("fs.defaultFS", fs.getUri().toString()); FsShell shell = new FsShell(); shell.setConf(conf); shell.init(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java index aff74b573b0..07688137d5e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java @@ -70,21 +70,6 @@ public class HttpServerFunctionalTest extends Assert { return createServer(TEST, conf); } - /** - * Create but do not start the test webapp server. The test webapp dir is - * prepared/checked in advance. - * @param conf the server configuration to use - * @return the server instance - * - * @throws IOException if a problem occurs - * @throws AssertionError if a condition was not met - */ - public static HttpServer createTestServer(Configuration conf, - String[] pathSpecs) throws IOException { - prepareTestWebapp(); - return createServer(TEST, conf, pathSpecs); - } - /** * Prepare the test webapp by creating the directory from the test properties * fail if the directory cannot be created. @@ -119,18 +104,6 @@ public class HttpServerFunctionalTest extends Assert { throws IOException { return new HttpServer(webapp, "0.0.0.0", 0, true, conf); } - /** - * Create an HttpServer instance for the given webapp - * @param webapp the webapp to work with - * @param conf the configuration to use for the server - * @param pathSpecs the paths specifications the server will service - * @return the server - * @throws IOException if it could not be created - */ - public static HttpServer createServer(String webapp, Configuration conf, - String[] pathSpecs) throws IOException { - return new HttpServer(webapp, "0.0.0.0", 0, true, conf, pathSpecs); - } /** * Create and start a server with the test webapp diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java deleted file mode 100644 index 73aebea486f..00000000000 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java +++ /dev/null @@ -1,145 +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.http; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.net.URL; -import java.net.URLConnection; -import java.util.Set; -import java.util.TreeSet; - -import javax.servlet.Filter; -import javax.servlet.FilterChain; -import javax.servlet.FilterConfig; -import javax.servlet.ServletException; -import javax.servlet.ServletRequest; -import javax.servlet.ServletResponse; -import javax.servlet.http.HttpServletRequest; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.junit.Test; - -public class TestPathFilter extends HttpServerFunctionalTest { - static final Log LOG = LogFactory.getLog(HttpServer.class); - static final Set RECORDS = new TreeSet(); - - /** A very simple filter that records accessed uri's */ - static public class RecordingFilter implements Filter { - private FilterConfig filterConfig = null; - - public void init(FilterConfig filterConfig) { - this.filterConfig = filterConfig; - } - - public void destroy() { - this.filterConfig = null; - } - - public void doFilter(ServletRequest request, ServletResponse response, - FilterChain chain) throws IOException, ServletException { - if (filterConfig == null) - return; - - String uri = ((HttpServletRequest)request).getRequestURI(); - LOG.info("filtering " + uri); - RECORDS.add(uri); - chain.doFilter(request, response); - } - - /** Configuration for RecordingFilter */ - static public class Initializer extends FilterInitializer { - public Initializer() {} - - public void initFilter(FilterContainer container, Configuration conf) { - container.addFilter("recording", RecordingFilter.class.getName(), null); - } - } - } - - - /** access a url, ignoring some IOException such as the page does not exist */ - static void access(String urlstring) throws IOException { - LOG.warn("access " + urlstring); - URL url = new URL(urlstring); - - URLConnection connection = url.openConnection(); - connection.connect(); - - try { - BufferedReader in = new BufferedReader(new InputStreamReader( - connection.getInputStream())); - try { - for(; in.readLine() != null; ); - } finally { - in.close(); - } - } catch(IOException ioe) { - LOG.warn("urlstring=" + urlstring, ioe); - } - } - - @Test - public void testPathSpecFilters() throws Exception { - Configuration conf = new Configuration(); - - //start a http server with CountingFilter - conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY, - RecordingFilter.Initializer.class.getName()); - String[] pathSpecs = { "/path", "/path/*" }; - HttpServer http = createTestServer(conf, pathSpecs); - http.start(); - - final String baseURL = "/path"; - final String baseSlashURL = "/path/"; - final String addedURL = "/path/nodes"; - final String addedSlashURL = "/path/nodes/"; - final String longURL = "/path/nodes/foo/job"; - final String rootURL = "/"; - final String allURL = "/*"; - - final String[] filteredUrls = {baseURL, baseSlashURL, addedURL, - addedSlashURL, longURL}; - final String[] notFilteredUrls = {rootURL, allURL}; - - // access the urls and verify our paths specs got added to the - // filters - final String prefix = "http://localhost:" + http.getPort(); - try { - for(int i = 0; i < filteredUrls.length; i++) { - access(prefix + filteredUrls[i]); - } - for(int i = 0; i < notFilteredUrls.length; i++) { - access(prefix + notFilteredUrls[i]); - } - } finally { - http.stop(); - } - - LOG.info("RECORDS = " + RECORDS); - - //verify records - for(int i = 0; i < filteredUrls.length; i++) { - assertTrue(RECORDS.remove(filteredUrls[i])); - } - assertTrue(RECORDS.isEmpty()); - } -} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java index 7cc6f4d5213..f49d4c886ec 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java @@ -18,17 +18,13 @@ package org.apache.hadoop.net; import org.junit.Test; - import static org.junit.Assert.*; -import java.net.InetAddress; -import java.net.NetworkInterface; import java.net.Socket; import java.net.ConnectException; import java.net.SocketException; import java.net.InetSocketAddress; import java.net.UnknownHostException; -import java.util.Enumeration; import org.apache.hadoop.conf.Configuration; @@ -92,32 +88,4 @@ public class TestNetUtils { fail("NetUtils.verifyHostnames threw unexpected UnknownHostException"); } } - - /** - * Test for {@link NetUtils#isLocalAddress(java.net.InetAddress)} - */ - @Test - public void testIsLocalAddress() throws Exception { - // Test - local host is local address - assertTrue(NetUtils.isLocalAddress(InetAddress.getLocalHost())); - - // Test - all addresses bound network interface is local address - Enumeration interfaces = NetworkInterface - .getNetworkInterfaces(); - if (interfaces != null) { // Iterate through all network interfaces - while (interfaces.hasMoreElements()) { - NetworkInterface i = interfaces.nextElement(); - Enumeration addrs = i.getInetAddresses(); - if (addrs == null) { - continue; - } - // Iterate through all the addresses of a network interface - while (addrs.hasMoreElements()) { - InetAddress addr = addrs.nextElement(); - assertTrue(NetUtils.isLocalAddress(addr)); - } - } - } - assertFalse(NetUtils.isLocalAddress(InetAddress.getByName("8.8.8.8"))); - } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java index 2d699ddcf1f..7a21e4c6b87 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java @@ -25,28 +25,14 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.File; -import java.io.FileWriter; -import java.io.Writer; import java.util.Map; public class TestAuthenticationFilter extends TestCase { @SuppressWarnings("unchecked") - public void testConfiguration() throws Exception { + public void testConfiguration() { Configuration conf = new Configuration(); conf.set("hadoop.http.authentication.foo", "bar"); - - File testDir = new File(System.getProperty("test.build.data", - "target/test-dir")); - testDir.mkdirs(); - File secretFile = new File(testDir, "http-secret.txt"); - Writer writer = new FileWriter(new File(testDir, "http-secret.txt")); - writer.write("hadoop"); - writer.close(); - conf.set(AuthenticationFilterInitializer.PREFIX + - AuthenticationFilterInitializer.SIGNATURE_SECRET_FILE, - secretFile.getAbsolutePath()); FilterContainer container = Mockito.mock(FilterContainer.class); Mockito.doAnswer( diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 43c360fcb0c..459d2325d20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -16,9 +16,6 @@ Trunk (unreleased changes) HDFS-2318. Provide authentication to webhdfs using SPNEGO and delegation tokens. (szetszwo) - HDFS-2340. Support getFileBlockLocations and getDelegationToken in webhdfs. - (szetszwo) - IMPROVEMENTS HADOOP-7524 Change RPC to allow multiple protocols including multuple versions of the same protocol (sanjay Radia) @@ -38,18 +35,6 @@ Trunk (unreleased changes) not use ArrayWritable for writing non-array items. (Uma Maheswara Rao G via szetszwo) - HDFS-2351 Change Namenode and Datanode to register each of their protocols - seperately. (Sanjay Radia) - - HDFS-2356. Support case insensitive query parameter names in webhdfs. - (szetszwo) - - HDFS-2368. Move SPNEGO conf properties from hdfs-default.xml to - hdfs-site.xml. (szetszwo) - - HDFS-2355. Federation: enable using the same configuration file across - all the nodes in the cluster. (suresh) - BUG FIXES HDFS-2287. TestParallelRead has a small off-by-one bug. (todd) @@ -72,17 +57,6 @@ Trunk (unreleased changes) IOExceptions of stream closures can mask root exceptions. (Uma Maheswara Rao G via szetszwo) - HDFS-46. Change default namespace quota of root directory from - Integer.MAX_VALUE to Long.MAX_VALUE. (Uma Maheswara Rao G via szetszwo) - - HDFS-2366. Initialize WebHdfsFileSystem.ugi in object construction. - (szetszwo) - - HDFS-2373. Commands using webhdfs and hftp print unnecessary debug - info on the console with security enabled. (Arpit Gupta via suresh) - - HDFS-2361. hftp is broken, fixed username checks in JspHelper. (jitendra) - Release 0.23.0 - Unreleased INCOMPATIBLE CHANGES @@ -765,12 +739,6 @@ Release 0.23.0 - Unreleased HDFS-1217. Change some NameNode methods from public to package private. (Laxman via szetszwo) - HDFS-2332. Add test for HADOOP-7629 (using an immutable FsPermission - object as an RPC parameter fails). (todd) - - HDFS-2363. Move datanodes size printing from FSNamesystem.metasave(..) - to BlockManager. (Uma Maheswara Rao G via szetszwo) - OPTIMIZATIONS HDFS-1458. Improve checkpoint performance by avoiding unnecessary image @@ -1639,11 +1607,7 @@ Release 0.22.0 - Unreleased HDFS-2232. Generalize regular expressions in TestHDFSCLI. (Plamen Jeliazkov via shv) - HDFS-2290. Block with corrupt replica is not getting replicated. - (Benoy Antony via shv) - Release 0.21.1 - Unreleased - HDFS-1466. TestFcHdfsSymlink relies on /tmp/test not existing. (eli) HDFS-874. TestHDFSFileContextMainOperations fails on weirdly diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index c10d185acfc..f92064239c0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -244,6 +244,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max"; public static final int DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED = 500; + // HA related configuration + public static final String DFS_HA_NAMENODE_IDS_KEY = "dfs.ha.namenode.ids"; + public static final String DFS_HA_NAMENODE_IDS_DEFAULT = ""; // property for fsimage compression public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java index 34ea9e5697c..fea81f3d04e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java @@ -38,7 +38,6 @@ import java.util.Random; import java.util.StringTokenizer; import java.util.concurrent.TimeUnit; -import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; @@ -64,6 +63,7 @@ import org.apache.hadoop.security.UserGroupInformation; @InterfaceAudience.Private public class DFSUtil { + private DFSUtil() { /* Hidden constructor */ } private static final ThreadLocal RANDOM = new ThreadLocal() { @Override protected Random initialValue() { @@ -577,6 +577,17 @@ public class DFSUtil { } } + /** + * Returns the configured nameservice Id + * + * @param conf + * Configuration object to lookup the nameserviceId + * @return nameserviceId string from conf + */ + public static String getNameServiceId(Configuration conf) { + return conf.get(DFS_FEDERATION_NAMESERVICE_ID); + } + /** Return used as percentage of capacity */ public static float getPercentUsed(long used, long capacity) { return capacity <= 0 ? 100 : ((float)used * 100.0f)/(float)capacity; @@ -696,77 +707,4 @@ public class DFSUtil { // TODO:HA configuration changes pending return false; } - - /** - * Get name service Id for the {@link NameNode} based on namenode RPC address - * matching the local node address. - */ - public static String getNamenodeNameServiceId(Configuration conf) { - return getNameServiceId(conf, DFS_NAMENODE_RPC_ADDRESS_KEY); - } - - /** - * Get name service Id for the BackupNode based on backup node RPC address - * matching the local node address. - */ - public static String getBackupNameServiceId(Configuration conf) { - return getNameServiceId(conf, DFS_NAMENODE_BACKUP_ADDRESS_KEY); - } - - /** - * Get name service Id for the secondary node based on secondary http address - * matching the local node address. - */ - public static String getSecondaryNameServiceId(Configuration conf) { - return getNameServiceId(conf, DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY); - } - - /** - * Get the nameservice Id by matching the {@code addressKey} with the - * the address of the local node. - * - * If {@link DFSConfigKeys#DFS_FEDERATION_NAMESERVICE_ID} is not specifically - * configured, this method determines the nameservice Id by matching the local - * nodes address with the configured addresses. When a match is found, it - * returns the nameservice Id from the corresponding configuration key. - * - * @param conf Configuration - * @param addressKey configuration key to get the address. - * @return name service Id on success, null on failure. - * @throws HadoopIllegalArgumentException on error - */ - private static String getNameServiceId(Configuration conf, String addressKey) { - String nameserviceId = conf.get(DFS_FEDERATION_NAMESERVICE_ID); - if (nameserviceId != null) { - return nameserviceId; - } - - Collection ids = getNameServiceIds(conf); - if (ids == null || ids.size() == 0) { - // Not federation configuration, hence no nameservice Id - return null; - } - - // Match the rpc address with that of local address - int found = 0; - for (String id : ids) { - String addr = conf.get(getNameServiceIdKey(addressKey, id)); - InetSocketAddress s = NetUtils.createSocketAddr(addr); - if (NetUtils.isLocalAddress(s.getAddress())) { - nameserviceId = id; - found++; - } - } - if (found > 1) { // Only one address must match the local address - throw new HadoopIllegalArgumentException( - "Configuration has multiple RPC addresses that matches " - + "the local node's address. Please configure the system with " - + "the parameter " + DFS_FEDERATION_NAMESERVICE_ID); - } - if (found == 0) { - throw new HadoopIllegalArgumentException("Configuration address " - + addressKey + " is missing in configuration with name service Id"); - } - return nameserviceId; - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java index af3283ee718..17a09f695ec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java @@ -115,26 +115,6 @@ public class DatanodeInfo extends DatanodeID implements Node { this.location = location; this.hostName = hostName; } - - /** Constructor */ - public DatanodeInfo(final String name, final String storageID, - final int infoPort, final int ipcPort, - final long capacity, final long dfsUsed, final long remaining, - final long blockPoolUsed, final long lastUpdate, final int xceiverCount, - final String networkLocation, final String hostName, - final AdminStates adminState) { - super(name, storageID, infoPort, ipcPort); - - this.capacity = capacity; - this.dfsUsed = dfsUsed; - this.remaining = remaining; - this.blockPoolUsed = blockPoolUsed; - this.lastUpdate = lastUpdate; - this.xceiverCount = xceiverCount; - this.location = networkLocation; - this.hostName = hostName; - this.adminState = adminState; - } /** The raw capacity. */ public long getCapacity() { return capacity; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 994275aec06..682d272922b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -308,11 +308,6 @@ public class BlockManager { /** Dump meta data to out. */ public void metaSave(PrintWriter out) { assert namesystem.hasWriteLock(); - final List live = new ArrayList(); - final List dead = new ArrayList(); - datanodeManager.fetchDatanodes(live, dead, false); - out.println("Live Datanodes: " + live.size()); - out.println("Dead Datanodes: " + dead.size()); // // Dump contents of neededReplication // @@ -847,7 +842,7 @@ public class BlockManager { // Add this replica to corruptReplicas Map corruptReplicas.addToCorruptReplicasMap(storedBlock, node); - if (countNodes(storedBlock).liveReplicas() >= inode.getReplication()) { + if (countNodes(storedBlock).liveReplicas() > inode.getReplication()) { // the block is over-replicated so invalidate the replicas immediately invalidateBlock(storedBlock, node); } else if (namesystem.isPopulatingReplQueues()) { @@ -872,7 +867,7 @@ public class BlockManager { // Check how many copies we have of the block. If we have at least one // copy on a live node, then we can delete it. int count = countNodes(blk).liveReplicas(); - if (count >= 1) { + if (count > 1) { addToInvalidates(blk, dn); removeStoredBlock(blk, node); if(NameNode.stateChangeLog.isDebugEnabled()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java index 67f67c03958..e2ce26df6b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java @@ -54,13 +54,11 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer; -import org.apache.hadoop.hdfs.web.resources.DelegationParam; import org.apache.hadoop.hdfs.web.resources.UserParam; import org.apache.hadoop.http.HtmlQuoting; import org.apache.hadoop.io.Text; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.AccessControlException; -import org.apache.hadoop.security.authentication.util.KerberosName; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.security.token.Token; @@ -70,7 +68,7 @@ import org.apache.hadoop.util.VersionInfo; public class JspHelper { public static final String CURRENT_CONF = "current.conf"; final static public String WEB_UGI_PROPERTY_NAME = DFSConfigKeys.DFS_WEB_UGI_KEY; - public static final String DELEGATION_PARAMETER_NAME = DelegationParam.NAME; + public static final String DELEGATION_PARAMETER_NAME = "delegation"; public static final String NAMENODE_ADDRESS = "nnaddr"; static final String SET_DELEGATION = "&" + DELEGATION_PARAMETER_NAME + "="; @@ -553,8 +551,7 @@ public class JspHelper { DelegationTokenIdentifier id = new DelegationTokenIdentifier(); id.readFields(in); ugi = id.getUser(); - checkUsername(ugi.getShortUserName(), usernameFromQuery); - checkUsername(ugi.getShortUserName(), user); + checkUsername(ugi.getUserName(), user); ugi.addToken(token); ugi.setAuthenticationMethod(AuthenticationMethod.TOKEN); } else { @@ -563,11 +560,13 @@ public class JspHelper { "authenticated by filter"); } ugi = UserGroupInformation.createRemoteUser(user); - checkUsername(ugi.getShortUserName(), usernameFromQuery); // This is not necessarily true, could have been auth'ed by user-facing // filter ugi.setAuthenticationMethod(secureAuthMethod); } + + checkUsername(user, usernameFromQuery); + } else { // Security's not on, pull from url ugi = usernameFromQuery == null? getDefaultWebUser(conf) // not specified in request @@ -580,18 +579,10 @@ public class JspHelper { return ugi; } - /** - * Expected user name should be a short name. - */ private static void checkUsername(final String expected, final String name ) throws IOException { - if (name == null) { - return; - } - KerberosName u = new KerberosName(name); - String shortName = u.getShortName(); - if (!shortName.equals(expected)) { - throw new IOException("Usernames not matched: name=" + shortName + if (name != null && !name.equals(expected)) { + throw new IOException("Usernames not matched: name=" + name + " != expected=" + expected); } } 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 b6ec0c05b4d..73fbe50e535 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 @@ -425,7 +425,7 @@ public class DataNode extends Configured private List plugins; // For InterDataNodeProtocol - public RPC.Server ipcServer; + public Server ipcServer; private SecureResources secureResources = null; private AbstractList dataDirs; @@ -575,15 +575,11 @@ public class DataNode extends Configured private void initIpcServer(Configuration conf) throws IOException { InetSocketAddress ipcAddr = NetUtils.createSocketAddr( conf.get("dfs.datanode.ipc.address")); - - // Add all the RPC protocols that the Datanode implements - ipcServer = RPC.getServer(ClientDatanodeProtocol.class, this, ipcAddr.getHostName(), + ipcServer = RPC.getServer(DataNode.class, this, ipcAddr.getHostName(), ipcAddr.getPort(), conf.getInt(DFS_DATANODE_HANDLER_COUNT_KEY, DFS_DATANODE_HANDLER_COUNT_DEFAULT), false, conf, blockPoolTokenSecretManager); - ipcServer.addProtocol(InterDatanodeProtocol.class, this); - // set service-level authorization security policy if (conf.getBoolean( CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) { 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 index 0305024e4f0..4c5c61aac7c 100644 --- 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 @@ -50,7 +50,6 @@ import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.namenode.NameNode; -import org.apache.hadoop.hdfs.web.ParamFilter; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; import org.apache.hadoop.hdfs.web.resources.BufferSizeParam; @@ -67,11 +66,8 @@ import org.apache.hadoop.hdfs.web.resources.UriFsPathParam; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; -import com.sun.jersey.spi.container.ResourceFilters; - /** Web-hdfs DataNode implementation. */ @Path("") -@ResourceFilters(ParamFilter.class) public class DatanodeWebHdfsMethods { public static final Log LOG = LogFactory.getLog(DatanodeWebHdfsMethods.class); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java index 1e8be5b7075..d8f68a0aaab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java @@ -25,7 +25,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.hdfs.DFSConfigKeys; -import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; @@ -81,13 +80,13 @@ public class BackupNode extends NameNode { // Common NameNode methods implementation for backup node. ///////////////////////////////////////////////////// @Override // NameNode - protected InetSocketAddress getRpcServerAddress(Configuration conf) { + protected InetSocketAddress getRpcServerAddress(Configuration conf) throws IOException { String addr = conf.get(BN_ADDRESS_NAME_KEY, BN_ADDRESS_DEFAULT); return NetUtils.createSocketAddr(addr); } @Override - protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) { + protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) throws IOException { String addr = conf.get(BN_SERVICE_RPC_ADDRESS_KEY); if (addr == null || addr.isEmpty()) { return null; @@ -135,6 +134,11 @@ public class BackupNode extends NameNode { CommonConfigurationKeys.FS_TRASH_INTERVAL_DEFAULT); NamespaceInfo nsInfo = handshake(conf); super.initialize(conf); + // Backup node should never do lease recovery, + // therefore lease hard limit should never expire. + namesystem.leaseManager.setLeasePeriod( + HdfsConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE); + clusterId = nsInfo.getClusterID(); blockPoolId = nsInfo.getBlockPoolID(); @@ -368,9 +372,4 @@ public class BackupNode extends NameNode { throw new UnsupportedActionException(msg); } } - - @Override - protected String getNameServiceId(Configuration conf) { - return DFSUtil.getBackupNameServiceId(conf); - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index 654c3a231d4..4d7f2b9ca6e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -120,7 +120,7 @@ public class FSDirectory implements Closeable { this.cond = dirLock.writeLock().newCondition(); rootDir = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME, ns.createFsOwnerPermissions(new FsPermission((short)0755)), - Long.MAX_VALUE, UNKNOWN_DISK_SPACE); + Integer.MAX_VALUE, UNKNOWN_DISK_SPACE); this.fsImage = fsImage; int configuredLimit = conf.getInt( DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 9fd01446ba6..116fa4826ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -130,6 +130,7 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.util.Daemon; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.VersionInfo; import org.mortbay.util.ajax.JSON; @@ -346,30 +347,28 @@ public class FSNamesystem implements Namesystem, FSClusterStats, dir.imageLoadComplete(); } - void startSecrectManager() throws IOException { + void activateSecretManager() throws IOException { if (dtSecretManager != null) { dtSecretManager.startThreads(); } } - void stopSecretManager() { - if (dtSecretManager != null) { - dtSecretManager.stopThreads(); - } - } - - /** - * Start services common to both active and standby states - * @throws IOException + /** + * Activate FSNamesystem daemons. */ - void startCommonServices(Configuration conf) throws IOException { + void activate(Configuration conf) throws IOException { this.registerMBean(); // register the MBean for the FSNamesystemState + writeLock(); try { nnResourceChecker = new NameNodeResourceChecker(conf); checkAvailableResources(); + setBlockTotal(); blockManager.activate(conf); + + this.lmthread = new Daemon(leaseManager.new Monitor()); + lmthread.start(); this.nnrmthread = new Daemon(new NameNodeResourceMonitor()); nnrmthread.start(); } finally { @@ -379,70 +378,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, registerMXBean(); DefaultMetricsSystem.instance().register(this); } - - /** - * Stop services common to both active and standby states - * @throws IOException - */ - void stopCommonServices() { - writeLock(); - try { - if (blockManager != null) blockManager.close(); - if (nnrmthread != null) nnrmthread.interrupt(); - } finally { - writeUnlock(); - } - } - - /** - * Start services required in active state - * @throws IOException - */ - void startActiveServices() throws IOException { - LOG.info("Starting services required for active state"); - writeLock(); - try { - startSecrectManager(); - lmthread = new Daemon(leaseManager.new Monitor()); - lmthread.start(); - } finally { - writeUnlock(); - } - } - - /** - * Start services required in active state - * @throws InterruptedException - */ - void stopActiveServices() { - LOG.info("Stopping services started for active state"); - writeLock(); - try { - stopSecretManager(); - if (lmthread != null) { - try { - lmthread.interrupt(); - lmthread.join(3000); - } catch (InterruptedException ie) { - LOG.warn("Encountered exception ", ie); - } - lmthread = null; - } - } finally { - writeUnlock(); - } - } - - /** Start services required in standby state */ - void startStandbyServices() { - LOG.info("Starting services required for standby state"); - } - /** Stop services required in standby state */ - void stopStandbyServices() { - LOG.info("Stopping services started for standby state"); - } - public static Collection getNamespaceDirs(Configuration conf) { return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY); } @@ -566,7 +502,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, } /** - * Version of @see #getNamespaceInfo() that is not protected by a lock. + * Version of {@see #getNamespaceInfo()} that is not protected by a lock. */ NamespaceInfo unprotectedGetNamespaceInfo() { return new NamespaceInfo(dir.fsImage.getStorage().getNamespaceID(), @@ -583,16 +519,23 @@ public class FSNamesystem implements Namesystem, FSClusterStats, void close() { fsRunning = false; try { - stopCommonServices(); + if (blockManager != null) blockManager.close(); if (smmthread != null) smmthread.interrupt(); + if (dtSecretManager != null) dtSecretManager.stopThreads(); + if (nnrmthread != null) nnrmthread.interrupt(); + } catch (Exception e) { + LOG.warn("Exception shutting down FSNamesystem", e); } finally { // using finally to ensure we also wait for lease daemon try { - stopActiveServices(); - stopStandbyServices(); + if (lmthread != null) { + lmthread.interrupt(); + lmthread.join(3000); + } if (dir != null) { dir.close(); } + } catch (InterruptedException ie) { } catch (IOException ie) { LOG.error("Error closing FSDirectory", ie); IOUtils.cleanup(LOG, dir); @@ -621,6 +564,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats, out.println(totalInodes + " files and directories, " + totalBlocks + " blocks = " + (totalInodes + totalBlocks) + " total"); + final List live = new ArrayList(); + final List dead = new ArrayList(); + blockManager.getDatanodeManager().fetchDatanodes(live, dead, false); + out.println("Live Datanodes: "+live.size()); + out.println("Dead Datanodes: "+dead.size()); blockManager.metaSave(out); out.flush(); @@ -1443,7 +1391,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, try { lb = startFileInternal(src, null, holder, clientMachine, EnumSet.of(CreateFlag.APPEND), - false, blockManager.maxReplication, 0); + false, blockManager.maxReplication, (long)0); } finally { writeUnlock(); } @@ -1526,7 +1474,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, fileLength = pendingFile.computeContentSummary().getLength(); blockSize = pendingFile.getPreferredBlockSize(); clientNode = pendingFile.getClientNode(); - replication = pendingFile.getReplication(); + replication = (int)pendingFile.getReplication(); } finally { writeUnlock(); } @@ -2321,7 +2269,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, } Lease reassignLeaseInternal(Lease lease, String src, String newHolder, - INodeFileUnderConstruction pendingFile) { + INodeFileUnderConstruction pendingFile) throws IOException { assert hasWriteLock(); pendingFile.setClientName(newHolder); return leaseManager.reassignLease(lease, src, newHolder); @@ -2926,9 +2874,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats, * @return true if in safe mode */ private synchronized boolean isOn() { - assert isConsistent() : " SafeMode: Inconsistent filesystem state: " - + "Total num of blocks, active blocks, or " - + "total safe blocks don't match."; + try { + assert isConsistent() : " SafeMode: Inconsistent filesystem state: " + + "Total num of blocks, active blocks, or " + + "total safe blocks don't match."; + } catch(IOException e) { + System.err.print(StringUtils.stringifyException(e)); + } return this.reached >= 0; } @@ -3082,7 +3034,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, this.blockTotal = total; this.blockThreshold = (int) (blockTotal * threshold); this.blockReplQueueThreshold = - (int) (blockTotal * replQueueThreshold); + (int) (((double) blockTotal) * replQueueThreshold); checkMode(); } @@ -3092,7 +3044,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, * @param replication current replication */ private synchronized void incrementSafeBlockCount(short replication) { - if (replication == safeReplication) + if ((int)replication == safeReplication) this.blockSafe++; checkMode(); } @@ -3225,7 +3177,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, * Checks consistency of the class state. * This is costly and currently called only in assert. */ - private boolean isConsistent() { + private boolean isConsistent() throws IOException { if (blockTotal == -1 && blockSafe == -1) { return true; // manual safe mode } 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 9f41ad54b12..6db96924c24 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 @@ -27,7 +27,6 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ha.HealthCheckFailedException; @@ -38,13 +37,15 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Trash; import static org.apache.hadoop.hdfs.DFSConfigKeys.*; import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.HDFSPolicyProvider; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; +import org.apache.hadoop.hdfs.server.common.IncorrectVersionException; import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState; -import org.apache.hadoop.hdfs.server.namenode.ha.HAContext; import org.apache.hadoop.hdfs.server.namenode.ha.HAState; import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState; import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; @@ -53,6 +54,9 @@ import org.apache.hadoop.hdfs.server.protocol.JournalProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; +import org.apache.hadoop.hdfs.server.protocol.NodeRegistration; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.RefreshUserMappingsProtocol; @@ -167,18 +171,19 @@ public class NameNode { } } + + public static final int DEFAULT_PORT = 8020; + public static final Log LOG = LogFactory.getLog(NameNode.class.getName()); public static final Log stateChangeLog = LogFactory.getLog("org.apache.hadoop.hdfs.StateChange"); public static final HAState ACTIVE_STATE = new ActiveState(); public static final HAState STANDBY_STATE = new StandbyState(); protected FSNamesystem namesystem; - protected final Configuration conf; protected NamenodeRole role; private HAState state; private final boolean haEnabled; - private final HAContext haContext; /** httpServer */ @@ -307,11 +312,12 @@ public class NameNode { * Given a configuration get the address of the service rpc server * If the service rpc is not configured returns null */ - protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) { + protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) + throws IOException { return NameNode.getServiceAddress(conf, false); } - protected InetSocketAddress getRpcServerAddress(Configuration conf) { + protected InetSocketAddress getRpcServerAddress(Configuration conf) throws IOException { return getAddress(conf); } @@ -374,6 +380,7 @@ public class NameNode { * @param conf the configuration */ protected void initialize(Configuration conf) throws IOException { + initializeGenericKeys(conf); UserGroupInformation.setConfiguration(conf); loginAsNameNodeUser(conf); @@ -389,7 +396,7 @@ public class NameNode { throw e; } - startCommonServices(conf); + activate(conf); } /** @@ -423,10 +430,19 @@ public class NameNode { } } - /** Start the services common to active and standby states */ - private void startCommonServices(Configuration conf) throws IOException { - namesystem.startCommonServices(conf); + /** + * Activate name-node servers and threads. + */ + void activate(Configuration conf) throws IOException { + if ((isRole(NamenodeRole.NAMENODE)) + && (UserGroupInformation.isSecurityEnabled())) { + namesystem.activateSecretManager(); + } + namesystem.activate(conf); + startHttpServer(conf); rpcServer.start(); + startTrashEmptier(conf); + plugins = conf.getInstances(DFS_NAMENODE_PLUGINS_KEY, ServicePlugin.class); for (ServicePlugin p: plugins) { @@ -436,29 +452,13 @@ public class NameNode { LOG.warn("ServicePlugin " + p + " could not be started", t); } } + LOG.info(getRole() + " up at: " + rpcServer.getRpcAddress()); if (rpcServer.getServiceRpcAddress() != null) { - LOG.info(getRole() + " service server is up at: " - + rpcServer.getServiceRpcAddress()); + LOG.info(getRole() + " service server is up at: " + rpcServer.getServiceRpcAddress()); } - startHttpServer(conf); } - - private void stopCommonServices() { - if(namesystem != null) namesystem.close(); - if(rpcServer != null) rpcServer.stop(); - if (plugins != null) { - for (ServicePlugin p : plugins) { - try { - p.stop(); - } catch (Throwable t) { - LOG.warn("ServicePlugin " + p + " could not be stopped", t); - } - } - } - stopHttpServer(); - } - + private void startTrashEmptier(Configuration conf) throws IOException { long trashInterval = conf.getLong(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, @@ -470,26 +470,11 @@ public class NameNode { this.emptier.start(); } - private void stopTrashEmptier() { - if (this.emptier != null) { - emptier.interrupt(); - emptier = null; - } - } - private void startHttpServer(final Configuration conf) throws IOException { httpServer = new NameNodeHttpServer(conf, this, getHttpServerAddress(conf)); httpServer.start(); setHttpServerAddress(conf); } - - private void stopHttpServer() { - try { - if (httpServer != null) httpServer.stop(); - } catch (Exception e) { - LOG.error("Exception while stopping httpserver", e); - } - } /** * Start NameNode. @@ -516,36 +501,22 @@ public class NameNode { * zero in the conf. * * @param conf confirguration - * @throws IOException on error + * @throws IOException */ public NameNode(Configuration conf) throws IOException { this(conf, NamenodeRole.NAMENODE); } protected NameNode(Configuration conf, NamenodeRole role) - throws IOException { - this.conf = conf; + throws IOException { this.role = role; this.haEnabled = DFSUtil.isHAEnabled(conf); - this.haContext = new NameNodeHAContext(); + this.state = !haEnabled ? ACTIVE_STATE : STANDBY_STATE; try { - initializeGenericKeys(conf, getNameServiceId(conf)); initialize(conf); - if (!haEnabled) { - state = ACTIVE_STATE; - } else { - state = STANDBY_STATE;; - } - state.enterState(haContext); } catch (IOException e) { this.stop(); throw e; - } catch (ServiceFailedException e) { - this.stop(); - throw new IOException("Service failed to start", e); - } catch (HadoopIllegalArgumentException e) { - this.stop(); - throw e; } } @@ -557,7 +528,6 @@ public class NameNode { try { this.rpcServer.join(); } catch (InterruptedException ie) { - LOG.info("Caught interrupted exception " + ie); } } @@ -570,12 +540,23 @@ public class NameNode { return; stopRequested = true; } - try { - state.exitState(haContext); - } catch (ServiceFailedException e) { - LOG.info("Encountered exception while exiting state " + e); + if (plugins != null) { + for (ServicePlugin p : plugins) { + try { + p.stop(); + } catch (Throwable t) { + LOG.warn("ServicePlugin " + p + " could not be stopped", t); + } + } } - stopCommonServices(); + try { + if (httpServer != null) httpServer.stop(); + } catch (Exception e) { + LOG.error("Exception while stopping httpserver", e); + } + if(namesystem != null) namesystem.close(); + if(emptier != null) emptier.interrupt(); + if(rpcServer != null) rpcServer.stop(); if (metrics != null) { metrics.shutdown(); } @@ -840,16 +821,16 @@ public class NameNode { * @param conf * Configuration object to lookup specific key and to set the value * to the key passed. Note the conf object is modified - * @param nameserviceId name service Id * @see DFSUtil#setGenericConf(Configuration, String, String...) */ - public static void initializeGenericKeys(Configuration conf, String - nameserviceId) { + public static void initializeGenericKeys(Configuration conf) { + final String nameserviceId = DFSUtil.getNameServiceId(conf); if ((nameserviceId == null) || nameserviceId.isEmpty()) { return; } DFSUtil.setGenericConf(conf, nameserviceId, NAMESERVICE_SPECIFIC_KEYS); + if (conf.get(DFS_NAMENODE_RPC_ADDRESS_KEY) != null) { URI defaultUri = URI.create(HdfsConstants.HDFS_URI_SCHEME + "://" + conf.get(DFS_NAMENODE_RPC_ADDRESS_KEY)); @@ -857,14 +838,6 @@ public class NameNode { } } - /** - * Get the name service Id for the node - * @return name service Id or null if federation is not configured - */ - protected String getNameServiceId(Configuration conf) { - return DFSUtil.getNamenodeNameServiceId(conf); - } - /** */ public static void main(String argv[]) throws Exception { @@ -891,56 +864,27 @@ public class NameNode { if (!haEnabled) { throw new ServiceFailedException("HA for namenode is not enabled"); } - state.setState(haContext, ACTIVE_STATE); + state.setState(this, ACTIVE_STATE); } synchronized void transitionToStandby() throws ServiceFailedException { if (!haEnabled) { throw new ServiceFailedException("HA for namenode is not enabled"); } - state.setState(haContext, STANDBY_STATE); + state.setState(this, STANDBY_STATE); } /** Check if an operation of given category is allowed */ protected synchronized void checkOperation(final OperationCategory op) throws UnsupportedActionException { - state.checkOperation(haContext, op); + state.checkOperation(this, op); } - /** - * Class used as expose {@link NameNode} as context to {@link HAState} - */ - private class NameNodeHAContext implements HAContext { - @Override - public void setState(HAState s) { - state = s; - } - - @Override - public HAState getState() { - return state; - } - - @Override - public void startActiveServices() throws IOException { - namesystem.startActiveServices(); - startTrashEmptier(conf); - } - - @Override - public void stopActiveServices() throws IOException { - namesystem.stopActiveServices(); - stopTrashEmptier(); - } - - @Override - public void startStandbyServices() throws IOException { - // TODO:HA Start reading editlog from active - } - - @Override - public void stopStandbyServices() throws IOException { - // TODO:HA Stop reading editlog from active - } + public synchronized HAState getState() { + return state; + } + + public synchronized void setState(final HAState s) { + state = s; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java index 3e4eaf5f9dc..7fdf3e60d39 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java @@ -66,7 +66,6 @@ import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics; -import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; @@ -146,17 +145,10 @@ class NameNodeRpcServer implements NamenodeProtocols { serviceRpcServer = null; serviceRPCAddress = null; } - // Add all the RPC protocols that the namenode implements - this.server = RPC.getServer(ClientProtocol.class, this, + this.server = RPC.getServer(NamenodeProtocols.class, this, socAddr.getHostName(), socAddr.getPort(), handlerCount, false, conf, namesystem.getDelegationTokenSecretManager()); - this.server.addProtocol(DatanodeProtocol.class, this); - this.server.addProtocol(NamenodeProtocol.class, this); - this.server.addProtocol(RefreshAuthorizationPolicyProtocol.class, this); - this.server.addProtocol(RefreshUserMappingsProtocol.class, this); - this.server.addProtocol(GetUserMappingsProtocol.class, this); - // set service-level authorization security policy if (serviceAuthEnabled = @@ -979,11 +971,8 @@ class NameNodeRpcServer implements NamenodeProtocols { } private static String getClientMachine() { - String clientMachine = NamenodeWebHdfsMethods.getRemoteAddress(); - if (clientMachine == null) { //not a web client - clientMachine = Server.getRemoteAddress(); - } - if (clientMachine == null) { //not a RPC client + String clientMachine = Server.getRemoteAddress(); + if (clientMachine == null) { clientMachine = ""; } return clientMachine; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java index d403629146f..9c5ef6f2c36 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java @@ -38,12 +38,10 @@ import org.apache.commons.cli.ParseException; import org.apache.commons.cli.PosixParser; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import static org.apache.hadoop.hdfs.DFSConfigKeys.*; - import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator; import org.apache.hadoop.hdfs.HdfsConfiguration; @@ -175,17 +173,12 @@ public class SecondaryNameNode implements Runnable { public SecondaryNameNode(Configuration conf, CommandLineOpts commandLineOpts) throws IOException { try { - NameNode.initializeGenericKeys(conf, - DFSUtil.getSecondaryNameServiceId(conf)); + NameNode.initializeGenericKeys(conf); initialize(conf, commandLineOpts); } catch(IOException e) { shutdown(); LOG.fatal("Failed to start secondary namenode. ", e); throw e; - } catch(HadoopIllegalArgumentException e) { - shutdown(); - LOG.fatal("Failed to start secondary namenode. ", e); - throw e; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java index 76e0f6a7151..1cf24f7f23a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.hdfs.server.namenode.ha; -import java.io.IOException; - import org.apache.hadoop.ha.ServiceFailedException; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; @@ -35,35 +33,27 @@ public class ActiveState extends HAState { } @Override - public void checkOperation(HAContext context, OperationCategory op) + public void checkOperation(NameNode nn, OperationCategory op) throws UnsupportedActionException { return; // Other than journal all operations are allowed in active state } @Override - public void setState(HAContext context, HAState s) throws ServiceFailedException { + public void setState(NameNode nn, HAState s) throws ServiceFailedException { if (s == NameNode.STANDBY_STATE) { - setStateInternal(context, s); + setStateInternal(nn, s); return; } - super.setState(context, s); + super.setState(nn, s); } @Override - public void enterState(HAContext context) throws ServiceFailedException { - try { - context.startActiveServices(); - } catch (IOException e) { - throw new ServiceFailedException("Failed to start active services", e); - } + protected void enterState(NameNode nn) throws ServiceFailedException { + // TODO:HA } @Override - public void exitState(HAContext context) throws ServiceFailedException { - try { - context.stopActiveServices(); - } catch (IOException e) { - throw new ServiceFailedException("Failed to stop active services", e); - } + protected void exitState(NameNode nn) throws ServiceFailedException { + // TODO:HA } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java index 6ee516c4cab..1828f9c83db 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode.ha; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.ha.ServiceFailedException; +import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; @@ -43,38 +44,38 @@ abstract public class HAState { * @param s new state * @throws ServiceFailedException on failure to transition to new state. */ - protected final void setStateInternal(final HAContext context, final HAState s) + protected final void setStateInternal(final NameNode nn, final HAState s) throws ServiceFailedException { - exitState(context); - context.setState(s); - s.enterState(context); + exitState(nn); + nn.setState(s); + s.enterState(nn); } /** * Method to be overridden by subclasses to perform steps necessary for * entering a state. - * @param context HA context + * @param nn Namenode * @throws ServiceFailedException on failure to enter the state. */ - public abstract void enterState(final HAContext context) + protected abstract void enterState(final NameNode nn) throws ServiceFailedException; /** * Method to be overridden by subclasses to perform steps necessary for * exiting a state. - * @param context HA context + * @param nn Namenode * @throws ServiceFailedException on failure to enter the state. */ - public abstract void exitState(final HAContext context) + protected abstract void exitState(final NameNode nn) throws ServiceFailedException; /** * Move from the existing state to a new state - * @param context HA context + * @param nn Namenode * @param s new state * @throws ServiceFailedException on failure to transition to new state. */ - public void setState(HAContext context, HAState s) throws ServiceFailedException { + public void setState(NameNode nn, HAState s) throws ServiceFailedException { if (this == s) { // Aleady in the new state return; } @@ -84,15 +85,15 @@ abstract public class HAState { /** * Check if an operation is supported in a given state. - * @param context HA context + * @param nn Namenode * @param op Type of the operation. * @throws UnsupportedActionException if a given type of operation is not * supported in this state. */ - public void checkOperation(final HAContext context, final OperationCategory op) + public void checkOperation(final NameNode nn, final OperationCategory op) throws UnsupportedActionException { String msg = "Operation category " + op + " is not supported in state " - + context.getState(); + + nn.getState(); throw new UnsupportedActionException(msg); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java index e0dc9af5919..b63866dc713 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.hdfs.server.namenode.ha; -import java.io.IOException; - import org.apache.hadoop.ha.ServiceFailedException; import org.apache.hadoop.hdfs.server.namenode.NameNode; @@ -39,30 +37,22 @@ public class StandbyState extends HAState { } @Override - public void setState(HAContext context, HAState s) throws ServiceFailedException { + public void setState(NameNode nn, HAState s) throws ServiceFailedException { if (s == NameNode.ACTIVE_STATE) { - setStateInternal(context, s); + setStateInternal(nn, s); return; } - super.setState(context, s); + super.setState(nn, s); } @Override - public void enterState(HAContext context) throws ServiceFailedException { - try { - context.startStandbyServices(); - } catch (IOException e) { - throw new ServiceFailedException("Failed to start standby services", e); - } + protected void enterState(NameNode nn) throws ServiceFailedException { + // TODO:HA } @Override - public void exitState(HAContext context) throws ServiceFailedException { - try { - context.stopStandbyServices(); - } catch (IOException e) { - throw new ServiceFailedException("Failed to stop standby services", e); - } + protected void exitState(NameNode nn) throws ServiceFailedException { + // TODO:HA } } 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 index 2dd1db33410..948466f638f 100644 --- 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 @@ -57,7 +57,6 @@ 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.ParamFilter; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.hdfs.web.resources.AccessTimeParam; import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; @@ -79,7 +78,6 @@ 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.RenewerParam; import org.apache.hadoop.hdfs.web.resources.ReplicationParam; import org.apache.hadoop.hdfs.web.resources.UriFsPathParam; import org.apache.hadoop.hdfs.web.resources.UserParam; @@ -91,20 +89,10 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; -import com.sun.jersey.spi.container.ResourceFilters; - /** Web-hdfs NameNode implementation. */ @Path("") -@ResourceFilters(ParamFilter.class) public class NamenodeWebHdfsMethods { - public static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class); - - private static final ThreadLocal REMOTE_ADDRESS = new ThreadLocal(); - - /** @return the remote client address. */ - public static String getRemoteAddress() { - return REMOTE_ADDRESS.get(); - } + private static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class); private @Context ServletContext context; private @Context HttpServletRequest request; @@ -227,8 +215,6 @@ public class NamenodeWebHdfsMethods { return ugi.doAs(new PrivilegedExceptionAction() { @Override public Response run() throws IOException, URISyntaxException { - REMOTE_ADDRESS.set(request.getRemoteAddr()); - try { final String fullpath = path.getAbsolutePath(); final NameNode namenode = (NameNode)context.getAttribute("name.node"); @@ -286,10 +272,6 @@ public class NamenodeWebHdfsMethods { default: throw new UnsupportedOperationException(op + " is not supported"); } - - } finally { - REMOTE_ADDRESS.set(null); - } } }); } @@ -319,8 +301,6 @@ public class NamenodeWebHdfsMethods { return ugi.doAs(new PrivilegedExceptionAction() { @Override public Response run() throws IOException, URISyntaxException { - REMOTE_ADDRESS.set(request.getRemoteAddr()); - try { final String fullpath = path.getAbsolutePath(); final NameNode namenode = (NameNode)context.getAttribute("name.node"); @@ -335,10 +315,6 @@ public class NamenodeWebHdfsMethods { default: throw new UnsupportedOperationException(op + " is not supported"); } - - } finally { - REMOTE_ADDRESS.set(null); - } } }); } @@ -359,12 +335,10 @@ public class NamenodeWebHdfsMethods { final OffsetParam offset, @QueryParam(LengthParam.NAME) @DefaultValue(LengthParam.DEFAULT) final LengthParam length, - @QueryParam(RenewerParam.NAME) @DefaultValue(RenewerParam.DEFAULT) - final RenewerParam renewer, @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) final BufferSizeParam bufferSize ) throws IOException, URISyntaxException, InterruptedException { - return get(ugi, delegation, ROOT, op, offset, length, renewer, bufferSize); + return get(ugi, delegation, ROOT, op, offset, length, bufferSize); } /** Handle HTTP GET request. */ @@ -382,23 +356,19 @@ public class NamenodeWebHdfsMethods { final OffsetParam offset, @QueryParam(LengthParam.NAME) @DefaultValue(LengthParam.DEFAULT) final LengthParam length, - @QueryParam(RenewerParam.NAME) @DefaultValue(RenewerParam.DEFAULT) - final RenewerParam renewer, @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) final BufferSizeParam bufferSize ) throws IOException, URISyntaxException, InterruptedException { if (LOG.isTraceEnabled()) { LOG.trace(op + ": " + path + ", ugi=" + ugi - + Param.toSortedString(", ", offset, length, renewer, bufferSize)); + + Param.toSortedString(", ", offset, length, bufferSize)); } return ugi.doAs(new PrivilegedExceptionAction() { @Override public Response run() throws IOException, URISyntaxException { - REMOTE_ADDRESS.set(request.getRemoteAddr()); - try { final NameNode namenode = (NameNode)context.getAttribute("name.node"); final String fullpath = path.getAbsolutePath(); @@ -411,15 +381,6 @@ public class NamenodeWebHdfsMethods { op.getValue(), offset.getValue(), offset, length, bufferSize); return Response.temporaryRedirect(uri).build(); } - case GETFILEBLOCKLOCATIONS: - { - final long offsetValue = offset.getValue(); - final Long lengthValue = length.getValue(); - final LocatedBlocks locatedblocks = np.getBlockLocations(fullpath, - offsetValue, lengthValue != null? lengthValue: offsetValue + 1); - final String js = JsonUtil.toJsonString(locatedblocks); - return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); - } case GETFILESTATUS: { final HdfsFileStatus status = np.getFileInfo(fullpath); @@ -431,20 +392,9 @@ public class NamenodeWebHdfsMethods { final StreamingOutput streaming = getListingStream(np, fullpath); return Response.ok(streaming).type(MediaType.APPLICATION_JSON).build(); } - case GETDELEGATIONTOKEN: - { - final Token token = generateDelegationToken( - namenode, ugi, renewer.getValue()); - final String js = JsonUtil.toJsonString(token); - return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); - } default: throw new UnsupportedOperationException(op + " is not supported"); } - - } finally { - REMOTE_ADDRESS.set(null); - } } }); } @@ -512,9 +462,6 @@ public class NamenodeWebHdfsMethods { return ugi.doAs(new PrivilegedExceptionAction() { @Override public Response run() throws IOException { - REMOTE_ADDRESS.set(request.getRemoteAddr()); - try { - final NameNode namenode = (NameNode)context.getAttribute("name.node"); final String fullpath = path.getAbsolutePath(); @@ -528,10 +475,6 @@ public class NamenodeWebHdfsMethods { default: throw new UnsupportedOperationException(op + " is not supported"); } - - } finally { - REMOTE_ADDRESS.set(null); - } } }); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java index 1e853933433..d085534e110 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java @@ -149,9 +149,7 @@ public class DelegationTokenFetcher { DataInputStream in = new DataInputStream( new ByteArrayInputStream(token.getIdentifier())); id.readFields(in); - if(LOG.isDebugEnabled()) { - LOG.debug("Token (" + id + ") for " + token.getService()); - } + System.out.println("Token (" + id + ") for " + token.getService()); } return null; } @@ -162,28 +160,22 @@ public class DelegationTokenFetcher { for (Token token : readTokens(tokenFile, conf)) { result = renewDelegationToken(webUrl, (Token) token); - if(LOG.isDebugEnabled()) { - LOG.debug("Renewed token via " + webUrl + " for " - + token.getService() + " until: " + new Date(result)); - } + System.out.println("Renewed token via " + webUrl + " for " + + token.getService() + " until: " + new Date(result)); } } else if (cancel) { for (Token token : readTokens(tokenFile, conf)) { cancelDelegationToken(webUrl, (Token) token); - if(LOG.isDebugEnabled()) { - LOG.debug("Cancelled token via " + webUrl + " for " - + token.getService()); - } + System.out.println("Cancelled token via " + webUrl + " for " + + token.getService()); } } else { Credentials creds = getDTfromRemote(webUrl, renewer); creds.writeTokenStorageFile(tokenFile, conf); for (Token token : creds.getAllTokens()) { - if(LOG.isDebugEnabled()) { - LOG.debug("Fetched token via " + webUrl + " for " - + token.getService() + " into " + tokenFile); - } + System.out.println("Fetched token via " + webUrl + " for " + + token.getService() + " into " + tokenFile); } } } else { @@ -192,30 +184,24 @@ public class DelegationTokenFetcher { for (Token token : readTokens(tokenFile, conf)) { ((DistributedFileSystem) fs) .cancelDelegationToken((Token) token); - if(LOG.isDebugEnabled()) { - LOG.debug("Cancelled token for " - + token.getService()); - } + System.out.println("Cancelled token for " + + token.getService()); } } else if (renew) { long result; for (Token token : readTokens(tokenFile, conf)) { result = ((DistributedFileSystem) fs) .renewDelegationToken((Token) token); - if(LOG.isDebugEnabled()) { - LOG.debug("Renewed token for " + token.getService() - + " until: " + new Date(result)); - } + System.out.println("Renewed token for " + token.getService() + + " until: " + new Date(result)); } } else { Token token = fs.getDelegationToken(renewer); Credentials cred = new Credentials(); cred.addToken(token.getService(), token); cred.writeTokenStorageFile(tokenFile, conf); - if(LOG.isDebugEnabled()) { - LOG.debug("Fetched token for " + token.getService() - + " into " + tokenFile); - } + System.out.println("Fetched token for " + token.getService() + + " into " + tokenFile); } } return null; @@ -235,11 +221,6 @@ public class DelegationTokenFetcher { } else { url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC); } - - if(LOG.isDebugEnabled()) { - LOG.debug("Retrieving token from: " + url); - } - URL remoteURL = new URL(url.toString()); SecurityUtil.fetchServiceTicket(remoteURL); URLConnection connection = remoteURL.openConnection(); 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 index 314d53b38f2..1c18dc334e2 100644 --- 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 @@ -17,31 +17,19 @@ */ package org.apache.hadoop.hdfs.web; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; 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.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; -import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; -import org.apache.hadoop.hdfs.protocol.LocatedBlock; -import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; -import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; import org.mortbay.util.ajax.JSON; /** JSON Utilities */ public class JsonUtil { - private static class ThreadLocalMap extends ThreadLocal> { + private static final ThreadLocal> jsonMap + = new ThreadLocal>() { @Override protected Map initialValue() { return new TreeMap(); @@ -53,54 +41,7 @@ public class JsonUtil { m.clear(); return m; } - } - - private static final ThreadLocalMap jsonMap = new ThreadLocalMap(); - private static final ThreadLocalMap tokenMap = new ThreadLocalMap(); - private static final ThreadLocalMap datanodeInfoMap = new ThreadLocalMap(); - private static final ThreadLocalMap extendedBlockMap = new ThreadLocalMap(); - private static final ThreadLocalMap locatedBlockMap = new ThreadLocalMap(); - - private static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {}; - - /** Convert a token object to a Json string. */ - public static String toJsonString(final Token token - ) throws IOException { - if (token == null) { - return null; - } - - final Map m = tokenMap.get(); - m.put("urlString", token.encodeToUrlString()); - return JSON.toString(m); - } - - /** Convert a Json map to a Token. */ - public static Token toToken( - final Map m) throws IOException { - if (m == null) { - return null; - } - - final Token token - = new Token(); - token.decodeFromUrlString((String)m.get("urlString")); - return token; - } - - /** Convert a Json map to a Token of DelegationTokenIdentifier. */ - @SuppressWarnings("unchecked") - public static Token toDelegationToken( - final Map m) throws IOException { - return (Token)toToken(m); - } - - /** Convert a Json map to a Token of BlockTokenIdentifier. */ - @SuppressWarnings("unchecked") - public static Token toBlockToken( - final Map m) throws IOException { - return (Token)toToken(m); - } + }; /** Convert an exception object to a Json string. */ public static String toJsonString(final Exception e) { @@ -136,10 +77,11 @@ public class JsonUtil { /** Convert a HdfsFileStatus object to a Json string. */ public static String toJsonString(final HdfsFileStatus status) { + final Map m = jsonMap.get(); if (status == null) { - return null; + m.put("isNull", true); } else { - final Map m = jsonMap.get(); + m.put("isNull", false); m.put("localName", status.getLocalName()); m.put("isDir", status.isDir()); m.put("isSymlink", status.isSymlink()); @@ -155,8 +97,8 @@ public class JsonUtil { m.put("modificationTime", status.getModificationTime()); m.put("blockSize", status.getBlockSize()); m.put("replication", status.getReplication()); - return JSON.toString(m); } + return JSON.toString(m); } @SuppressWarnings("unchecked") @@ -164,9 +106,9 @@ public class JsonUtil { return (Map) JSON.parse(jsonString); } - /** Convert a Json map to a HdfsFileStatus object. */ + /** Convert a Json string to a HdfsFileStatus object. */ public static HdfsFileStatus toFileStatus(final Map m) { - if (m == null) { + if ((Boolean)m.get("isNull")) { return null; } @@ -188,214 +130,4 @@ public class JsonUtil { permission, owner, group, symlink, DFSUtil.string2Bytes(localName)); } - - /** Convert a LocatedBlock to a Json string. */ - public static String toJsonString(final ExtendedBlock extendedblock) { - if (extendedblock == null) { - return null; - } - - final Map m = extendedBlockMap.get(); - m.put("blockPoolId", extendedblock.getBlockPoolId()); - m.put("blockId", extendedblock.getBlockId()); - m.put("numBytes", extendedblock.getNumBytes()); - m.put("generationStamp", extendedblock.getGenerationStamp()); - return JSON.toString(m); - } - - /** Convert a Json map to an ExtendedBlock object. */ - public static ExtendedBlock toExtendedBlock(final Map m) { - if (m == null) { - return null; - } - - final String blockPoolId = (String)m.get("blockPoolId"); - final long blockId = (Long)m.get("blockId"); - final long numBytes = (Long)m.get("numBytes"); - final long generationStamp = (Long)m.get("generationStamp"); - return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp); - } - - /** Convert a DatanodeInfo to a Json string. */ - public static String toJsonString(final DatanodeInfo datanodeinfo) { - if (datanodeinfo == null) { - return null; - } - - final Map m = datanodeInfoMap.get(); - m.put("name", datanodeinfo.getName()); - m.put("storageID", datanodeinfo.getStorageID()); - m.put("infoPort", datanodeinfo.getInfoPort()); - - m.put("ipcPort", datanodeinfo.getIpcPort()); - - m.put("capacity", datanodeinfo.getCapacity()); - m.put("dfsUsed", datanodeinfo.getDfsUsed()); - m.put("remaining", datanodeinfo.getRemaining()); - m.put("blockPoolUsed", datanodeinfo.getBlockPoolUsed()); - m.put("lastUpdate", datanodeinfo.getLastUpdate()); - m.put("xceiverCount", datanodeinfo.getXceiverCount()); - m.put("networkLocation", datanodeinfo.getNetworkLocation()); - m.put("hostName", datanodeinfo.getHostName()); - m.put("adminState", datanodeinfo.getAdminState().name()); - return JSON.toString(m); - } - - /** Convert a Json map to an DatanodeInfo object. */ - public static DatanodeInfo toDatanodeInfo(final Map m) { - if (m == null) { - return null; - } - - return new DatanodeInfo( - (String)m.get("name"), - (String)m.get("storageID"), - (int)(long)(Long)m.get("infoPort"), - (int)(long)(Long)m.get("ipcPort"), - - (Long)m.get("capacity"), - (Long)m.get("dfsUsed"), - (Long)m.get("remaining"), - (Long)m.get("blockPoolUsed"), - (Long)m.get("lastUpdate"), - (int)(long)(Long)m.get("xceiverCount"), - (String)m.get("networkLocation"), - (String)m.get("hostName"), - AdminStates.valueOf((String)m.get("adminState"))); - } - - /** Convert a DatanodeInfo[] to a Json string. */ - public static String toJsonString(final DatanodeInfo[] array - ) throws IOException { - if (array == null) { - return null; - } else if (array.length == 0) { - return "[]"; - } else { - final StringBuilder b = new StringBuilder().append('[').append( - toJsonString(array[0])); - for(int i = 1; i < array.length; i++) { - b.append(", ").append(toJsonString(array[i])); - } - return b.append(']').toString(); - } - } - - /** Convert an Object[] to a DatanodeInfo[]. */ - public static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) { - if (objects == null) { - return null; - } else if (objects.length == 0) { - return EMPTY_DATANODE_INFO_ARRAY; - } else { - final DatanodeInfo[] array = new DatanodeInfo[objects.length]; - for(int i = 0; i < array.length; i++) { - array[i] = (DatanodeInfo)toDatanodeInfo((Map) objects[i]); - } - return array; - } - } - - /** Convert a LocatedBlock to a Json string. */ - public static String toJsonString(final LocatedBlock locatedblock - ) throws IOException { - if (locatedblock == null) { - return null; - } - - final Map m = locatedBlockMap.get(); - m.put("blockToken", toJsonString(locatedblock.getBlockToken())); - m.put("isCorrupt", locatedblock.isCorrupt()); - m.put("startOffset", locatedblock.getStartOffset()); - m.put("block", toJsonString(locatedblock.getBlock())); - - m.put("locations", toJsonString(locatedblock.getLocations())); - return JSON.toString(m); - } - - /** Convert a Json map to LocatedBlock. */ - public static LocatedBlock toLocatedBlock(final Map m) throws IOException { - if (m == null) { - return null; - } - - final ExtendedBlock b = toExtendedBlock((Map)JSON.parse((String)m.get("block"))); - final DatanodeInfo[] locations = toDatanodeInfoArray( - (Object[])JSON.parse((String)m.get("locations"))); - final long startOffset = (Long)m.get("startOffset"); - final boolean isCorrupt = (Boolean)m.get("isCorrupt"); - - final LocatedBlock locatedblock = new LocatedBlock(b, locations, startOffset, isCorrupt); - locatedblock.setBlockToken(toBlockToken((Map)JSON.parse((String)m.get("blockToken")))); - return locatedblock; - } - - /** Convert a LocatedBlock[] to a Json string. */ - public static String toJsonString(final List array - ) throws IOException { - if (array == null) { - return null; - } else if (array.size() == 0) { - return "[]"; - } else { - final StringBuilder b = new StringBuilder().append('[').append( - toJsonString(array.get(0))); - for(int i = 1; i < array.size(); i++) { - b.append(",\n ").append(toJsonString(array.get(i))); - } - return b.append(']').toString(); - } - } - - /** Convert an Object[] to a List of LocatedBlock. - * @throws IOException */ - public static List toLocatedBlockList(final Object[] objects - ) throws IOException { - if (objects == null) { - return null; - } else if (objects.length == 0) { - return Collections.emptyList(); - } else { - final List list = new ArrayList(objects.length); - for(int i = 0; i < objects.length; i++) { - list.add((LocatedBlock)toLocatedBlock((Map)objects[i])); - } - return list; - } - } - - /** Convert LocatedBlocks to a Json string. */ - public static String toJsonString(final LocatedBlocks locatedblocks - ) throws IOException { - if (locatedblocks == null) { - return null; - } - - final Map m = jsonMap.get(); - m.put("fileLength", locatedblocks.getFileLength()); - m.put("isUnderConstruction", locatedblocks.isUnderConstruction()); - - m.put("locatedBlocks", toJsonString(locatedblocks.getLocatedBlocks())); - m.put("lastLocatedBlock", toJsonString(locatedblocks.getLastLocatedBlock())); - m.put("isLastBlockComplete", locatedblocks.isLastBlockComplete()); - return JSON.toString(m); - } - - /** Convert a Json map to LocatedBlock. */ - public static LocatedBlocks toLocatedBlocks(final Map m - ) throws IOException { - if (m == null) { - return null; - } - - final long fileLength = (Long)m.get("fileLength"); - final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction"); - final List locatedBlocks = toLocatedBlockList( - (Object[])JSON.parse((String) m.get("locatedBlocks"))); - final LocatedBlock lastLocatedBlock = toLocatedBlock( - (Map)JSON.parse((String)m.get("lastLocatedBlock"))); - final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete"); - return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks, - lastLocatedBlock, isLastBlockComplete); - } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java deleted file mode 100644 index 687b8747673..00000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java +++ /dev/null @@ -1,85 +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.hdfs.web; - -import java.net.URI; -import java.util.List; -import java.util.Map; - -import javax.ws.rs.core.MultivaluedMap; -import javax.ws.rs.core.UriBuilder; - -import com.sun.jersey.spi.container.ContainerRequest; -import com.sun.jersey.spi.container.ContainerRequestFilter; -import com.sun.jersey.spi.container.ContainerResponseFilter; -import com.sun.jersey.spi.container.ResourceFilter; - -/** - * A filter to change parameter names to lower cases - * so that parameter names are considered as case insensitive. - */ -public class ParamFilter implements ResourceFilter { - private static final ContainerRequestFilter LOWER_CASE - = new ContainerRequestFilter() { - @Override - public ContainerRequest filter(final ContainerRequest request) { - final MultivaluedMap parameters = request.getQueryParameters(); - if (containsUpperCase(parameters.keySet())) { - //rebuild URI - final URI lower = rebuildQuery(request.getRequestUri(), parameters); - request.setUris(request.getBaseUri(), lower); - } - return request; - } - }; - - @Override - public ContainerRequestFilter getRequestFilter() { - return LOWER_CASE; - } - - @Override - public ContainerResponseFilter getResponseFilter() { - return null; - } - - /** Do the strings contain upper case letters? */ - private static boolean containsUpperCase(final Iterable strings) { - for(String s : strings) { - for(int i = 0; i < s.length(); i++) { - if (Character.isUpperCase(s.charAt(i))) { - return true; - } - } - } - return false; - } - - /** Rebuild the URI query with lower case parameter names. */ - private static URI rebuildQuery(final URI uri, - final MultivaluedMap parameters) { - UriBuilder b = UriBuilder.fromUri(uri).replaceQuery(""); - for(Map.Entry> e : parameters.entrySet()) { - final String key = e.getKey().toLowerCase(); - for(String v : e.getValue()) { - b = b.queryParam(key, v); - } - } - return b.build(); - } -} \ 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 index 35c325281b5..df86456e899 100644 --- 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 @@ -27,12 +27,9 @@ import java.net.HttpURLConnection; import java.net.URI; import java.net.URISyntaxException; import java.net.URL; -import java.util.Arrays; -import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileAlreadyExistsException; @@ -48,7 +45,6 @@ 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.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.server.namenode.SafeModeException; import org.apache.hadoop.hdfs.web.resources.AccessTimeParam; import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; @@ -58,9 +54,7 @@ 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; @@ -69,16 +63,13 @@ 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.RenewerParam; import org.apache.hadoop.hdfs.web.resources.ReplicationParam; import org.apache.hadoop.hdfs.web.resources.UserParam; -import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.client.AuthenticatedURL; import org.apache.hadoop.security.authentication.client.AuthenticationException; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Progressable; import org.mortbay.util.ajax.JSON; @@ -91,24 +82,17 @@ public class WebHdfsFileSystem extends HftpFileSystem { private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator(); - private final UserGroupInformation ugi; + private UserGroupInformation ugi; private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token(); protected Path workingDir; - { - try { - ugi = UserGroupInformation.getCurrentUser(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - @Override public synchronized void initialize(URI uri, Configuration conf ) throws IOException { super.initialize(uri, conf); setConf(conf); + ugi = UserGroupInformation.getCurrentUser(); this.workingDir = getHomeDirectory(); } @@ -179,11 +163,11 @@ public class WebHdfsFileSystem extends HftpFileSystem { } } - URL toUrl(final HttpOpParam.Op op, final Path fspath, + 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 - + (fspath == null? "/": makeQualified(fspath).toUri().getPath()); + + makeQualified(fspath).toUri().getPath(); final String query = op.toQueryString() + '&' + new UserParam(ugi) + Param.toSortedString("&", parameters); @@ -412,41 +396,4 @@ public class WebHdfsFileSystem extends HftpFileSystem { } return statuses; } - - @Override - public Token getDelegationToken(final String renewer - ) throws IOException { - final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN; - final Map m = run(op, null, new RenewerParam(renewer)); - final Token token = JsonUtil.toDelegationToken(m); - token.setService(new Text(getCanonicalServiceName())); - return token; - } - - @Override - public List> getDelegationTokens(final String renewer - ) throws IOException { - final Token[] t = {getDelegationToken(renewer)}; - return Arrays.asList(t); - } - - @Override - public BlockLocation[] getFileBlockLocations(final FileStatus status, - final long offset, final long length) throws IOException { - if (status == null) { - return null; - } - return getFileBlockLocations(status.getPath(), offset, length); - } - - @Override - public BlockLocation[] getFileBlockLocations(final Path p, - final long offset, final long length) throws IOException { - statistics.incrementReadOps(1); - - final HttpOpParam.Op op = GetOpParam.Op.GETFILEBLOCKLOCATIONS; - final Map m = run(op, p, new OffsetParam(offset), - new LengthParam(length)); - return DFSUtil.locatedBlocks2Locations(JsonUtil.toLocatedBlocks(m)); - } } \ 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 index 8d82131c703..830e5cd32dd 100644 --- 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 @@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.web.resources; /** Access time parameter. */ public class AccessTimeParam extends LongParam { /** Parameter name. */ - public static final String NAME = "accesstime"; + public static final String NAME = "accessTime"; /** Default parameter value. */ public static final String DEFAULT = "-1"; 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 index 96114968074..e50b282f33b 100644 --- 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 @@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration; /** Block size parameter. */ public class BlockSizeParam extends LongParam { /** Parameter name. */ - public static final String NAME = "blocksize"; + public static final String NAME = "blockSize"; /** Default parameter value. */ public static final String DEFAULT = NULL; 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 index 148834b1024..424e5ba2533 100644 --- 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 @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; /** Buffer size parameter. */ public class BufferSizeParam extends IntegerParam { /** Parameter name. */ - public static final String NAME = "buffersize"; + public static final String NAME = "bufferSize"; /** Default parameter value. */ public static final String DEFAULT = NULL; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java index ad08773ea24..80f0c4b0b33 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java @@ -17,12 +17,13 @@ */ package org.apache.hadoop.hdfs.web.resources; +import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.security.UserGroupInformation; /** Delegation token parameter. */ public class DelegationParam extends StringParam { /** Parameter name. */ - public static final String NAME = "delegation"; + public static final String NAME = JspHelper.DELEGATION_PARAMETER_NAME; /** Default parameter value. */ public static final String DEFAULT = ""; 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 index 12962b4a4ee..e61e858ee49 100644 --- 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 @@ -21,6 +21,9 @@ 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), 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 index 5fa52456f92..7d522a38770 100644 --- 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 @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Path; /** Destination path parameter. */ public class DstPathParam extends StringParam { /** Parameter name. */ - public static final String NAME = "dstpath"; + public static final String NAME = "dstPath"; /** Default parameter value. */ public static final String DEFAULT = ""; 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 index d547f1b1b4d..6f11871ebb8 100644 --- 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 @@ -21,16 +21,16 @@ 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), - GETFILEBLOCKLOCATIONS(HttpURLConnection.HTTP_OK), GETFILESTATUS(HttpURLConnection.HTTP_OK), LISTSTATUS(HttpURLConnection.HTTP_OK), - GETDELEGATIONTOKEN(HttpURLConnection.HTTP_OK), - NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED); final int expectedHttpResponseCode; 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 index 422ec0f2f2f..644c4032dbc 100644 --- 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 @@ -20,9 +20,6 @@ package org.apache.hadoop.hdfs.web.resources; /** Http operation parameter. */ public abstract class HttpOpParam & HttpOpParam.Op> extends EnumParam { - /** Parameter name. */ - public static final String NAME = "op"; - /** Default parameter value. */ public static final String DEFAULT = NULL; 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 index a0e38a97e7d..d43da073280 100644 --- 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 @@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.web.resources; /** Modification time parameter. */ public class ModificationTimeParam extends LongParam { /** Parameter name. */ - public static final String NAME = "modificationtime"; + public static final String NAME = "modificationTime"; /** Default parameter value. */ public static final String DEFAULT = "-1"; 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 index f6945bb4351..6639ece7b25 100644 --- 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 @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hdfs.web.resources; -/** Overwrite parameter. */ +/** Recursive parameter. */ public class OverwriteParam extends BooleanParam { /** Parameter name. */ public static final String NAME = "overwrite"; 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 index b553ecc6701..116d6af8b36 100644 --- 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 @@ -21,6 +21,9 @@ 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), 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 index dcfaa6f06cd..00703fefbc7 100644 --- 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 @@ -21,6 +21,9 @@ 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), 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 index d7c157d5086..ec66a51c788 100644 --- 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 @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Options; /** Rename option set parameter. */ public class RenameOptionSetParam extends EnumSetParam { /** Parameter name. */ - public static final String NAME = "renameoptions"; + public static final String NAME = "renameOptions"; /** Default parameter value. */ public static final String DEFAULT = ""; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RenewerParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RenewerParam.java deleted file mode 100644 index 750e8bc91b1..00000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/RenewerParam.java +++ /dev/null @@ -1,41 +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.hdfs.web.resources; - -/** Renewer parameter. */ -public class RenewerParam extends StringParam { - /** Parameter name. */ - public static final String NAME = "renewer"; - /** Default parameter value. */ - public static final String DEFAULT = NULL; - - private static final Domain DOMAIN = new Domain(NAME, null); - - /** - * Constructor. - * @param str a string representation of the parameter value. - */ - public RenewerParam(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/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 59a8ff645ff..9fae462f04f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -683,4 +683,24 @@ creations/deletions), or "all". + + dfs.web.authentication.kerberos.principal + HTTP/${dfs.web.hostname}@${kerberos.realm} + + The HTTP Kerberos principal used by Hadoop-Auth in the HTTP endpoint. + + The HTTP Kerberos principal MUST start with 'HTTP/' per Kerberos + HTTP SPENGO specification. + + + + + dfs.web.authentication.kerberos.keytab + ${user.home}/dfs.web.keytab + + The Kerberos keytab file with the credentials for the + HTTP Kerberos principal used by Hadoop-Auth in the HTTP endpoint. + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java index 495e8e191a3..2d50ce440f9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java @@ -72,7 +72,6 @@ public class TestDFSPermission extends TestCase { final private static Path NON_EXISTENT_FILE = new Path("/NonExistentFile"); private FileSystem fs; - private MiniDFSCluster cluster; private static Random r; static { @@ -106,25 +105,18 @@ public class TestDFSPermission extends TestCase { } } - @Override - public void setUp() throws IOException { - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); - cluster.waitActive(); - } - - @Override - public void tearDown() throws IOException { - if (cluster != null) { - cluster.shutdown(); - } - } - /** This tests if permission setting in create, mkdir, and * setPermission works correctly */ public void testPermissionSetting() throws Exception { - testPermissionSetting(OpType.CREATE); // test file creation - testPermissionSetting(OpType.MKDIRS); // test directory creation + MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); + try { + cluster.waitActive(); + testPermissionSetting(OpType.CREATE); // test file creation + testPermissionSetting(OpType.MKDIRS); // test directory creation + } finally { + cluster.shutdown(); + } } private void initFileSystem(short umask) throws Exception { @@ -253,22 +245,17 @@ public class TestDFSPermission extends TestCase { } } - /** - * check that ImmutableFsPermission can be used as the argument - * to setPermission - */ - public void testImmutableFsPermission() throws IOException { - fs = FileSystem.get(conf); - - // set the permission of the root to be world-wide rwx - fs.setPermission(new Path("/"), - FsPermission.createImmutable((short)0777)); - } - /* check if the ownership of a file/directory is set correctly */ public void testOwnership() throws Exception { - testOwnership(OpType.CREATE); // test file creation - testOwnership(OpType.MKDIRS); // test directory creation + MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); + try { + cluster.waitActive(); + testOwnership(OpType.CREATE); // test file creation + testOwnership(OpType.MKDIRS); // test directory creation + } finally { + fs.close(); + cluster.shutdown(); + } } /* change a file/directory's owner and group. @@ -355,7 +342,9 @@ public class TestDFSPermission extends TestCase { /* Check if namenode performs permission checking correctly for * superuser, file owner, group owner, and other users */ public void testPermissionChecking() throws Exception { + MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); try { + cluster.waitActive(); fs = FileSystem.get(conf); // set the permission of the root to be world-wide rwx @@ -412,6 +401,7 @@ public class TestDFSPermission extends TestCase { parentPermissions, permissions, parentPaths, filePaths, dirPaths); } finally { fs.close(); + cluster.shutdown(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java index f154ff7d203..fc883118f82 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java @@ -29,7 +29,8 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.HadoopIllegalArgumentException; +import junit.framework.Assert; + import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -39,7 +40,8 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; -import static org.apache.hadoop.hdfs.DFSConfigKeys.*; +import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION; + public class TestDFSUtil { /** @@ -74,141 +76,79 @@ public class TestDFSUtil { } } - assertTrue("expected 1 corrupt files but got " + corruptCount, - corruptCount == 1); - + assertTrue("expected 1 corrupt files but got " + corruptCount, + corruptCount == 1); + // test an empty location bs = DFSUtil.locatedBlocks2Locations(new LocatedBlocks()); assertEquals(0, bs.length); } - - private Configuration setupAddress(String key) { + /** + * Test for + * {@link DFSUtil#getNameServiceIds(Configuration)} + * {@link DFSUtil#getNameServiceId(Configuration)} + * {@link DFSUtil#getNNServiceRpcAddresses(Configuration)} + */ + @Test + public void testMultipleNamenodes() throws IOException { HdfsConfiguration conf = new HdfsConfiguration(); - conf.set(DFS_FEDERATION_NAMESERVICES, "nn1"); - conf.set(DFSUtil.getNameServiceIdKey(key, "nn1"), "localhost:9000"); - return conf; - } - - /** - * Test {@link DFSUtil#getNamenodeNameServiceId(Configuration)} to ensure - * nameserviceId from the configuration returned - */ - @Test - public void getNameServiceId() { - HdfsConfiguration conf = new HdfsConfiguration(); - conf.set(DFS_FEDERATION_NAMESERVICE_ID, "nn1"); - assertEquals("nn1", DFSUtil.getNamenodeNameServiceId(conf)); - } - - /** - * Test {@link DFSUtil#getNameNodeNameServiceId(Configuration)} to ensure - * nameserviceId for namenode is determined based on matching the address with - * local node's address - */ - @Test - public void getNameNodeNameServiceId() { - Configuration conf = setupAddress(DFS_NAMENODE_RPC_ADDRESS_KEY); - assertEquals("nn1", DFSUtil.getNamenodeNameServiceId(conf)); - } - - /** - * Test {@link DFSUtil#getBackupNameServiceId(Configuration)} to ensure - * nameserviceId for backup node is determined based on matching the address - * with local node's address - */ - @Test - public void getBackupNameServiceId() { - Configuration conf = setupAddress(DFS_NAMENODE_BACKUP_ADDRESS_KEY); - assertEquals("nn1", DFSUtil.getBackupNameServiceId(conf)); - } - - /** - * Test {@link DFSUtil#getSecondaryNameServiceId(Configuration)} to ensure - * nameserviceId for backup node is determined based on matching the address - * with local node's address - */ - @Test - public void getSecondaryNameServiceId() { - Configuration conf = setupAddress(DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY); - assertEquals("nn1", DFSUtil.getSecondaryNameServiceId(conf)); - } - - /** - * Test {@link DFSUtil#getNameServiceId(Configuration, String))} to ensure - * exception is thrown when multiple rpc addresses match the local node's - * address - */ - @Test(expected = HadoopIllegalArgumentException.class) - public void testGetNameServiceIdException() { - HdfsConfiguration conf = new HdfsConfiguration(); - conf.set(DFS_FEDERATION_NAMESERVICES, "nn1,nn2"); - conf.set(DFSUtil.getNameServiceIdKey(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"), - "localhost:9000"); - conf.set(DFSUtil.getNameServiceIdKey(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"), - "localhost:9001"); - DFSUtil.getNamenodeNameServiceId(conf); - fail("Expected exception is not thrown"); - } - - /** - * Test {@link DFSUtil#getNameServiceIds(Configuration)} - */ - @Test - public void testGetNameServiceIds() { - HdfsConfiguration conf = new HdfsConfiguration(); - conf.set(DFS_FEDERATION_NAMESERVICES, "nn1,nn2"); + conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, "nn1,nn2"); + + // Test - The configured nameserviceIds are returned Collection nameserviceIds = DFSUtil.getNameServiceIds(conf); Iterator it = nameserviceIds.iterator(); assertEquals(2, nameserviceIds.size()); assertEquals("nn1", it.next().toString()); assertEquals("nn2", it.next().toString()); - } - - /** - * Test for {@link DFSUtil#getNNServiceRpcAddresses(Configuration)} - * {@link DFSUtil#getNameServiceIdFromAddress(Configuration, InetSocketAddress, String...) - * (Configuration)} - */ - @Test - public void testMultipleNamenodes() throws IOException { - HdfsConfiguration conf = new HdfsConfiguration(); - conf.set(DFS_FEDERATION_NAMESERVICES, "nn1,nn2"); + + // Tests default nameserviceId is returned + conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID, "nn1"); + assertEquals("nn1", DFSUtil.getNameServiceId(conf)); + // Test - configured list of namenodes are returned final String NN1_ADDRESS = "localhost:9000"; final String NN2_ADDRESS = "localhost:9001"; final String NN3_ADDRESS = "localhost:9002"; - conf.set(DFSUtil.getNameServiceIdKey(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"), - NN1_ADDRESS); - conf.set(DFSUtil.getNameServiceIdKey(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"), - NN2_ADDRESS); - - Collection nnAddresses = DFSUtil - .getNNServiceRpcAddresses(conf); + conf.set(DFSUtil.getNameServiceIdKey( + DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"), NN1_ADDRESS); + conf.set(DFSUtil.getNameServiceIdKey( + DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"), NN2_ADDRESS); + + Collection nnAddresses = + DFSUtil.getNNServiceRpcAddresses(conf); assertEquals(2, nnAddresses.size()); Iterator iterator = nnAddresses.iterator(); + assertEquals(2, nameserviceIds.size()); InetSocketAddress addr = iterator.next(); assertEquals("localhost", addr.getHostName()); assertEquals(9000, addr.getPort()); addr = iterator.next(); assertEquals("localhost", addr.getHostName()); assertEquals(9001, addr.getPort()); - + // Test - can look up nameservice ID from service address - checkNameServiceId(conf, NN1_ADDRESS, "nn1"); - checkNameServiceId(conf, NN2_ADDRESS, "nn2"); - checkNameServiceId(conf, NN3_ADDRESS, null); + InetSocketAddress testAddress1 = NetUtils.createSocketAddr(NN1_ADDRESS); + String nameserviceId = DFSUtil.getNameServiceIdFromAddress( + conf, testAddress1, + DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, + DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY); + assertEquals("nn1", nameserviceId); + InetSocketAddress testAddress2 = NetUtils.createSocketAddr(NN2_ADDRESS); + nameserviceId = DFSUtil.getNameServiceIdFromAddress( + conf, testAddress2, + DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, + DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY); + assertEquals("nn2", nameserviceId); + InetSocketAddress testAddress3 = NetUtils.createSocketAddr(NN3_ADDRESS); + nameserviceId = DFSUtil.getNameServiceIdFromAddress( + conf, testAddress3, + DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, + DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY); + assertNull(nameserviceId); } - - public void checkNameServiceId(Configuration conf, String addr, - String expectedNameServiceId) { - InetSocketAddress s = NetUtils.createSocketAddr(addr); - String nameserviceId = DFSUtil.getNameServiceIdFromAddress(conf, s, - DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY); - assertEquals(expectedNameServiceId, nameserviceId); - } - - /** + + /** * Test for * {@link DFSUtil#isDefaultNamenodeAddress(Configuration, InetSocketAddress, String...)} */ @@ -217,25 +157,27 @@ public class TestDFSUtil { HdfsConfiguration conf = new HdfsConfiguration(); final String DEFAULT_ADDRESS = "localhost:9000"; final String NN2_ADDRESS = "localhost:9001"; - conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, DEFAULT_ADDRESS); - + conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, DEFAULT_ADDRESS); + InetSocketAddress testAddress1 = NetUtils.createSocketAddr(DEFAULT_ADDRESS); boolean isDefault = DFSUtil.isDefaultNamenodeAddress(conf, testAddress1, - DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY); + DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, + DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY); assertTrue(isDefault); InetSocketAddress testAddress2 = NetUtils.createSocketAddr(NN2_ADDRESS); isDefault = DFSUtil.isDefaultNamenodeAddress(conf, testAddress2, - DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY); + DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, + DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY); assertFalse(isDefault); } - + /** Tests to ensure default namenode is used as fallback */ @Test public void testDefaultNamenode() throws IOException { HdfsConfiguration conf = new HdfsConfiguration(); final String hdfs_default = "hdfs://localhost:9999/"; - conf.set(FS_DEFAULT_NAME_KEY, hdfs_default); - // If DFS_FEDERATION_NAMESERVICES is not set, verify that + conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, hdfs_default); + // If DFSConfigKeys.DFS_FEDERATION_NAMESERVICES is not set, verify that // default namenode address is returned. List addrList = DFSUtil.getNNServiceRpcAddresses(conf); assertEquals(1, addrList.size()); @@ -249,26 +191,26 @@ public class TestDFSUtil { @Test public void testConfModification() throws IOException { final HdfsConfiguration conf = new HdfsConfiguration(); - conf.set(DFS_FEDERATION_NAMESERVICES, "nn1"); - conf.set(DFS_FEDERATION_NAMESERVICE_ID, "nn1"); - final String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf); - + conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, "nn1"); + conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID, "nn1"); + final String nameserviceId = DFSUtil.getNameServiceId(conf); + // Set the nameservice specific keys with nameserviceId in the config key for (String key : NameNode.NAMESERVICE_SPECIFIC_KEYS) { // Note: value is same as the key conf.set(DFSUtil.getNameServiceIdKey(key, nameserviceId), key); } - + // Initialize generic keys from specific keys - NameNode.initializeGenericKeys(conf, nameserviceId); - + NameNode.initializeGenericKeys(conf); + // Retrieve the keys without nameserviceId and Ensure generic keys are set // to the correct value for (String key : NameNode.NAMESERVICE_SPECIFIC_KEYS) { assertEquals(key, conf.get(key)); } } - + /** * Tests for empty configuration, an exception is thrown from * {@link DFSUtil#getNNServiceRpcAddresses(Configuration)} @@ -296,16 +238,16 @@ public class TestDFSUtil { } catch (IOException expected) { } } - + @Test - public void testGetServerInfo() { + public void testGetServerInfo(){ HdfsConfiguration conf = new HdfsConfiguration(); conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); UserGroupInformation.setConfiguration(conf); String httpsport = DFSUtil.getInfoServer(null, conf, true); - assertEquals("0.0.0.0:50470", httpsport); + Assert.assertEquals("0.0.0.0:50470", httpsport); String httpport = DFSUtil.getInfoServer(null, conf, false); - assertEquals("0.0.0.0:50070", httpport); + Assert.assertEquals("0.0.0.0:50070", httpport); } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java index d7ee516b0a8..a0727a6c90b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java @@ -17,10 +17,6 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - import java.io.OutputStream; import java.security.PrivilegedExceptionAction; @@ -28,15 +24,17 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException; +import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; -import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil; import org.apache.hadoop.hdfs.tools.DFSAdmin; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; +import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException; + import org.junit.Test; +import static org.junit.Assert.*; /** A class for testing quota-related commands */ public class TestQuota { @@ -843,14 +841,6 @@ public class TestQuota { DFSAdmin admin = new DFSAdmin(conf); try { - - //Test for deafult NameSpace Quota - long nsQuota = FSImageTestUtil.getNSQuota(cluster.getNameNode() - .getNamesystem()); - assertTrue( - "Default namespace quota expected as long max. But the value is :" - + nsQuota, nsQuota == Long.MAX_VALUE); - Path dir = new Path("/test"); boolean exceededQuota = false; ContentSummary c; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java index 9c577f740ee..d6397b6a2ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java @@ -23,12 +23,12 @@ package org.apache.hadoop.hdfs.security; import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; -import java.net.URI; import java.security.PrivilegedExceptionAction; +import junit.framework.Assert; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -38,16 +38,12 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; -import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods; -import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; -import org.apache.log4j.Level; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -60,13 +56,12 @@ public class TestDelegationToken { @Before public void setUp() throws Exception { config = new HdfsConfiguration(); - config.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true); config.setLong(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000); config.setLong(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 5000); config.set("hadoop.security.auth_to_local", "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT"); FileSystem.setDefaultUri(config, "hdfs://localhost:" + "0"); - cluster = new MiniDFSCluster.Builder(config).numDataNodes(0).build(); + cluster = new MiniDFSCluster.Builder(config).build(); cluster.waitActive(); dtSecretManager = NameNodeAdapter.getDtSecretManager( cluster.getNamesystem()); @@ -158,31 +153,6 @@ public class TestDelegationToken { dtSecretManager.renewToken(token, "JobTracker"); } - @Test - public void testDelegationTokenWebHdfsApi() throws Exception { - ((Log4JLogger)NamenodeWebHdfsMethods.LOG).getLogger().setLevel(Level.ALL); - final String uri = WebHdfsFileSystem.SCHEME + "://" - + config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY); - //get file system as JobTracker - final UserGroupInformation ugi = UserGroupInformation.createUserForTesting( - "JobTracker", new String[]{"user"}); - final WebHdfsFileSystem webhdfs = ugi.doAs( - new PrivilegedExceptionAction() { - @Override - public WebHdfsFileSystem run() throws Exception { - return (WebHdfsFileSystem)FileSystem.get(new URI(uri), config); - } - }); - - final Token token = webhdfs.getDelegationToken("JobTracker"); - DelegationTokenIdentifier identifier = new DelegationTokenIdentifier(); - byte[] tokenId = token.getIdentifier(); - identifier.readFields(new DataInputStream(new ByteArrayInputStream(tokenId))); - LOG.info("A valid token should have non-null password, and should be renewed successfully"); - Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier)); - dtSecretManager.renewToken(token, "JobTracker"); - } - @Test public void testDelegationTokenWithDoAs() throws Exception { final DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHost2NodesMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHost2NodesMap.java index d34cf1c4c6e..7b9126f7de8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHost2NodesMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHost2NodesMap.java @@ -18,34 +18,31 @@ package org.apache.hadoop.hdfs.server.blockmanagement; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import junit.framework.TestCase; import org.apache.hadoop.hdfs.protocol.DatanodeID; -import org.junit.Before; -import org.junit.Test; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.Host2NodesMap; -public class TestHost2NodesMap { - private Host2NodesMap map = new Host2NodesMap(); - private final DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] { +public class TestHost2NodesMap extends TestCase { + static private Host2NodesMap map = new Host2NodesMap(); + private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] { new DatanodeDescriptor(new DatanodeID("h1:5020"), "/d1/r1"), new DatanodeDescriptor(new DatanodeID("h2:5020"), "/d1/r1"), new DatanodeDescriptor(new DatanodeID("h3:5020"), "/d1/r2"), new DatanodeDescriptor(new DatanodeID("h3:5030"), "/d1/r2"), }; - private final DatanodeDescriptor NULL_NODE = null; - private final DatanodeDescriptor NODE = new DatanodeDescriptor(new DatanodeID("h3:5040"), - "/d1/r4"); + private final static DatanodeDescriptor NULL_NODE = null; + private final static DatanodeDescriptor NODE = + new DatanodeDescriptor(new DatanodeID("h3:5040"), "/d1/r4"); - @Before - public void setup() { + static { for(DatanodeDescriptor node:dataNodes) { map.add(node); } map.add(NULL_NODE); } - @Test public void testContains() throws Exception { for(int i=0; i " + line); - } - - //check if the command successes. - assertTrue(fs.getFileStatus(p).isDirectory()); - } } diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 43fed6baa1c..1d637419eb5 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -29,8 +29,6 @@ Trunk (unreleased changes) findBugs, correct links to findBugs artifacts and no links to the artifacts when there are no warnings. (Tom White via vinodkv). - MAPREDUCE-3081. Fix vaidya startup script. (gkesavan via suhas). - Release 0.23.0 - Unreleased INCOMPATIBLE CHANGES @@ -72,9 +70,6 @@ Release 0.23.0 - Unreleased MAPREDUCE-2037. Capture intermediate progress, CPU and memory usage for tasks. (Dick King via acmurthy) - MAPREDUCE-2930. Added the ability to be able to generate graphs from the - state-machine definitions. (Binglin Chang via vinodkv) - IMPROVEMENTS MAPREDUCE-2187. Reporter sends progress during sort/merge. (Anupam Seth via @@ -312,15 +307,6 @@ Release 0.23.0 - Unreleased MAPREDUCE-2726. Added job-file to the AM and JobHistoryServer web interfaces. (Jeffrey Naisbitt via vinodkv) - MAPREDUCE-3055. Simplified ApplicationAttemptId passing to - ApplicationMaster via environment variable. (vinodkv) - - MAPREDUCE-3092. Removed a special comparator for JobIDs in JobHistory as - JobIDs are already comparable. (Devaraj K via vinodkv) - - MAPREDUCE-3099. Add docs for setting up a single node MRv2 cluster. - (mahadev) - OPTIMIZATIONS MAPREDUCE-2026. Make JobTracker.getJobCounters() and @@ -332,9 +318,6 @@ Release 0.23.0 - Unreleased MAPREDUCE-901. Efficient framework counters. (llu via acmurthy) - MAPREDUCE-2880. Improve classpath-construction for mapreduce AM and - containers. (Arun C Murthy via vinodkv) - BUG FIXES MAPREDUCE-2603. Disable High-Ram emulation in system tests. @@ -1387,91 +1370,6 @@ Release 0.23.0 - Unreleased YarnClientProtocolProvider and ensured MiniMRYarnCluster sets JobHistory configuration for tests. (acmurthy) - MAPREDUCE-3018. Fixed -file option for streaming. (mahadev via acmurthy) - - MAPREDUCE-3036. Fixed metrics for reserved resources in CS. (Robert Evans - via acmurthy) - - MAPREDUCE-2998. Fixed a bug in TaskAttemptImpl which caused it to fork - bin/mapred too many times. (vinodkv via acmurthy) - - MAPREDUCE-3023. Fixed clients to display queue state correctly. (Ravi - Prakash via acmurthy) - - MAPREDUCE-2970. Fixed NPEs in corner cases with different configurations - for mapreduce.framework.name. (Venu Gopala Rao via vinodkv) - - MAPREDUCE-3062. Fixed default RMAdmin address. (Chris Riccomini - via acmurthy) - - MAPREDUCE-3066. Fixed default ResourceTracker address for the NodeManager. - (Chris Riccomini via acmurthy) - - MAPREDUCE-3044. Pipes jobs stuck without making progress. (mahadev) - - MAPREDUCE-2754. Fixed MR AM stdout, stderr and syslog to redirect to - correct log-files. (Ravi Teja Ch N V via vinodkv) - - MAPREDUCE-3073. Fixed build issues in MR1. (mahadev via acmurthy) - - MAPREDUCE-2691. Increase threadpool size for launching containers in - MapReduce ApplicationMaster. (vinodkv via acmurthy) - - - MAPREDUCE-2990. Fixed display of NodeHealthStatus. (Subroto Sanyal via - acmurthy) - - MAPREDUCE-3053. Better diagnostic message for unknown methods in ProtoBuf - RPCs. (vinodkv via acmurthy) - - MAPREDUCE-2952. Fixed ResourceManager/MR-client to consume diagnostics - for AM failures in a couple of corner cases. (Arun C Murthy via vinodkv) - - MAPREDUCE-3064. 27 unit test failures with Invalid - "mapreduce.jobtracker.address" configuration value for - JobTracker: "local" (Venu Gopala Rao via mahadev) - - MAPREDUCE-3090. Fix MR AM to use ApplicationAttemptId rather than - (ApplicationId, startCount) consistently. (acmurthy) - - MAPREDUCE-2646. Fixed AMRMProtocol to return containers based on - priority. (Sharad Agarwal and Arun C Murthy via vinodkv) - - MAPREDUCE-3031. Proper handling of killed containers to prevent stuck - containers/AMs on an external kill signal. (Siddharth Seth via vinodkv) - - MAPREDUCE-2984. Better error message for displaying completed containers. - (Devaraj K via acmurthy) - - MAPREDUCE-3071. app master configuration web UI link under the Job menu - opens up application menu. (thomas graves via mahadev) - - MAPREDUCE-3067. Ensure exit-code is set correctly for containers. (Hitesh - Shah via acmurthy) - - MAPREDUCE-2999. Fix YARN webapp framework to properly filter servlet - paths. (Thomas Graves via vinodkv) - - MAPREDUCE-3095. fairscheduler ivy including wrong version for hdfs. - (John George via mahadev) - - MAPREDUCE-3054. Unable to kill submitted jobs. (mahadev) - - MAPREDUCE-3021. Change base urls for RM web-ui. (Thomas Graves via - acmurthy) - - MAPREDUCE-3041. Fixed ClientRMProtocol to provide min/max resource - capabilities along-with new ApplicationId for application submission. - (Hitesh Shah via acmurthy) - - MAPREDUCE-2843. Fixed the node-table to be completely displayed and making - node entries on RM UI to be sortable. (Abhijit Suresh Shingate via vinodkv) - - MAPREDUCE-3110. Fixed TestRPC failure. (vinodkv) - - MAPREDUCE-3078. Ensure MapReduce AM reports progress correctly for - displaying on the RM Web-UI. (vinodkv via acmurthy) - Release 0.22.0 - Unreleased INCOMPATIBLE CHANGES diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml index 0f12598fc17..17d4b8b0e54 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml @@ -55,12 +55,6 @@ hadoop-yarn-server-resourcemanager test - - org.apache.hadoop - hadoop-yarn-server-resourcemanager - test-jar - test - org.apache.hadoop hadoop-mapreduce-client-shuffle @@ -119,41 +113,4 @@ - - - - visualize - - false - - - - - org.codehaus.mojo - exec-maven-plugin - 1.2 - - - compile - - java - - - test - org.apache.hadoop.yarn.util.VisualizeStateMachine - - MapReduce - org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl, - org.apache.hadoop.mapreduce.v2.app.job.impl.TaskImpl, - org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl - MapReduce.gv - - - - - - - - - diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java index ce6557abd03..d9d5b1f3076 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java @@ -18,27 +18,27 @@ package org.apache.hadoop.mapred; +import java.io.File; import java.net.InetSocketAddress; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Vector; -import org.apache.hadoop.fs.Path; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.mapred.TaskLog.LogName; import org.apache.hadoop.mapreduce.ID; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.v2.util.MRApps; -import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.yarn.conf.YarnConfiguration; public class MapReduceChildJVM { + private static final String SYSTEM_PATH_SEPARATOR = + System.getProperty("path.separator"); - private static String getTaskLogFile(LogName filter) { - return ApplicationConstants.LOG_DIR_EXPANSION_VAR + Path.SEPARATOR + - filter.toString(); + private static final Log LOG = LogFactory.getLog(MapReduceChildJVM.class); + + private static File getTaskLogFile(String logDir, LogName filter) { + return new File(logDir, filter.toString()); } private static String getChildEnv(JobConf jobConf, boolean isMap) { @@ -50,53 +50,32 @@ public class MapReduceChildJVM { jobConf.get(jobConf.MAPRED_TASK_ENV)); } - private static String getChildLogLevel(JobConf conf, boolean isMap) { - if (isMap) { - return conf.get( - MRJobConfig.MAP_LOG_LEVEL, - JobConf.DEFAULT_LOG_LEVEL.toString() - ); - } else { - return conf.get( - MRJobConfig.REDUCE_LOG_LEVEL, - JobConf.DEFAULT_LOG_LEVEL.toString() - ); - } - } - - public static void setVMEnv(Map environment, - Task task) { + public static void setVMEnv(Map env, + List classPaths, String pwd, String containerLogDir, + String nmLdLibraryPath, Task task, CharSequence applicationTokensFile) { JobConf conf = task.conf; - // Shell - environment.put( - Environment.SHELL.name(), - conf.get( - MRJobConfig.MAPRED_ADMIN_USER_SHELL, - MRJobConfig.DEFAULT_SHELL) - ); - - // Add pwd to LD_LIBRARY_PATH, add this before adding anything else - MRApps.addToEnvironment( - environment, - Environment.LD_LIBRARY_PATH.name(), - Environment.PWD.$()); + // Add classpath. + CharSequence cp = env.get("CLASSPATH"); + String classpath = StringUtils.join(SYSTEM_PATH_SEPARATOR, classPaths); + if (null == cp) { + env.put("CLASSPATH", classpath); + } else { + env.put("CLASSPATH", classpath + SYSTEM_PATH_SEPARATOR + cp); + } - // Add the env variables passed by the user & admin - String mapredChildEnv = getChildEnv(conf, task.isMapTask()); - MRApps.setEnvFromInputString(environment, mapredChildEnv); - MRApps.setEnvFromInputString( - environment, - conf.get( - MRJobConfig.MAPRED_ADMIN_USER_ENV, - MRJobConfig.DEFAULT_MAPRED_ADMIN_USER_ENV) - ); + /////// Environmental variable LD_LIBRARY_PATH + StringBuilder ldLibraryPath = new StringBuilder(); - // Set logging level - environment.put( - "HADOOP_ROOT_LOGGER", - getChildLogLevel(conf, task.isMapTask()) + ",CLA"); + ldLibraryPath.append(nmLdLibraryPath); + ldLibraryPath.append(SYSTEM_PATH_SEPARATOR); + ldLibraryPath.append(pwd); + env.put("LD_LIBRARY_PATH", ldLibraryPath.toString()); + /////// Environmental variable LD_LIBRARY_PATH + + // for the child of task jvm, set hadoop.root.logger + env.put("HADOOP_ROOT_LOGGER", "DEBUG,CLA"); // TODO: Debug // TODO: The following is useful for instance in streaming tasks. Should be // set in ApplicationMaster's env by the RM. @@ -110,69 +89,76 @@ public class MapReduceChildJVM { // properties. long logSize = TaskLog.getTaskLogLength(conf); Vector logProps = new Vector(4); - setupLog4jProperties(logProps, logSize); + setupLog4jProperties(logProps, logSize, containerLogDir); Iterator it = logProps.iterator(); StringBuffer buffer = new StringBuffer(); while (it.hasNext()) { buffer.append(" " + it.next()); } hadoopClientOpts = hadoopClientOpts + buffer.toString(); - environment.put("HADOOP_CLIENT_OPTS", hadoopClientOpts); + + env.put("HADOOP_CLIENT_OPTS", hadoopClientOpts); - // Add stdout/stderr env - environment.put( - MRJobConfig.STDOUT_LOGFILE_ENV, - getTaskLogFile(TaskLog.LogName.STDOUT) - ); - environment.put( - MRJobConfig.STDERR_LOGFILE_ENV, - getTaskLogFile(TaskLog.LogName.STDERR) - ); + // add the env variables passed by the user + String mapredChildEnv = getChildEnv(conf, task.isMapTask()); + if (mapredChildEnv != null && mapredChildEnv.length() > 0) { + String childEnvs[] = mapredChildEnv.split(","); + for (String cEnv : childEnvs) { + String[] parts = cEnv.split("="); // split on '=' + String value = (String) env.get(parts[0]); + if (value != null) { + // replace $env with the child's env constructed by tt's + // example LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp + value = parts[1].replace("$" + parts[0], value); + } else { + // this key is not configured by the tt for the child .. get it + // from the tt's env + // example PATH=$PATH:/tmp + value = System.getenv(parts[0]); // Get from NM? + if (value != null) { + // the env key is present in the tt's env + value = parts[1].replace("$" + parts[0], value); + } else { + // the env key is note present anywhere .. simply set it + // example X=$X:/tmp or X=/tmp + value = parts[1].replace("$" + parts[0], ""); + } + } + env.put(parts[0], value); + } + } + + //This should not be set here (If an OS check is requied. moved to ContainerLuanch) + // env.put("JVM_PID", "`echo $$`"); + + env.put(Constants.STDOUT_LOGFILE_ENV, + getTaskLogFile(containerLogDir, TaskLog.LogName.STDOUT).toString()); + env.put(Constants.STDERR_LOGFILE_ENV, + getTaskLogFile(containerLogDir, TaskLog.LogName.STDERR).toString()); } private static String getChildJavaOpts(JobConf jobConf, boolean isMapTask) { - String userClasspath = ""; - String adminClasspath = ""; if (isMapTask) { - userClasspath = - jobConf.get( - JobConf.MAPRED_MAP_TASK_JAVA_OPTS, - jobConf.get( - JobConf.MAPRED_TASK_JAVA_OPTS, - JobConf.DEFAULT_MAPRED_TASK_JAVA_OPTS) - ); - adminClasspath = - jobConf.get( - MRJobConfig.MAPRED_MAP_ADMIN_JAVA_OPTS, - MRJobConfig.DEFAULT_MAPRED_ADMIN_JAVA_OPTS); - } else { - userClasspath = - jobConf.get( - JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, - jobConf.get( - JobConf.MAPRED_TASK_JAVA_OPTS, - JobConf.DEFAULT_MAPRED_TASK_JAVA_OPTS) - ); - adminClasspath = - jobConf.get( - MRJobConfig.MAPRED_REDUCE_ADMIN_JAVA_OPTS, - MRJobConfig.DEFAULT_MAPRED_ADMIN_JAVA_OPTS); + return jobConf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, jobConf.get( + JobConf.MAPRED_TASK_JAVA_OPTS, + JobConf.DEFAULT_MAPRED_TASK_JAVA_OPTS)); } - - // Add admin classpath first so it can be overridden by user. - return adminClasspath + " " + userClasspath; + return jobConf + .get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, jobConf.get( + JobConf.MAPRED_TASK_JAVA_OPTS, + JobConf.DEFAULT_MAPRED_TASK_JAVA_OPTS)); } private static void setupLog4jProperties(Vector vargs, - long logSize) { + long logSize, String containerLogDir) { vargs.add("-Dlog4j.configuration=container-log4j.properties"); - vargs.add("-D" + MRJobConfig.TASK_LOG_DIR + "=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR); - vargs.add("-D" + MRJobConfig.TASK_LOG_SIZE + "=" + logSize); + vargs.add("-Dhadoop.yarn.mr.containerLogDir=" + containerLogDir); + vargs.add("-Dhadoop.yarn.mr.totalLogFileSize=" + logSize); } public static List getVMCommand( - InetSocketAddress taskAttemptListenerAddr, Task task, - ID jvmID) { + InetSocketAddress taskAttemptListenerAddr, Task task, String javaHome, + String workDir, String logDir, String childTmpDir, ID jvmID) { TaskAttemptID attemptID = task.getTaskID(); JobConf conf = task.conf; @@ -180,7 +166,7 @@ public class MapReduceChildJVM { Vector vargs = new Vector(8); vargs.add("exec"); - vargs.add(Environment.JAVA_HOME.$() + "/bin/java"); + vargs.add(javaHome + "/bin/java"); // Add child (task) java-vm options. // @@ -213,26 +199,44 @@ public class MapReduceChildJVM { String javaOpts = getChildJavaOpts(conf, task.isMapTask()); javaOpts = javaOpts.replace("@taskid@", attemptID.toString()); String [] javaOptsSplit = javaOpts.split(" "); + + // Add java.library.path; necessary for loading native libraries. + // + // 1. We add the 'cwd' of the task to it's java.library.path to help + // users distribute native libraries via the DistributedCache. + // 2. The user can also specify extra paths to be added to the + // java.library.path via mapred.{map|reduce}.child.java.opts. + // + String libraryPath = workDir; + boolean hasUserLDPath = false; + for(int i=0; i" + getTaskLogFile(TaskLog.LogName.STDOUT)); - vargs.add("2>" + getTaskLogFile(TaskLog.LogName.STDERR)); + vargs.add("1>" + getTaskLogFile(logDir, TaskLog.LogName.STDERR)); + vargs.add("2>" + getTaskLogFile(logDir, TaskLog.LogName.STDOUT)); // Final commmand StringBuilder mergedCommand = new StringBuilder(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java index 0ab220bf383..3021004f9dd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java @@ -47,6 +47,7 @@ import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.mapreduce.security.TokenCache; import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier; import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; +import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.source.JvmMetrics; import org.apache.hadoop.security.Credentials; @@ -70,7 +71,7 @@ class YarnChild { LOG.debug("Child starting"); final JobConf defaultConf = new JobConf(); - defaultConf.addResource(MRJobConfig.JOB_CONF_FILE); + defaultConf.addResource(MRConstants.JOB_CONF_FILE); UserGroupInformation.setConfiguration(defaultConf); String host = args[0]; @@ -237,7 +238,7 @@ class YarnChild { private static JobConf configureTask(Task task, Credentials credentials, Token jt) throws IOException { - final JobConf job = new JobConf(MRJobConfig.JOB_CONF_FILE); + final JobConf job = new JobConf(MRConstants.JOB_CONF_FILE); job.setCredentials(credentials); // set tcp nodelay job.setBoolean("ipc.client.tcpnodelay", true); @@ -259,7 +260,7 @@ class YarnChild { // Overwrite the localized task jobconf which is linked to in the current // work-dir. - Path localTaskFile = new Path(MRJobConfig.JOB_CONF_FILE); + Path localTaskFile = new Path(Constants.JOBFILE); writeLocalJobFile(localTaskFile, job); task.setJobFile(localTaskFile.toString()); task.setConf(job); 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 8b7d578fc9b..20c7e9779e8 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 @@ -39,6 +39,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler; import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; +import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.app.client.ClientService; @@ -77,7 +78,6 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.Clock; import org.apache.hadoop.yarn.SystemClock; import org.apache.hadoop.yarn.YarnException; -import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -88,7 +88,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.service.AbstractService; import org.apache.hadoop.yarn.service.CompositeService; import org.apache.hadoop.yarn.service.Service; -import org.apache.hadoop.yarn.util.ConverterUtils; /** * The Map-Reduce Application Master. @@ -115,6 +114,8 @@ public class MRAppMaster extends CompositeService { private Clock clock; private final long startTime = System.currentTimeMillis(); private String appName; + private final int startCount; + private final ApplicationId appID; private final ApplicationAttemptId appAttemptID; protected final MRAppMetrics metrics; private Set completedTasksFromPreviousRun; @@ -132,16 +133,21 @@ public class MRAppMaster extends CompositeService { private Job job; - public MRAppMaster(ApplicationAttemptId applicationAttemptId) { - this(applicationAttemptId, new SystemClock()); + public MRAppMaster(ApplicationId applicationId, int startCount) { + this(applicationId, new SystemClock(), startCount); } - public MRAppMaster(ApplicationAttemptId applicationAttemptId, Clock clock) { + public MRAppMaster(ApplicationId applicationId, Clock clock, int startCount) { super(MRAppMaster.class.getName()); this.clock = clock; - this.appAttemptID = applicationAttemptId; + this.appID = applicationId; + this.appAttemptID = RecordFactoryProvider.getRecordFactory(null) + .newRecordInstance(ApplicationAttemptId.class); + this.appAttemptID.setApplicationId(appID); + this.appAttemptID.setAttemptId(startCount); + this.startCount = startCount; this.metrics = MRAppMetrics.create(); - LOG.info("Created MRAppMaster for application " + applicationAttemptId); + LOG.info("Created MRAppMaster for application " + applicationId); } @Override @@ -153,9 +159,9 @@ public class MRAppMaster extends CompositeService { appName = conf.get(MRJobConfig.JOB_NAME, ""); if (conf.getBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, false) - && appAttemptID.getAttemptId() > 1) { + && startCount > 1) { LOG.info("Recovery is enabled. Will try to recover from previous life."); - Recovery recoveryServ = new RecoveryService(appAttemptID, clock); + Recovery recoveryServ = new RecoveryService(appID, clock, startCount); addIfService(recoveryServ); dispatcher = recoveryServ.getDispatcher(); clock = recoveryServ.getClock(); @@ -237,10 +243,10 @@ public class MRAppMaster extends CompositeService { // Read the file-system tokens from the localized tokens-file. Path jobSubmitDir = FileContext.getLocalFSFileContext().makeQualified( - new Path(new File(MRJobConfig.JOB_SUBMIT_DIR) + new Path(new File(MRConstants.JOB_SUBMIT_DIR) .getAbsolutePath())); Path jobTokenFile = - new Path(jobSubmitDir, MRJobConfig.APPLICATION_TOKENS_FILE); + new Path(jobSubmitDir, MRConstants.APPLICATION_TOKENS_FILE); fsTokens.addAll(Credentials.readTokenStorageFile(jobTokenFile, conf)); LOG.info("jobSubmitDir=" + jobSubmitDir + " jobTokenFile=" + jobTokenFile); @@ -258,8 +264,8 @@ public class MRAppMaster extends CompositeService { // ////////// End of obtaining the tokens needed by the job. ////////// // create single job - Job newJob = new JobImpl(appAttemptID, conf, dispatcher.getEventHandler(), - taskAttemptListener, jobTokenSecretManager, fsTokens, clock, + Job newJob = new JobImpl(appID, conf, dispatcher.getEventHandler(), + taskAttemptListener, jobTokenSecretManager, fsTokens, clock, startCount, completedTasksFromPreviousRun, metrics, currentUser.getUserName()); ((RunningAppContext) context).jobs.put(newJob.getID(), newJob); @@ -370,11 +376,11 @@ public class MRAppMaster extends CompositeService { } public ApplicationId getAppID() { - return appAttemptID.getApplicationId(); + return appID; } public int getStartCount() { - return appAttemptID.getAttemptId(); + return startCount; } public AppContext getContext() { @@ -499,7 +505,7 @@ public class MRAppMaster extends CompositeService { @Override public ApplicationId getApplicationID() { - return appAttemptID.getApplicationId(); + return appID; } @Override @@ -549,9 +555,9 @@ public class MRAppMaster extends CompositeService { // It's more test friendly to put it here. DefaultMetricsSystem.initialize("MRAppMaster"); - // create a job event for job intialization + /** create a job event for job intialization */ JobEvent initJobEvent = new JobEvent(job.getID(), JobEventType.JOB_INIT); - // Send init to the job (this does NOT trigger job execution) + /** send init to the job (this does NOT trigger job execution) */ // This is a synchronous call, not an event through dispatcher. We want // job-init to be done completely here. jobEventDispatcher.handle(initJobEvent); @@ -642,21 +648,17 @@ public class MRAppMaster extends CompositeService { public static void main(String[] args) { try { - String applicationAttemptIdStr = System - .getenv(ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV); - if (applicationAttemptIdStr == null) { - String msg = ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV - + " is null"; - LOG.error(msg); - throw new IOException(msg); - } - ApplicationAttemptId applicationAttemptId = ConverterUtils - .toApplicationAttemptId(applicationAttemptIdStr); - MRAppMaster appMaster = new MRAppMaster(applicationAttemptId); + //Configuration.addDefaultResource("job.xml"); + ApplicationId applicationId = RecordFactoryProvider + .getRecordFactory(null).newRecordInstance(ApplicationId.class); + applicationId.setClusterTimestamp(Long.valueOf(args[0])); + applicationId.setId(Integer.valueOf(args[1])); + int failCount = Integer.valueOf(args[2]); + MRAppMaster appMaster = new MRAppMaster(applicationId, failCount); Runtime.getRuntime().addShutdownHook( new CompositeServiceShutdownHook(appMaster)); YarnConfiguration conf = new YarnConfiguration(new JobConf()); - conf.addResource(new Path(MRJobConfig.JOB_CONF_FILE)); + conf.addResource(new Path(MRConstants.JOB_CONF_FILE)); conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name")); UserGroupInformation.setConfiguration(conf); 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 f84a4d9dbe4..73359bb12a5 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 @@ -149,7 +149,7 @@ public class MRClientService extends AbstractService + ":" + server.getPort()); LOG.info("Instantiated MRClientService at " + this.bindAddress); try { - webApp = WebApps.$for("mapreduce", AppContext.class, appContext).with(conf). + webApp = WebApps.$for("yarn", AppContext.class, appContext).with(conf). start(new AMWebApp()); } catch (Exception e) { LOG.error("Webapps failed to start. Ignoring for now:", e); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java index c26bc24695c..69de493b16b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java @@ -64,6 +64,7 @@ import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo; import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader; import org.apache.hadoop.mapreduce.task.JobContextImpl; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.api.records.Counter; import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup; import org.apache.hadoop.mapreduce.v2.api.records.Counters; @@ -92,7 +93,6 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType; import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics; import org.apache.hadoop.mapreduce.v2.util.MRApps; -import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; @@ -101,7 +101,6 @@ import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.Clock; 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.event.EventHandler; import org.apache.hadoop.yarn.factories.RecordFactory; @@ -130,11 +129,11 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, RecordFactoryProvider.getRecordFactory(null); //final fields - private final ApplicationAttemptId applicationAttemptId; private final Clock clock; private final JobACLsManager aclsManager; private final String username; private final Map jobACLs; + private final int startCount; private final Set completedTasksFromPreviousRun; private final Lock readLock; private final Lock writeLock; @@ -366,26 +365,26 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, private Token jobToken; private JobTokenSecretManager jobTokenSecretManager; - public JobImpl(ApplicationAttemptId applicationAttemptId, Configuration conf, + public JobImpl(ApplicationId appID, Configuration conf, EventHandler eventHandler, TaskAttemptListener taskAttemptListener, JobTokenSecretManager jobTokenSecretManager, - Credentials fsTokenCredentials, Clock clock, + Credentials fsTokenCredentials, Clock clock, int startCount, Set completedTasksFromPreviousRun, MRAppMetrics metrics, String userName) { - this.applicationAttemptId = applicationAttemptId; + this.jobId = recordFactory.newRecordInstance(JobId.class); this.jobName = conf.get(JobContext.JOB_NAME, ""); this.conf = conf; this.metrics = metrics; this.clock = clock; this.completedTasksFromPreviousRun = completedTasksFromPreviousRun; + this.startCount = startCount; this.userName = userName; - ApplicationId applicationId = applicationAttemptId.getApplicationId(); - jobId.setAppId(applicationId); - jobId.setId(applicationId.getId()); + jobId.setAppId(appID); + jobId.setId(appID.getId()); oldJobId = TypeConverter.fromYarn(jobId); LOG.info("Job created" + - " appId=" + applicationId + + " appId=" + appID + " jobId=" + jobId + " oldJobId=" + oldJobId); @@ -585,17 +584,25 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, public JobReport getReport() { readLock.lock(); try { - JobState state = getState(); - - if (getState() == JobState.NEW) { - return MRBuilderUtils.newJobReport(jobId, jobName, username, state, - startTime, finishTime, setupProgress, 0.0f, - 0.0f, cleanupProgress); + JobReport report = recordFactory.newRecordInstance(JobReport.class); + report.setJobId(jobId); + report.setJobState(getState()); + + // TODO - Fix to correctly setup report and to check state + if (report.getJobState() == JobState.NEW) { + return report; } + + report.setStartTime(startTime); + report.setFinishTime(finishTime); + report.setSetupProgress(setupProgress); + report.setCleanupProgress(cleanupProgress); + report.setMapProgress(computeProgress(mapTasks)); + report.setReduceProgress(computeProgress(reduceTasks)); + report.setJobName(jobName); + report.setUser(username); - return MRBuilderUtils.newJobReport(jobId, jobName, username, state, - startTime, finishTime, setupProgress, computeProgress(mapTasks), - computeProgress(reduceTasks), cleanupProgress); + return report; } finally { readLock.unlock(); } @@ -1000,7 +1007,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, FileSystem.get(job.conf).makeQualified( new Path(path, oldJobIDString)); job.remoteJobConfFile = - new Path(job.remoteJobSubmitDir, MRJobConfig.JOB_CONF_FILE); + new Path(job.remoteJobSubmitDir, MRConstants.JOB_CONF_FILE); // Prepare the TaskAttemptListener server for authentication of Containers // TaskAttemptListener gets the information via jobTokenSecretManager. @@ -1026,7 +1033,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, Path remoteJobTokenFile = new Path(job.remoteJobSubmitDir, - MRJobConfig.APPLICATION_TOKENS_FILE); + MRConstants.APPLICATION_TOKENS_FILE); tokenStorage.writeTokenStorageFile(remoteJobTokenFile, job.conf); LOG.info("Writing back the job-token file on the remote file system:" + remoteJobTokenFile.toString()); @@ -1071,8 +1078,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, job.conf, splits[i], job.taskAttemptListener, job.committer, job.jobToken, job.fsTokens.getAllTokens(), - job.clock, job.completedTasksFromPreviousRun, - job.applicationAttemptId.getAttemptId(), + job.clock, job.completedTasksFromPreviousRun, job.startCount, job.metrics); job.addTask(task); } @@ -1089,9 +1095,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, job.conf, job.numMapTasks, job.taskAttemptListener, job.committer, job.jobToken, job.fsTokens.getAllTokens(), job.clock, - job.completedTasksFromPreviousRun, - job.applicationAttemptId.getAttemptId(), - job.metrics); + job.completedTasksFromPreviousRun, job.startCount, job.metrics); job.addTask(task); } LOG.info("Number of reduces for job " + job.jobId + " = " 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 495d00e22c9..cc9f6bddf06 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 @@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app.job.impl; import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; +import java.net.URI; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; @@ -61,6 +62,7 @@ import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStartedEvent; import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptUnsuccessfulCompletionEvent; import org.apache.hadoop.mapreduce.security.TokenCache; import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier; +import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.api.records.Counter; import org.apache.hadoop.mapreduce.v2.api.records.Counters; import org.apache.hadoop.mapreduce.v2.api.records.Phase; @@ -101,7 +103,6 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.Clock; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerToken; @@ -116,6 +117,7 @@ import org.apache.hadoop.yarn.state.InvalidStateTransitonException; import org.apache.hadoop.yarn.state.SingleArcTransition; import org.apache.hadoop.yarn.state.StateMachine; import org.apache.hadoop.yarn.state.StateMachineFactory; +import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.RackResolver; @@ -151,7 +153,7 @@ public abstract class TaskAttemptImpl implements private Token jobToken; private static AtomicBoolean initialClasspathFlag = new AtomicBoolean(); private static String initialClasspath = null; - private static final Object classpathLock = new Object(); + private final Object classpathLock = new Object(); private long launchTime; private long finishTime; private WrappedProgressSplitsBlock progressSplitBlock; @@ -516,8 +518,8 @@ public abstract class TaskAttemptImpl implements return initialClasspath; } Map env = new HashMap(); - MRApps.setClasspath(env); - initialClasspath = env.get(Environment.CLASSPATH.name()); + MRApps.setInitialClasspath(env); + initialClasspath = env.get(MRApps.CLASSPATH); initialClasspathFlag.set(true); return initialClasspath; } @@ -529,18 +531,16 @@ public abstract class TaskAttemptImpl implements */ private ContainerLaunchContext createContainerLaunchContext() { + ContainerLaunchContext container = + recordFactory.newRecordInstance(ContainerLaunchContext.class); + // Application resources Map localResources = new HashMap(); // Application environment Map environment = new HashMap(); - - // Service data - Map serviceData = new HashMap(); - - // Tokens - ByteBuffer tokens = ByteBuffer.wrap(new byte[]{}); + try { FileSystem remoteFS = FileSystem.get(conf); @@ -550,7 +550,7 @@ public abstract class TaskAttemptImpl implements MRJobConfig.JAR))).makeQualified(remoteFS.getUri(), remoteFS.getWorkingDirectory()); localResources.put( - MRJobConfig.JOB_JAR, + MRConstants.JOB_JAR, createLocalResource(remoteFS, recordFactory, remoteJobJar, LocalResourceType.FILE, LocalResourceVisibility.APPLICATION)); LOG.info("The job-jar file on the remote FS is " @@ -570,9 +570,9 @@ public abstract class TaskAttemptImpl implements Path remoteJobSubmitDir = new Path(path, oldJobId.toString()); Path remoteJobConfPath = - new Path(remoteJobSubmitDir, MRJobConfig.JOB_CONF_FILE); + new Path(remoteJobSubmitDir, MRConstants.JOB_CONF_FILE); localResources.put( - MRJobConfig.JOB_CONF_FILE, + MRConstants.JOB_CONF_FILE, createLocalResource(remoteFS, recordFactory, remoteJobConfPath, LocalResourceType.FILE, LocalResourceVisibility.APPLICATION)); LOG.info("The job-conf file on the remote FS is " @@ -580,8 +580,12 @@ public abstract class TaskAttemptImpl implements // //////////// End of JobConf setup // Setup DistributedCache - MRApps.setupDistributedCache(conf, localResources); + MRApps.setupDistributedCache(conf, localResources, environment); + // Set local-resources and environment + container.setLocalResources(localResources); + container.setEnvironment(environment); + // Setup up tokens Credentials taskCredentials = new Credentials(); @@ -602,43 +606,52 @@ public abstract class TaskAttemptImpl implements LOG.info("Size of containertokens_dob is " + taskCredentials.numberOfTokens()); taskCredentials.writeTokenStorageToStream(containerTokens_dob); - tokens = + container.setContainerTokens( ByteBuffer.wrap(containerTokens_dob.getData(), 0, - containerTokens_dob.getLength()); + containerTokens_dob.getLength())); // Add shuffle token LOG.info("Putting shuffle token in serviceData"); + Map serviceData = new HashMap(); serviceData.put(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID, ShuffleHandler.serializeServiceData(jobToken)); + container.setServiceData(serviceData); - MRApps.addToEnvironment( - environment, - Environment.CLASSPATH.name(), - getInitialClasspath()); + MRApps.addToClassPath(container.getEnvironment(), getInitialClasspath()); } catch (IOException e) { throw new YarnException(e); } - - // Setup environment - MapReduceChildJVM.setVMEnv(environment, remoteTask); - - // Set up the launch command - List commands = MapReduceChildJVM.getVMCommand( - taskAttemptListener.getAddress(), remoteTask, - jvmID); + container.setContainerId(containerID); + container.setUser(conf.get(MRJobConfig.USER_NAME)); // TODO: Fix + + File workDir = new File("$PWD"); // Will be expanded by the shell. + String containerLogDir = + new File(ApplicationConstants.LOG_DIR_EXPANSION_VAR).toString(); + String childTmpDir = new File(workDir, "tmp").toString(); + String javaHome = "${JAVA_HOME}"; // Will be expanded by the shell. + String nmLdLibraryPath = "{LD_LIBRARY_PATH}"; // Expanded by the shell? + List classPaths = new ArrayList(); + + String localizedApplicationTokensFile = + new File(workDir, MRConstants.APPLICATION_TOKENS_FILE).toString(); + classPaths.add(MRConstants.JOB_JAR); + classPaths.add(MRConstants.YARN_MAPREDUCE_APP_JAR_PATH); + classPaths.add(workDir.toString()); // TODO + + // Construct the actual Container + container.setCommands(MapReduceChildJVM.getVMCommand( + taskAttemptListener.getAddress(), remoteTask, javaHome, + workDir.toString(), containerLogDir, childTmpDir, jvmID)); + + MapReduceChildJVM.setVMEnv(container.getEnvironment(), classPaths, + workDir.toString(), containerLogDir, nmLdLibraryPath, remoteTask, + localizedApplicationTokensFile); + // Construct the actual Container - ContainerLaunchContext container = - recordFactory.newRecordInstance(ContainerLaunchContext.class); container.setContainerId(containerID); container.setUser(conf.get(MRJobConfig.USER_NAME)); container.setResource(assignedCapability); - container.setLocalResources(localResources); - container.setEnvironment(environment); - container.setCommands(commands); - container.setServiceData(serviceData); - container.setContainerTokens(tokens); - return container; } 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 95e17d8f4f6..982f7d334ae 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 @@ -73,8 +73,6 @@ public class ContainerLauncherImpl extends AbstractService implements private AppContext context; private ThreadPoolExecutor launcherPool; - private static final int INITIAL_POOL_SIZE = 10; - private int limitOnPoolSize; private Thread eventHandlingThread; private BlockingQueue eventQueue = new LinkedBlockingQueue(); @@ -98,17 +96,16 @@ public class ContainerLauncherImpl extends AbstractService implements YarnConfiguration.YARN_SECURITY_INFO, ContainerManagerSecurityInfo.class, SecurityInfo.class); this.recordFactory = RecordFactoryProvider.getRecordFactory(conf); - this.limitOnPoolSize = conf.getInt( - MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT, - MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT); super.init(myLocalConfig); } public void start() { - // Start with a default core-pool size of 10 and change it dynamically. - launcherPool = new ThreadPoolExecutor(INITIAL_POOL_SIZE, - Integer.MAX_VALUE, 1, TimeUnit.HOURS, - new LinkedBlockingQueue()); + launcherPool = + new ThreadPoolExecutor(getConfig().getInt( + MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT, 10), + Integer.MAX_VALUE, 1, TimeUnit.HOURS, + new LinkedBlockingQueue()); + launcherPool.prestartAllCoreThreads(); // Wait for work. eventHandlingThread = new Thread(new Runnable() { @Override public void run() { @@ -120,26 +117,6 @@ public class ContainerLauncherImpl extends AbstractService implements LOG.error("Returning, interrupted : " + e); return; } - - int poolSize = launcherPool.getCorePoolSize(); - - // See if we need up the pool size only if haven't reached the - // maximum limit yet. - if (poolSize != limitOnPoolSize) { - - // nodes where containers will run at *this* point of time. This is - // *not* the cluster size and doesn't need to be. - int numNodes = ugiMap.size(); - int idealPoolSize = Math.min(limitOnPoolSize, numNodes); - - if (poolSize <= idealPoolSize) { - // Bump up the pool size to idealPoolSize+INITIAL_POOL_SIZE, the - // later is just a buffer so we are not always increasing the - // pool-size - launcherPool.setCorePoolSize(idealPoolSize + INITIAL_POOL_SIZE); - } - } - // the events from the queue are handled in parallel // using a thread pool launcherPool.execute(new EventProcessor(event)); 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 0261e18b56f..18a0f2d5a6a 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 @@ -18,7 +18,6 @@ package org.apache.hadoop.mapreduce.v2.app.local; -import java.util.ArrayList; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; @@ -31,19 +30,15 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent; import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator; import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent; +import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent; import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator; -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.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.Records; /** @@ -70,20 +65,6 @@ public class LocalContainerAllocator extends RMCommunicator this.appID = context.getApplicationID(); } - @Override - protected synchronized void heartbeat() throws Exception { - AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest( - this.applicationAttemptId, this.lastResponseID, super - .getApplicationProgress(), new ArrayList(), - new ArrayList()); - AllocateResponse allocateResponse = scheduler.allocate(allocateRequest); - AMResponse response = allocateResponse.getAMResponse(); - if (response.getReboot()) { - // TODO - LOG.info("Event from RM: shutting down Application Master"); - } - } - @Override public void handle(ContainerAllocatorEvent event) { if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/RecoveryService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/RecoveryService.java index ca213f17f86..073411c9b47 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/RecoveryService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/RecoveryService.java @@ -58,7 +58,7 @@ 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.JobHistoryUtils; 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.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; @@ -92,9 +92,10 @@ public class RecoveryService extends CompositeService implements Recovery { private static final Log LOG = LogFactory.getLog(RecoveryService.class); - private final ApplicationAttemptId applicationAttemptId; + private final ApplicationId appID; private final Dispatcher dispatcher; private final ControlledClock clock; + private final int startCount; private JobInfo jobInfo = null; private final Map completedTasks = @@ -105,10 +106,10 @@ public class RecoveryService extends CompositeService implements Recovery { private volatile boolean recoveryMode = false; - public RecoveryService(ApplicationAttemptId applicationAttemptId, - Clock clock) { + public RecoveryService(ApplicationId appID, Clock clock, int startCount) { super("RecoveringDispatcher"); - this.applicationAttemptId = applicationAttemptId; + this.appID = appID; + this.startCount = startCount; this.dispatcher = new RecoveryDispatcher(); this.clock = new ControlledClock(clock); addService((Service) dispatcher); @@ -151,8 +152,7 @@ public class RecoveryService extends CompositeService implements Recovery { private void parse() throws IOException { // TODO: parse history file based on startCount - String jobName = - TypeConverter.fromYarn(applicationAttemptId.getApplicationId()).toString(); + String jobName = TypeConverter.fromYarn(appID).toString(); String jobhistoryDir = JobHistoryUtils.getConfiguredHistoryStagingDirPrefix(getConfig()); FSDataInputStream in = null; Path historyFile = null; @@ -160,9 +160,8 @@ public class RecoveryService extends CompositeService implements Recovery { new Path(jobhistoryDir)); FileContext fc = FileContext.getFileContext(histDirPath.toUri(), getConfig()); - //read the previous history file historyFile = fc.makeQualified(JobHistoryUtils.getStagingJobHistoryFile( - histDirPath, jobName, (applicationAttemptId.getAttemptId() - 1))); + histDirPath, jobName, startCount - 1)); //read the previous history file in = fc.open(historyFile); JobHistoryParser parser = new JobHistoryParser(in); jobInfo = parser.parse(); 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 15a7e3f6a5a..db4a60b1dcc 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 @@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.app.rm; import java.io.IOException; import java.security.PrivilegedAction; +import java.util.ArrayList; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -28,7 +29,6 @@ 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.JobReport; import org.apache.hadoop.mapreduce.v2.api.records.JobState; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.client.ClientService; @@ -42,12 +42,17 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.AMRMProtocol; import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.AMResponse; 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.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.factories.RecordFactory; @@ -59,7 +64,7 @@ import org.apache.hadoop.yarn.service.AbstractService; /** * Registers/unregisters to RM and sends heartbeats to RM. */ -public abstract class RMCommunicator extends AbstractService { +public class RMCommunicator extends AbstractService { private static final Log LOG = LogFactory.getLog(RMContainerAllocator.class); private int rmPollInterval;//millis protected ApplicationId applicationId; @@ -69,7 +74,7 @@ public abstract class RMCommunicator extends AbstractService { protected EventHandler eventHandler; protected AMRMProtocol scheduler; private final ClientService clientService; - protected int lastResponseID; + private int lastResponseID; private Resource minContainerCapability; private Resource maxContainerCapability; @@ -116,34 +121,6 @@ public abstract class RMCommunicator extends AbstractService { return job; } - /** - * Get the appProgress. Can be used only after this component is started. - * @return the appProgress. - */ - protected float getApplicationProgress() { - // For now just a single job. In future when we have a DAG, we need an - // aggregate progress. - JobReport report = this.job.getReport(); - float setupWeight = 0.05f; - float cleanupWeight = 0.05f; - float mapWeight = 0.0f; - float reduceWeight = 0.0f; - int numMaps = this.job.getTotalMaps(); - int numReduces = this.job.getTotalReduces(); - if (numMaps == 0 && numReduces == 0) { - } else if (numMaps == 0) { - reduceWeight = 0.9f; - } else if (numReduces == 0) { - mapWeight = 0.9f; - } else { - mapWeight = reduceWeight = 0.45f; - } - return (report.getSetupProgress() * setupWeight - + report.getCleanupProgress() * cleanupWeight - + report.getMapProgress() * mapWeight + report.getReduceProgress() - * reduceWeight); - } - protected void register() { //Register String host = @@ -285,5 +262,18 @@ public abstract class RMCommunicator extends AbstractService { }); } - protected abstract void heartbeat() throws Exception; + protected synchronized void heartbeat() throws Exception { + AllocateRequest allocateRequest = + recordFactory.newRecordInstance(AllocateRequest.class); + allocateRequest.setApplicationAttemptId(applicationAttemptId); + allocateRequest.setResponseId(lastResponseID); + allocateRequest.addAllAsks(new ArrayList()); + allocateRequest.addAllReleases(new ArrayList()); + AllocateResponse allocateResponse = scheduler.allocate(allocateRequest); + AMResponse response = allocateResponse.getAMResponse(); + if (response.getReboot()) { + LOG.info("Event from RM: shutting down Application Master"); + } + } + } 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 7b75cd1fbd7..ff232104bd4 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 @@ -586,21 +586,37 @@ public class RMContainerAllocator extends RMContainerRequestor private ContainerRequest assign(Container allocated) { ContainerRequest assigned = null; - Priority priority = allocated.getPriority(); - if (PRIORITY_FAST_FAIL_MAP.equals(priority)) { - LOG.info("Assigning container " + allocated + " to fast fail map"); - assigned = assignToFailedMap(allocated); - } else if (PRIORITY_REDUCE.equals(priority)) { - LOG.info("Assigning container " + allocated + " to reduce"); - assigned = assignToReduce(allocated); - } else if (PRIORITY_MAP.equals(priority)) { - LOG.info("Assigning container " + allocated + " to map"); - assigned = assignToMap(allocated); - } else { - LOG.warn("Container allocated at unwanted priority: " + priority + - ". Returning to RM..."); - } + if (mapResourceReqt != reduceResourceReqt) { + //assign based on size + LOG.info("Assigning based on container size"); + if (allocated.getResource().getMemory() == mapResourceReqt) { + assigned = assignToFailedMap(allocated); + if (assigned == null) { + assigned = assignToMap(allocated); + } + } else if (allocated.getResource().getMemory() == reduceResourceReqt) { + assigned = assignToReduce(allocated); + } + return assigned; + } + + //container can be given to either map or reduce + //assign based on priority + + //try to assign to earlierFailedMaps if present + assigned = assignToFailedMap(allocated); + + //Assign to reduces before assigning to maps ? + if (assigned == null) { + assigned = assignToReduce(allocated); + } + + //try to assign to maps if present + if (assigned == null) { + assigned = assignToMap(allocated); + } + return assigned; } 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 cda2ed678af..b9f0c6ee45e 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 @@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest; 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.util.BuilderUtils; /** * Keeps the data structures to send container requests to RM. @@ -108,11 +107,15 @@ public abstract class RMContainerRequestor extends RMCommunicator { LOG.info("maxTaskFailuresPerNode is " + maxTaskFailuresPerNode); } + protected abstract void heartbeat() throws Exception; + protected AMResponse makeRemoteRequest() throws YarnRemoteException { - AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest( - applicationAttemptId, lastResponseID, super.getApplicationProgress(), - new ArrayList(ask), new ArrayList( - release)); + AllocateRequest allocateRequest = recordFactory + .newRecordInstance(AllocateRequest.class); + allocateRequest.setApplicationAttemptId(applicationAttemptId); + allocateRequest.setResponseId(lastResponseID); + allocateRequest.addAllAsks(new ArrayList(ask)); + allocateRequest.addAllReleases(new ArrayList(release)); AllocateResponse allocateResponse = scheduler.allocate(allocateRequest); AMResponse response = allocateResponse.getAMResponse(); lastResponseID = response.getResponseId(); 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 ab7d23ef9dc..feb019fe162 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 @@ -35,6 +35,7 @@ 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; @@ -86,7 +87,7 @@ public class DefaultSpeculator extends AbstractService implements private final ConcurrentMap reduceContainerNeeds = new ConcurrentHashMap(); - private final Set mayHaveSpeculated = new HashSet(); + private final Set mayHaveSpeculated = new HashSet(); private final Configuration conf; private AppContext context; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java index 983859e7d67..8bf2ce1955f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java @@ -44,7 +44,6 @@ public class JobConfPage extends AppView { set(TITLE, jobID.isEmpty() ? "Bad request: missing job ID" : join("Configuration for MapReduce Job ", $(JOB_ID))); commonPreHead(html); - set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}"); set(DATATABLES_ID, "conf"); set(initID(DATATABLES, "conf"), confTableInit()); set(postInitID(DATATABLES, "conf"), confPostTableInit()); 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 8b4524ad117..bb4e2390a75 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 @@ -38,9 +38,9 @@ public class NavBlock extends HtmlBlock { div("#nav"). h3("Cluster"). ul(). - li().a(url(rmweb, "cluster", "cluster"), "About")._(). - li().a(url(rmweb, "cluster", "apps"), "Applications")._(). - li().a(url(rmweb, "cluster", "scheduler"), "Scheduler")._()._(). + li().a(url(rmweb, prefix(), "cluster"), "About")._(). + li().a(url(rmweb, prefix(), "apps"), "Applications")._(). + li().a(url(rmweb, prefix(), "scheduler"), "Scheduler")._()._(). h3("Application"). ul(). li().a(url("app/info"), "About")._(). 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 736bef639e0..d9884d146a6 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 @@ -85,7 +85,7 @@ public class TaskPage extends AppView { if (containerId != null) { String containerIdStr = ConverterUtils.toString(containerId); nodeTd._(" "). - a(".logslink", url("http://", nodeHttpAddr, "node", "containerlogs", + a(".logslink", url("http://", nodeHttpAddr, "yarn", "containerlogs", containerIdStr), "logs"); } nodeTd._(). 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 d6e2d968173..548d754a6c6 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 @@ -66,7 +66,6 @@ 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.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; @@ -92,7 +91,7 @@ public class MRApp extends MRAppMaster { private File testWorkDir; private Path testAbsPath; - private static final RecordFactory recordFactory = + private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); //if true, tasks complete automatically as soon as they are launched @@ -101,7 +100,7 @@ public class MRApp extends MRAppMaster { static ApplicationId applicationId; static { - applicationId = recordFactory.newRecordInstance(ApplicationId.class); + applicationId = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class); applicationId.setClusterTimestamp(0); applicationId.setId(0); } @@ -109,19 +108,9 @@ public class MRApp extends MRAppMaster { public MRApp(int maps, int reduces, boolean autoComplete, String testName, boolean cleanOnStart) { this(maps, reduces, autoComplete, testName, cleanOnStart, 1); } - - private static ApplicationAttemptId getApplicationAttemptId( - ApplicationId applicationId, int startCount) { - ApplicationAttemptId applicationAttemptId = - recordFactory.newRecordInstance(ApplicationAttemptId.class); - applicationAttemptId.setApplicationId(applicationId); - applicationAttemptId.setAttemptId(startCount); - return applicationAttemptId; - } - public MRApp(int maps, int reduces, boolean autoComplete, String testName, - boolean cleanOnStart, int startCount) { - super(getApplicationAttemptId(applicationId, startCount)); + public MRApp(int maps, int reduces, boolean autoComplete, String testName, boolean cleanOnStart, int startCount) { + super(applicationId, startCount); this.testWorkDir = new File("target", testName); testAbsPath = new Path(testWorkDir.getAbsolutePath()); LOG.info("PathUsed: " + testAbsPath); @@ -402,12 +391,11 @@ public class MRApp extends MRAppMaster { return localStateMachine; } - public TestJob(Configuration conf, ApplicationId applicationId, + public TestJob(Configuration conf, ApplicationId appID, EventHandler eventHandler, TaskAttemptListener taskAttemptListener, Clock clock, String user) { - super(getApplicationAttemptId(applicationId, getStartCount()), - conf, eventHandler, taskAttemptListener, - new JobTokenSecretManager(), new Credentials(), clock, + super(appID, conf, eventHandler, taskAttemptListener, + new JobTokenSecretManager(), new Credentials(), clock, getStartCount(), getCompletedTaskFromPreviousRun(), metrics, user); // This "this leak" is okay because the retained pointer is in an diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java index a1eb928919e..cbf3ab0a658 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java @@ -18,15 +18,12 @@ package org.apache.hadoop.mapreduce.v2.app; -import static org.mockito.Matchers.isA; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import junit.framework.Assert; @@ -35,651 +32,475 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; 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.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.client.ClientService; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent; -import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl; import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent; import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator; -import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; -import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetworkTopology; +import org.apache.hadoop.yarn.Clock; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.AMRMProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.AMResponse; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationMaster; +import org.apache.hadoop.yarn.api.records.ApplicationStatus; +import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; +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.resourcemanager.MockNM; -import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.RMContext; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.ipc.RPCUtil; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; -import org.apache.hadoop.yarn.util.BuilderUtils; -import org.junit.After; +import org.junit.BeforeClass; import org.junit.Test; public class TestRMContainerAllocator { - - static final Log LOG = LogFactory - .getLog(TestRMContainerAllocator.class); - static final RecordFactory recordFactory = RecordFactoryProvider - .getRecordFactory(null); - - @After - public void tearDown() { - DefaultMetricsSystem.shutdown(); - } - - @Test - public void testSimple() throws Exception { - - LOG.info("Running testSimple"); - - Configuration conf = new Configuration(); - MyResourceManager rm = new MyResourceManager(conf); - rm.start(); - DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext() - .getDispatcher(); - - // Submit the application - RMApp app = rm.submitApp(1024); - dispatcher.await(); - - MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); - amNodeManager.nodeHeartbeat(true); - dispatcher.await(); - - ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() - .getAppAttemptId(); - rm.sendAMLaunched(appAttemptId); - dispatcher.await(); - - JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0); - Job mockJob = mock(Job.class); - when(mockJob.getReport()).thenReturn( - MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, - 0, 0, 0, 0, 0, 0)); - MyContainerAllocator allocator = new MyContainerAllocator(rm, conf, - appAttemptId, mockJob); - - // add resources to scheduler - MockNM nodeManager1 = rm.registerNode("h1:1234", 10240); - MockNM nodeManager2 = rm.registerNode("h2:1234", 10240); - MockNM nodeManager3 = rm.registerNode("h3:1234", 10240); - dispatcher.await(); - - // create the container request - ContainerRequestEvent event1 = createReq(jobId, 1, 1024, - new String[] { "h1" }); - allocator.sendRequest(event1); - - // send 1 more request with different resource req - ContainerRequestEvent event2 = createReq(jobId, 2, 1024, - new String[] { "h2" }); - allocator.sendRequest(event2); - - // this tells the scheduler about the requests - // as nodes are not added, no allocations - List assigned = allocator.schedule(); - dispatcher.await(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); - - // send another request with different resource and priority - ContainerRequestEvent event3 = createReq(jobId, 3, 1024, - new String[] { "h3" }); - allocator.sendRequest(event3); - - // this tells the scheduler about the requests - // as nodes are not added, no allocations - assigned = allocator.schedule(); - dispatcher.await(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); - - // update resources in scheduler - nodeManager1.nodeHeartbeat(true); // Node heartbeat - nodeManager2.nodeHeartbeat(true); // Node heartbeat - nodeManager3.nodeHeartbeat(true); // Node heartbeat - dispatcher.await(); - - assigned = allocator.schedule(); - dispatcher.await(); - checkAssignments(new ContainerRequestEvent[] { event1, event2, event3 }, - assigned, false); - } - - @Test - public void testResource() throws Exception { - - LOG.info("Running testResource"); - - Configuration conf = new Configuration(); - MyResourceManager rm = new MyResourceManager(conf); - rm.start(); - DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext() - .getDispatcher(); - - // Submit the application - RMApp app = rm.submitApp(1024); - dispatcher.await(); - - MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); - amNodeManager.nodeHeartbeat(true); - dispatcher.await(); - - ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() - .getAppAttemptId(); - rm.sendAMLaunched(appAttemptId); - dispatcher.await(); - - JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0); - Job mockJob = mock(Job.class); - when(mockJob.getReport()).thenReturn( - MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, - 0, 0, 0, 0, 0, 0)); - MyContainerAllocator allocator = new MyContainerAllocator(rm, conf, - appAttemptId, mockJob); - - // add resources to scheduler - MockNM nodeManager1 = rm.registerNode("h1:1234", 10240); - MockNM nodeManager2 = rm.registerNode("h2:1234", 10240); - MockNM nodeManager3 = rm.registerNode("h3:1234", 10240); - dispatcher.await(); - - // create the container request - ContainerRequestEvent event1 = createReq(jobId, 1, 1024, - new String[] { "h1" }); - allocator.sendRequest(event1); - - // send 1 more request with different resource req - ContainerRequestEvent event2 = createReq(jobId, 2, 2048, - new String[] { "h2" }); - allocator.sendRequest(event2); - - // this tells the scheduler about the requests - // as nodes are not added, no allocations - List assigned = allocator.schedule(); - dispatcher.await(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); - - // update resources in scheduler - nodeManager1.nodeHeartbeat(true); // Node heartbeat - nodeManager2.nodeHeartbeat(true); // Node heartbeat - nodeManager3.nodeHeartbeat(true); // Node heartbeat - dispatcher.await(); - - assigned = allocator.schedule(); - dispatcher.await(); - checkAssignments(new ContainerRequestEvent[] { event1, event2 }, - assigned, false); - } - - @Test - public void testMapReduceScheduling() throws Exception { - - LOG.info("Running testMapReduceScheduling"); - - Configuration conf = new Configuration(); - MyResourceManager rm = new MyResourceManager(conf); - rm.start(); - DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext() - .getDispatcher(); - - // Submit the application - RMApp app = rm.submitApp(1024); - dispatcher.await(); - - MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); - amNodeManager.nodeHeartbeat(true); - dispatcher.await(); - - ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() - .getAppAttemptId(); - rm.sendAMLaunched(appAttemptId); - dispatcher.await(); - - JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0); - Job mockJob = mock(Job.class); - when(mockJob.getReport()).thenReturn( - MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, - 0, 0, 0, 0, 0, 0)); - MyContainerAllocator allocator = new MyContainerAllocator(rm, conf, - appAttemptId, mockJob); - - // add resources to scheduler - MockNM nodeManager1 = rm.registerNode("h1:1234", 1024); - MockNM nodeManager2 = rm.registerNode("h2:1234", 10240); - MockNM nodeManager3 = rm.registerNode("h3:1234", 10240); - dispatcher.await(); - - // create the container request - // send MAP request - ContainerRequestEvent event1 = createReq(jobId, 1, 2048, new String[] { - "h1", "h2" }, true, false); - allocator.sendRequest(event1); - - // send REDUCE request - ContainerRequestEvent event2 = createReq(jobId, 2, 3000, - new String[] { "h1" }, false, true); - allocator.sendRequest(event2); - - // send MAP request - ContainerRequestEvent event3 = createReq(jobId, 3, 2048, - new String[] { "h3" }, false, false); - allocator.sendRequest(event3); - - // this tells the scheduler about the requests - // as nodes are not added, no allocations - List assigned = allocator.schedule(); - dispatcher.await(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); - - // update resources in scheduler - nodeManager1.nodeHeartbeat(true); // Node heartbeat - nodeManager2.nodeHeartbeat(true); // Node heartbeat - nodeManager3.nodeHeartbeat(true); // Node heartbeat - dispatcher.await(); - - assigned = allocator.schedule(); - dispatcher.await(); - checkAssignments(new ContainerRequestEvent[] { event1, event3 }, - assigned, false); - - // validate that no container is assigned to h1 as it doesn't have 2048 - for (TaskAttemptContainerAssignedEvent assig : assigned) { - Assert.assertFalse("Assigned count not correct", "h1".equals(assig - .getContainer().getNodeId().getHost())); - } - } - - private static class MyResourceManager extends MockRM { - - public MyResourceManager(Configuration conf) { - super(conf); - } - - @Override - protected Dispatcher createDispatcher() { - return new DrainDispatcher(); - } - - @Override - protected EventHandler createSchedulerEventDispatcher() { - // Dispatch inline for test sanity - return new EventHandler() { - @Override - public void handle(SchedulerEvent event) { - scheduler.handle(event); - } - }; - } - @Override - protected ResourceScheduler createScheduler() { - return new MyFifoScheduler(getRMContext()); - } - } - - private static class FakeJob extends JobImpl { - - public FakeJob(ApplicationAttemptId appAttemptID, Configuration conf, - int numMaps, int numReduces) { - super(appAttemptID, conf, null, null, null, null, null, null, null, - null); - this.jobId = MRBuilderUtils - .newJobId(appAttemptID.getApplicationId(), 0); - this.numMaps = numMaps; - this.numReduces = numReduces; - } - - private float setupProgress; - private float mapProgress; - private float reduceProgress; - private float cleanupProgress; - private final int numMaps; - private final int numReduces; - private JobId jobId; - - void setProgress(float setupProgress, float mapProgress, - float reduceProgress, float cleanupProgress) { - this.setupProgress = setupProgress; - this.mapProgress = mapProgress; - this.reduceProgress = reduceProgress; - this.cleanupProgress = cleanupProgress; - } - - @Override - public int getTotalMaps() { return this.numMaps; } - @Override - public int getTotalReduces() { return this.numReduces;} - - @Override - public JobReport getReport() { - return MRBuilderUtils.newJobReport(this.jobId, "job", "user", - JobState.RUNNING, 0, 0, this.setupProgress, this.mapProgress, - this.reduceProgress, this.cleanupProgress); - } - } - - @Test - public void testReportedAppProgress() throws Exception { - - LOG.info("Running testReportedAppProgress"); - - Configuration conf = new Configuration(); - MyResourceManager rm = new MyResourceManager(conf); - rm.start(); - DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext() - .getDispatcher(); - - // Submit the application - RMApp app = rm.submitApp(1024); - dispatcher.await(); - - MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); - amNodeManager.nodeHeartbeat(true); - dispatcher.await(); - - ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() - .getAppAttemptId(); - rm.sendAMLaunched(appAttemptId); - dispatcher.await(); - - FakeJob job = new FakeJob(appAttemptId, conf, 2, 2); - MyContainerAllocator allocator = new MyContainerAllocator(rm, conf, - appAttemptId, job); - - allocator.schedule(); // Send heartbeat - dispatcher.await(); - Assert.assertEquals(0.0, app.getProgress(), 0.0); - - job.setProgress(100, 10, 0, 0); - allocator.schedule(); - dispatcher.await(); - Assert.assertEquals(9.5f, app.getProgress(), 0.0); - - job.setProgress(100, 80, 0, 0); - allocator.schedule(); - dispatcher.await(); - Assert.assertEquals(41.0f, app.getProgress(), 0.0); - - job.setProgress(100, 100, 20, 0); - allocator.schedule(); - dispatcher.await(); - Assert.assertEquals(59.0f, app.getProgress(), 0.0); - - job.setProgress(100, 100, 100, 100); - allocator.schedule(); - dispatcher.await(); - Assert.assertEquals(100.0f, app.getProgress(), 0.0); - } - - @Test - public void testReportedAppProgressWithOnlyMaps() throws Exception { - - LOG.info("Running testReportedAppProgressWithOnlyMaps"); - - Configuration conf = new Configuration(); - MyResourceManager rm = new MyResourceManager(conf); - rm.start(); - DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext() - .getDispatcher(); - - // Submit the application - RMApp app = rm.submitApp(1024); - dispatcher.await(); - - MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); - amNodeManager.nodeHeartbeat(true); - dispatcher.await(); - - ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() - .getAppAttemptId(); - rm.sendAMLaunched(appAttemptId); - dispatcher.await(); - - FakeJob job = new FakeJob(appAttemptId, conf, 2, 0); - MyContainerAllocator allocator = new MyContainerAllocator(rm, conf, - appAttemptId, job); - - allocator.schedule(); // Send heartbeat - dispatcher.await(); - Assert.assertEquals(0.0, app.getProgress(), 0.0); - - job.setProgress(100, 10, 0, 0); - allocator.schedule(); - dispatcher.await(); - Assert.assertEquals(14f, app.getProgress(), 0.0); - - job.setProgress(100, 60, 0, 0); - allocator.schedule(); - dispatcher.await(); - Assert.assertEquals(59.0f, app.getProgress(), 0.0); - - job.setProgress(100, 100, 0, 100); - allocator.schedule(); - dispatcher.await(); - Assert.assertEquals(100.0f, app.getProgress(), 0.0); - } - - private static class MyFifoScheduler extends FifoScheduler { - - public MyFifoScheduler(RMContext rmContext) { - super(); - try { - reinitialize(new Configuration(), new ContainerTokenSecretManager(), - rmContext); - } catch (IOException ie) { - LOG.info("add application failed with ", ie); - assert (false); - } - } - - // override this to copy the objects otherwise FifoScheduler updates the - // numContainers in same objects as kept by RMContainerAllocator - @Override - public synchronized Allocation allocate( - ApplicationAttemptId applicationAttemptId, List ask, - List release) { - List askCopy = new ArrayList(); - for (ResourceRequest req : ask) { - ResourceRequest reqCopy = BuilderUtils.newResourceRequest(req - .getPriority(), req.getHostName(), req.getCapability(), req - .getNumContainers()); - askCopy.add(reqCopy); - } - return super.allocate(applicationAttemptId, askCopy, release); - } - } - - private ContainerRequestEvent createReq(JobId jobId, int taskAttemptId, - int memory, String[] hosts) { - return createReq(jobId, taskAttemptId, memory, hosts, false, false); - } - - private ContainerRequestEvent - createReq(JobId jobId, int taskAttemptId, int memory, String[] hosts, - boolean earlierFailedAttempt, boolean reduce) { - TaskId taskId; - if (reduce) { - taskId = MRBuilderUtils.newTaskId(jobId, 0, TaskType.REDUCE); - } else { - taskId = MRBuilderUtils.newTaskId(jobId, 0, TaskType.MAP); - } - TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, - taskAttemptId); - Resource containerNeed = BuilderUtils.newResource(memory); - if (earlierFailedAttempt) { - return ContainerRequestEvent - .createContainerRequestEventForFailedContainer(attemptId, - containerNeed); - } - return new ContainerRequestEvent(attemptId, containerNeed, hosts, - new String[] { NetworkTopology.DEFAULT_RACK }); - } - - private void checkAssignments(ContainerRequestEvent[] requests, - List assignments, - boolean checkHostMatch) { - Assert.assertNotNull("Container not assigned", assignments); - Assert.assertEquals("Assigned count not correct", requests.length, - assignments.size()); - - // check for uniqueness of containerIDs - Set containerIds = new HashSet(); - for (TaskAttemptContainerAssignedEvent assigned : assignments) { - containerIds.add(assigned.getContainer().getId()); - } - Assert.assertEquals("Assigned containers must be different", assignments - .size(), containerIds.size()); - - // check for all assignment - for (ContainerRequestEvent req : requests) { - TaskAttemptContainerAssignedEvent assigned = null; - for (TaskAttemptContainerAssignedEvent ass : assignments) { - if (ass.getTaskAttemptID().equals(req.getAttemptID())) { - assigned = ass; - break; - } - } - checkAssignment(req, assigned, checkHostMatch); - } - } - - private void checkAssignment(ContainerRequestEvent request, - TaskAttemptContainerAssignedEvent assigned, boolean checkHostMatch) { - Assert.assertNotNull("Nothing assigned to attempt " - + request.getAttemptID(), assigned); - Assert.assertEquals("assigned to wrong attempt", request.getAttemptID(), - assigned.getTaskAttemptID()); - if (checkHostMatch) { - Assert.assertTrue("Not assigned to requested host", Arrays.asList( - request.getHosts()).contains( - assigned.getContainer().getNodeId().toString())); - } - } - - // Mock RMContainerAllocator - // Instead of talking to remote Scheduler,uses the local Scheduler - private static class MyContainerAllocator extends RMContainerAllocator { - static final List events - = new ArrayList(); - - private MyResourceManager rm; - - @SuppressWarnings("rawtypes") - private static AppContext createAppContext( - ApplicationAttemptId appAttemptId, Job job) { - AppContext context = mock(AppContext.class); - ApplicationId appId = appAttemptId.getApplicationId(); - when(context.getApplicationID()).thenReturn(appId); - when(context.getApplicationAttemptId()).thenReturn(appAttemptId); - when(context.getJob(isA(JobId.class))).thenReturn(job); - when(context.getEventHandler()).thenReturn(new EventHandler() { - @Override - public void handle(Event event) { - // Only capture interesting events. - if (event instanceof TaskAttemptContainerAssignedEvent) { - events.add((TaskAttemptContainerAssignedEvent) event); - } - } - }); - return context; - } - - private static ClientService createMockClientService() { - ClientService service = mock(ClientService.class); - when(service.getBindAddress()).thenReturn( - NetUtils.createSocketAddr("localhost:4567")); - when(service.getHttpPort()).thenReturn(890); - return service; - } - - MyContainerAllocator(MyResourceManager rm, Configuration conf, - ApplicationAttemptId appAttemptId, Job job) { - super(createMockClientService(), createAppContext(appAttemptId, job)); - this.rm = rm; - super.init(conf); - super.start(); - } - - @Override - protected AMRMProtocol createSchedulerProxy() { - return this.rm.getApplicationMasterService(); - } - - @Override - protected void register() { - super.register(); - } - - @Override - protected void unregister() { - } - - @Override - protected Resource getMinContainerCapability() { - return BuilderUtils.newResource(1024); - } - - @Override - protected Resource getMaxContainerCapability() { - return BuilderUtils.newResource(10240); - } - - public void sendRequest(ContainerRequestEvent req) { - sendRequests(Arrays.asList(new ContainerRequestEvent[] { req })); - } - - public void sendRequests(List reqs) { - for (ContainerRequestEvent req : reqs) { - super.handle(req); - } - } - - // API to be used by tests - public List schedule() { - // run the scheduler - try { - super.heartbeat(); - } catch (Exception e) { - LOG.error("error in heartbeat ", e); - throw new YarnException(e); - } - - List result - = new ArrayList(events); - events.clear(); - return result; - } - - protected void startAllocatorThread() { - // override to NOT start thread - } - } - - public static void main(String[] args) throws Exception { - TestRMContainerAllocator t = new TestRMContainerAllocator(); - t.testSimple(); - t.testResource(); - t.testMapReduceScheduling(); - t.testReportedAppProgress(); - t.testReportedAppProgressWithOnlyMaps(); - } +// private static final Log LOG = LogFactory.getLog(TestRMContainerAllocator.class); +// private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); +// +// @BeforeClass +// public static void preTests() { +// DefaultMetricsSystem.shutdown(); +// } +// +// @Test +// public void testSimple() throws Exception { +// FifoScheduler scheduler = createScheduler(); +// LocalRMContainerAllocator allocator = new LocalRMContainerAllocator( +// scheduler, new Configuration()); +// +// //add resources to scheduler +// RMNode nodeManager1 = addNode(scheduler, "h1", 10240); +// RMNode nodeManager2 = addNode(scheduler, "h2", 10240); +// RMNode nodeManager3 = addNode(scheduler, "h3", 10240); +// +// //create the container request +// ContainerRequestEvent event1 = +// createReq(1, 1024, new String[]{"h1"}); +// allocator.sendRequest(event1); +// +// //send 1 more request with different resource req +// ContainerRequestEvent event2 = createReq(2, 1024, new String[]{"h2"}); +// allocator.sendRequest(event2); +// +// //this tells the scheduler about the requests +// //as nodes are not added, no allocations +// List assigned = allocator.schedule(); +// Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); +// +// //send another request with different resource and priority +// ContainerRequestEvent event3 = createReq(3, 1024, new String[]{"h3"}); +// allocator.sendRequest(event3); +// +// //this tells the scheduler about the requests +// //as nodes are not added, no allocations +// assigned = allocator.schedule(); +// Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); +// +// //update resources in scheduler +// scheduler.nodeUpdate(nodeManager1); // Node heartbeat +// scheduler.nodeUpdate(nodeManager2); // Node heartbeat +// scheduler.nodeUpdate(nodeManager3); // Node heartbeat +// +// +// assigned = allocator.schedule(); +// checkAssignments( +// new ContainerRequestEvent[]{event1, event2, event3}, assigned, false); +// } +// +// //TODO: Currently Scheduler seems to have bug where it does not work +// //for Application asking for containers with different capabilities. +// //@Test +// public void testResource() throws Exception { +// FifoScheduler scheduler = createScheduler(); +// LocalRMContainerAllocator allocator = new LocalRMContainerAllocator( +// scheduler, new Configuration()); +// +// //add resources to scheduler +// RMNode nodeManager1 = addNode(scheduler, "h1", 10240); +// RMNode nodeManager2 = addNode(scheduler, "h2", 10240); +// RMNode nodeManager3 = addNode(scheduler, "h3", 10240); +// +// //create the container request +// ContainerRequestEvent event1 = +// createReq(1, 1024, new String[]{"h1"}); +// allocator.sendRequest(event1); +// +// //send 1 more request with different resource req +// ContainerRequestEvent event2 = createReq(2, 2048, new String[]{"h2"}); +// allocator.sendRequest(event2); +// +// //this tells the scheduler about the requests +// //as nodes are not added, no allocations +// List assigned = allocator.schedule(); +// Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); +// +// //update resources in scheduler +// scheduler.nodeUpdate(nodeManager1); // Node heartbeat +// scheduler.nodeUpdate(nodeManager2); // Node heartbeat +// scheduler.nodeUpdate(nodeManager3); // Node heartbeat +// +// assigned = allocator.schedule(); +// checkAssignments( +// new ContainerRequestEvent[]{event1, event2}, assigned, false); +// } +// +// @Test +// public void testMapReduceScheduling() throws Exception { +// FifoScheduler scheduler = createScheduler(); +// Configuration conf = new Configuration(); +// LocalRMContainerAllocator allocator = new LocalRMContainerAllocator( +// scheduler, conf); +// +// //add resources to scheduler +// RMNode nodeManager1 = addNode(scheduler, "h1", 1024); +// RMNode nodeManager2 = addNode(scheduler, "h2", 10240); +// RMNode nodeManager3 = addNode(scheduler, "h3", 10240); +// +// //create the container request +// //send MAP request +// ContainerRequestEvent event1 = +// createReq(1, 2048, new String[]{"h1", "h2"}, true, false); +// allocator.sendRequest(event1); +// +// //send REDUCE request +// ContainerRequestEvent event2 = createReq(2, 3000, new String[]{"h1"}, false, true); +// allocator.sendRequest(event2); +// +// //send MAP request +// ContainerRequestEvent event3 = createReq(3, 2048, new String[]{"h3"}, false, false); +// allocator.sendRequest(event3); +// +// //this tells the scheduler about the requests +// //as nodes are not added, no allocations +// List assigned = allocator.schedule(); +// Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); +// +// //update resources in scheduler +// scheduler.nodeUpdate(nodeManager1); // Node heartbeat +// scheduler.nodeUpdate(nodeManager2); // Node heartbeat +// scheduler.nodeUpdate(nodeManager3); // Node heartbeat +// +// assigned = allocator.schedule(); +// checkAssignments( +// new ContainerRequestEvent[]{event1, event3}, assigned, false); +// +// //validate that no container is assigned to h1 as it doesn't have 2048 +// for (TaskAttemptContainerAssignedEvent assig : assigned) { +// Assert.assertFalse("Assigned count not correct", +// "h1".equals(assig.getContainer().getNodeId().getHost())); +// } +// } +// +// +// +// private RMNode addNode(FifoScheduler scheduler, +// String nodeName, int memory) { +// NodeId nodeId = recordFactory.newRecordInstance(NodeId.class); +// nodeId.setHost(nodeName); +// nodeId.setPort(1234); +// Resource resource = recordFactory.newRecordInstance(Resource.class); +// resource.setMemory(memory); +// RMNode nodeManager = new RMNodeImpl(nodeId, null, nodeName, 0, 0, +// ResourceTrackerService.resolve(nodeName), resource); +// scheduler.addNode(nodeManager); // Node registration +// return nodeManager; +// } +// +// private FifoScheduler createScheduler() throws YarnRemoteException { +// FifoScheduler fsc = new FifoScheduler() { +// //override this to copy the objects +// //otherwise FifoScheduler updates the numContainers in same objects as kept by +// //RMContainerAllocator +// +// @Override +// public synchronized void allocate(ApplicationAttemptId applicationId, +// List ask) { +// List askCopy = new ArrayList(); +// for (ResourceRequest req : ask) { +// ResourceRequest reqCopy = recordFactory.newRecordInstance(ResourceRequest.class); +// reqCopy.setPriority(req.getPriority()); +// reqCopy.setHostName(req.getHostName()); +// reqCopy.setCapability(req.getCapability()); +// reqCopy.setNumContainers(req.getNumContainers()); +// askCopy.add(reqCopy); +// } +// super.allocate(applicationId, askCopy); +// } +// }; +// try { +// fsc.reinitialize(new Configuration(), new ContainerTokenSecretManager(), null); +// fsc.addApplication(recordFactory.newRecordInstance(ApplicationId.class), +// recordFactory.newRecordInstance(ApplicationMaster.class), +// "test", null, null, StoreFactory.createVoidAppStore()); +// } catch(IOException ie) { +// LOG.info("add application failed with ", ie); +// assert(false); +// } +// return fsc; +// } +// +// private ContainerRequestEvent createReq( +// int attemptid, int memory, String[] hosts) { +// return createReq(attemptid, memory, hosts, false, false); +// } +// +// private ContainerRequestEvent createReq( +// int attemptid, int memory, String[] hosts, boolean earlierFailedAttempt, boolean reduce) { +// ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class); +// appId.setClusterTimestamp(0); +// appId.setId(0); +// JobId jobId = recordFactory.newRecordInstance(JobId.class); +// jobId.setAppId(appId); +// jobId.setId(0); +// TaskId taskId = recordFactory.newRecordInstance(TaskId.class); +// taskId.setId(0); +// taskId.setJobId(jobId); +// if (reduce) { +// taskId.setTaskType(TaskType.REDUCE); +// } else { +// taskId.setTaskType(TaskType.MAP); +// } +// TaskAttemptId attemptId = recordFactory.newRecordInstance(TaskAttemptId.class); +// attemptId.setId(attemptid); +// attemptId.setTaskId(taskId); +// Resource containerNeed = recordFactory.newRecordInstance(Resource.class); +// containerNeed.setMemory(memory); +// if (earlierFailedAttempt) { +// return ContainerRequestEvent. +// createContainerRequestEventForFailedContainer(attemptId, containerNeed); +// } +// return new ContainerRequestEvent(attemptId, +// containerNeed, +// hosts, new String[] {NetworkTopology.DEFAULT_RACK}); +// } +// +// private void checkAssignments(ContainerRequestEvent[] requests, +// List assignments, +// boolean checkHostMatch) { +// Assert.assertNotNull("Container not assigned", assignments); +// Assert.assertEquals("Assigned count not correct", +// requests.length, assignments.size()); +// +// //check for uniqueness of containerIDs +// Set containerIds = new HashSet(); +// for (TaskAttemptContainerAssignedEvent assigned : assignments) { +// containerIds.add(assigned.getContainer().getId()); +// } +// Assert.assertEquals("Assigned containers must be different", +// assignments.size(), containerIds.size()); +// +// //check for all assignment +// for (ContainerRequestEvent req : requests) { +// TaskAttemptContainerAssignedEvent assigned = null; +// for (TaskAttemptContainerAssignedEvent ass : assignments) { +// if (ass.getTaskAttemptID().equals(req.getAttemptID())){ +// assigned = ass; +// break; +// } +// } +// checkAssignment(req, assigned, checkHostMatch); +// } +// } +// +// private void checkAssignment(ContainerRequestEvent request, +// TaskAttemptContainerAssignedEvent assigned, boolean checkHostMatch) { +// Assert.assertNotNull("Nothing assigned to attempt " + request.getAttemptID(), +// assigned); +// Assert.assertEquals("assigned to wrong attempt", request.getAttemptID(), +// assigned.getTaskAttemptID()); +// if (checkHostMatch) { +// Assert.assertTrue("Not assigned to requested host", Arrays.asList( +// request.getHosts()).contains( +// assigned.getContainer().getNodeId().toString())); +// } +// +// } +// +// //Mock RMContainerAllocator +// //Instead of talking to remote Scheduler,uses the local Scheduler +// public static class LocalRMContainerAllocator extends RMContainerAllocator { +// private static final List events = +// new ArrayList(); +// +// public static class AMRMProtocolImpl implements AMRMProtocol { +// +// private ResourceScheduler resourceScheduler; +// +// public AMRMProtocolImpl(ResourceScheduler resourceScheduler) { +// this.resourceScheduler = resourceScheduler; +// } +// +// @Override +// public RegisterApplicationMasterResponse registerApplicationMaster(RegisterApplicationMasterRequest request) throws YarnRemoteException { +// RegisterApplicationMasterResponse response = recordFactory.newRecordInstance(RegisterApplicationMasterResponse.class); +// return response; +// } +// +// public AllocateResponse allocate(AllocateRequest request) throws YarnRemoteException { +// List ask = request.getAskList(); +// List release = request.getReleaseList(); +// try { +// AMResponse response = recordFactory.newRecordInstance(AMResponse.class); +// Allocation allocation = resourceScheduler.allocate(request.getApplicationAttemptId(), ask); +// response.addAllNewContainers(allocation.getContainers()); +// response.setAvailableResources(allocation.getResourceLimit()); +// AllocateResponse allocateResponse = recordFactory.newRecordInstance(AllocateResponse.class); +// allocateResponse.setAMResponse(response); +// return allocateResponse; +// } catch(IOException ie) { +// throw RPCUtil.getRemoteException(ie); +// } +// } +// +// @Override +// public FinishApplicationMasterResponse finishApplicationMaster(FinishApplicationMasterRequest request) throws YarnRemoteException { +// FinishApplicationMasterResponse response = recordFactory.newRecordInstance(FinishApplicationMasterResponse.class); +// return response; +// } +// +// } +// +// private ResourceScheduler scheduler; +// LocalRMContainerAllocator(ResourceScheduler scheduler, Configuration conf) { +// super(null, new TestContext(events)); +// this.scheduler = scheduler; +// super.init(conf); +// super.start(); +// } +// +// protected AMRMProtocol createSchedulerProxy() { +// return new AMRMProtocolImpl(scheduler); +// } +// +// @Override +// protected void register() {} +// @Override +// protected void unregister() {} +// +// @Override +// protected Resource getMinContainerCapability() { +// Resource res = recordFactory.newRecordInstance(Resource.class); +// res.setMemory(1024); +// return res; +// } +// +// @Override +// protected Resource getMaxContainerCapability() { +// Resource res = recordFactory.newRecordInstance(Resource.class); +// res.setMemory(10240); +// return res; +// } +// +// public void sendRequest(ContainerRequestEvent req) { +// sendRequests(Arrays.asList(new ContainerRequestEvent[]{req})); +// } +// +// public void sendRequests(List reqs) { +// for (ContainerRequestEvent req : reqs) { +// handle(req); +// } +// } +// +// //API to be used by tests +// public List schedule() { +// //run the scheduler +// try { +// heartbeat(); +// } catch (Exception e) { +// LOG.error("error in heartbeat ", e); +// throw new YarnException(e); +// } +// +// List result = new ArrayList(events); +// events.clear(); +// return result; +// } +// +// protected void startAllocatorThread() { +// //override to NOT start thread +// } +// +// static class TestContext implements AppContext { +// private List events; +// TestContext(List events) { +// this.events = events; +// } +// @Override +// public Map getAllJobs() { +// return null; +// } +// @Override +// public ApplicationAttemptId getApplicationAttemptId() { +// return recordFactory.newRecordInstance(ApplicationAttemptId.class); +// } +// @Override +// public ApplicationId getApplicationID() { +// return recordFactory.newRecordInstance(ApplicationId.class); +// } +// @Override +// public EventHandler getEventHandler() { +// return new EventHandler() { +// @Override +// public void handle(Event event) { +// events.add((TaskAttemptContainerAssignedEvent) event); +// } +// }; +// } +// @Override +// public Job getJob(JobId jobID) { +// return null; +// } +// +// @Override +// public String getUser() { +// return null; +// } +// +// @Override +// public Clock getClock() { +// return null; +// } +// +// @Override +// public String getApplicationName() { +// return null; +// } +// +// @Override +// public long getStartTime() { +// return 0; +// } +// } +// } +// +// public static void main(String[] args) throws Exception { +// TestRMContainerAllocator t = new TestRMContainerAllocator(); +// t.testSimple(); +// //t.testResource(); +// t.testMapReduceScheduling(); +// } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java index 9f221e6354a..a678e4660e7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java @@ -47,7 +47,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationState; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.QueueACL; -import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; @@ -281,28 +280,16 @@ public class TypeConverter { } public static org.apache.hadoop.mapred.JobStatus fromYarn( - JobReport jobreport, String jobFile) { + JobReport jobreport, String jobFile, String trackingUrl) { JobPriority jobPriority = JobPriority.NORMAL; - org.apache.hadoop.mapred.JobStatus jobStatus = - new org.apache.hadoop.mapred.JobStatus(fromYarn(jobreport.getJobId()), - jobreport.getSetupProgress(), jobreport.getMapProgress(), - jobreport.getReduceProgress(), jobreport.getCleanupProgress(), - fromYarn(jobreport.getJobState()), - jobPriority, jobreport.getUser(), jobreport.getJobName(), - jobFile, jobreport.getTrackingUrl()); - jobStatus.setFailureInfo(jobreport.getDiagnostics()); - return jobStatus; + return new org.apache.hadoop.mapred.JobStatus(fromYarn(jobreport.getJobId()), + jobreport.getSetupProgress(), jobreport.getMapProgress(), + jobreport.getReduceProgress(), jobreport.getCleanupProgress(), + fromYarn(jobreport.getJobState()), + jobPriority, jobreport.getUser(), jobreport.getJobName(), + jobFile, trackingUrl); } - public static org.apache.hadoop.mapreduce.QueueState fromYarn( - QueueState state) { - org.apache.hadoop.mapreduce.QueueState qState = - org.apache.hadoop.mapreduce.QueueState.getState( - state.toString().toLowerCase()); - return qState; - } - - public static int fromYarn(JobState state) { switch (state) { case NEW: @@ -425,7 +412,6 @@ public class TypeConverter { ); jobStatus.setSchedulingInfo(trackingUrl); // Set AM tracking url jobStatus.setStartTime(application.getStartTime()); - jobStatus.setFailureInfo(application.getDiagnostics()); return jobStatus; } @@ -445,9 +431,9 @@ public class TypeConverter { public static QueueInfo fromYarn(org.apache.hadoop.yarn.api.records.QueueInfo queueInfo, Configuration conf) { - return new QueueInfo(queueInfo.getQueueName(),queueInfo.toString(), - fromYarn(queueInfo.getQueueState()), TypeConverter.fromYarnApps( - queueInfo.getApplications(), conf)); + return new QueueInfo(queueInfo.getQueueName(), + queueInfo.toString(), QueueState.RUNNING, + TypeConverter.fromYarnApps(queueInfo.getApplications(), conf)); } public static QueueInfo[] fromYarnQueueInfo( 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 new file mode 100644 index 00000000000..6ac05361dce --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/MRConstants.java @@ -0,0 +1,50 @@ +/** +* 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 org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface MRConstants { + // This should be the directory where splits file gets localized on the node + // running ApplicationMaster. + public static final String JOB_SUBMIT_DIR = "jobSubmitDir"; + + // This should be the name of the localized job-configuration file on the node + // running ApplicationMaster and Task + public static final String JOB_CONF_FILE = "job.xml"; + // This should be the name of the localized job-jar file on the node running + // individual containers/tasks. + public static final String JOB_JAR = "job.jar"; + + public static final String HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME = + "hadoop-mapreduce-client-app-0.24.0-SNAPSHOT.jar"; + + public static final String YARN_MAPREDUCE_APP_JAR_PATH = + "$YARN_HOME/modules/" + HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME; + + // 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 + // Containers. + public static final String APPLICATION_TOKENS_FILE = "appTokens"; +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/JobReport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/JobReport.java index 0bfc9db3ed4..fb585e8dd27 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/JobReport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/JobReport.java @@ -29,8 +29,6 @@ public interface JobReport { public abstract long getFinishTime(); public abstract String getUser(); public abstract String getJobName(); - public abstract String getTrackingUrl(); - public abstract String getDiagnostics(); public abstract void setJobId(JobId jobId); public abstract void setJobState(JobState jobState); @@ -42,6 +40,4 @@ public interface JobReport { public abstract void setFinishTime(long finishTime); public abstract void setUser(String user); public abstract void setJobName(String jobName); - public abstract void setTrackingUrl(String trackingUrl); - public abstract void setDiagnostics(String diagnostics); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/JobReportPBImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/JobReportPBImpl.java index c5d2527a9da..a4033e695f2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/JobReportPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/JobReportPBImpl.java @@ -206,30 +206,6 @@ public class JobReportPBImpl extends ProtoBase implements JobRep builder.setJobName((jobName)); } - @Override - public String getTrackingUrl() { - JobReportProtoOrBuilder p = viaProto ? proto : builder; - return (p.getTrackingUrl()); - } - - @Override - public void setTrackingUrl(String trackingUrl) { - maybeInitBuilder(); - builder.setTrackingUrl(trackingUrl); - } - - @Override - public String getDiagnostics() { - JobReportProtoOrBuilder p = viaProto ? proto : builder; - return p.getDiagnostics(); - } - - @Override - public void setDiagnostics(String diagnostics) { - maybeInitBuilder(); - builder.setDiagnostics(diagnostics); - } - private JobIdPBImpl convertFromProtoFormat(JobIdProto p) { return new JobIdPBImpl(p); } 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 e57cf8d3c63..dcddd126cc2 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 @@ -489,7 +489,7 @@ public class JobHistoryUtils { sb.append(address.getHostName()); } sb.append(":").append(address.getPort()); - sb.append("/jobhistory/job/"); // TODO This will change when the history server + sb.append("/yarn/job/"); // TODO This will change when the history server // understands apps. // TOOD Use JobId toString once UI stops using _id_id sb.append("job_").append(appId.getClusterTimestamp()); 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 9094da39ba3..68499497ac3 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 @@ -39,14 +39,14 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.filecache.DistributedCache; +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.util.Shell.ShellCommandExecutor; import org.apache.hadoop.yarn.YarnException; -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; -import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; @@ -167,7 +167,7 @@ public class MRApps extends Apps { return TaskAttemptStateUI.valueOf(attemptStateStr); } - private static void setMRFrameworkClasspath( + public static void setInitialClasspath( Map environment) throws IOException { InputStream classpathFileStream = null; BufferedReader reader = null; @@ -182,17 +182,30 @@ public class MRApps extends Apps { reader = new BufferedReader(new InputStreamReader(classpathFileStream)); String cp = reader.readLine(); if (cp != null) { - addToEnvironment(environment, Environment.CLASSPATH.name(), cp.trim()); + addToClassPath(environment, cp.trim()); } // Put the file itself on classpath for tasks. - addToEnvironment( - environment, - Environment.CLASSPATH.name(), + addToClassPath(environment, thisClassLoader.getResource(mrAppGeneratedClasspathFile).getFile()); - // Add standard Hadoop classes - for (String c : ApplicationConstants.APPLICATION_CLASSPATH) { - addToEnvironment(environment, Environment.CLASSPATH.name(), c); + // If runtime env is different. + if (System.getenv().get("YARN_HOME") != null) { + ShellCommandExecutor exec = + new ShellCommandExecutor(new String[] { + System.getenv().get("YARN_HOME") + "/bin/yarn", + "classpath" }); + exec.execute(); + addToClassPath(environment, exec.getOutput().trim()); + } + + // Get yarn mapreduce-app classpath + if (System.getenv().get("HADOOP_MAPRED_HOME")!= null) { + ShellCommandExecutor exec = + new ShellCommandExecutor(new String[] { + System.getenv().get("HADOOP_MAPRED_HOME") + "/bin/mapred", + "classpath" }); + exec.execute(); + addToClassPath(environment, exec.getOutput().trim()); } } finally { if (classpathFileStream != null) { @@ -204,35 +217,20 @@ public class MRApps extends Apps { } // TODO: Remove duplicates. } - - private static final String SYSTEM_PATH_SEPARATOR = - System.getProperty("path.separator"); - public static void addToEnvironment( - Map environment, - String variable, String value) { - String val = environment.get(variable); - if (val == null) { - val = value; + public static void addToClassPath( + Map environment, String fileName) { + String classpath = environment.get(CLASSPATH); + if (classpath == null) { + classpath = fileName; } else { - val = val + SYSTEM_PATH_SEPARATOR + value; + classpath = classpath + ":" + fileName; } - environment.put(variable, val); + environment.put(CLASSPATH, classpath); } - public static void setClasspath(Map environment) - throws IOException { - MRApps.addToEnvironment( - environment, - Environment.CLASSPATH.name(), - MRJobConfig.JOB_JAR); - MRApps.addToEnvironment( - environment, - Environment.CLASSPATH.name(), - Environment.PWD.$() + Path.SEPARATOR + "*"); - MRApps.setMRFrameworkClasspath(environment); - } - + public static final String CLASSPATH = "CLASSPATH"; + private static final String STAGING_CONSTANT = ".staging"; public static Path getStagingAreaDir(Configuration conf, String user) { return new Path( @@ -243,7 +241,7 @@ public class MRApps extends Apps { public static String getJobFile(Configuration conf, String user, org.apache.hadoop.mapreduce.JobID jobId) { Path jobFile = new Path(MRApps.getStagingAreaDir(conf, user), - jobId.toString() + Path.SEPARATOR + MRJobConfig.JOB_CONF_FILE); + jobId.toString() + Path.SEPARATOR + MRConstants.JOB_CONF_FILE); return jobFile.toString(); } @@ -262,11 +260,12 @@ public class MRApps extends Apps { public static void setupDistributedCache( Configuration conf, - Map localResources) + Map localResources, + Map env) throws IOException { // Cache archives - parseDistributedCacheArtifacts(conf, localResources, + parseDistributedCacheArtifacts(conf, localResources, env, LocalResourceType.ARCHIVE, DistributedCache.getCacheArchives(conf), parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)), @@ -276,7 +275,7 @@ public class MRApps extends Apps { // Cache files parseDistributedCacheArtifacts(conf, - localResources, + localResources, env, LocalResourceType.FILE, DistributedCache.getCacheFiles(conf), parseTimeStamps(DistributedCache.getFileTimestamps(conf)), @@ -291,6 +290,7 @@ public class MRApps extends Apps { private static void parseDistributedCacheArtifacts( Configuration conf, Map localResources, + Map env, LocalResourceType type, URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[], Path[] pathsToPutOnClasspath) throws IOException { @@ -339,6 +339,9 @@ public class MRApps extends Apps { : LocalResourceVisibility.PRIVATE, sizes[i], timestamps[i]) ); + if (classPaths.containsKey(u.getPath())) { + MRApps.addToClassPath(env, linkName); + } } } } @@ -355,42 +358,6 @@ public class MRApps extends Apps { } return result; } - - public static void setEnvFromInputString(Map env, - String envString) { - if (envString != null && envString.length() > 0) { - String childEnvs[] = envString.split(","); - for (String cEnv : childEnvs) { - String[] parts = cEnv.split("="); // split on '=' - String value = env.get(parts[0]); - - if (value != null) { - // Replace $env with the child's env constructed by NM's - // For example: LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp - value = parts[1].replace("$" + parts[0], value); - } else { - // example PATH=$PATH:/tmp - value = System.getenv(parts[0]); - if (value != null) { - // the env key is present in the tt's env - value = parts[1].replace("$" + parts[0], value); - } else { - // check for simple variable substitution - // for e.g. ROOT=$HOME - String envValue = System.getenv(parts[1].substring(1)); - if (envValue != null) { - value = envValue; - } else { - // the env key is note present anywhere .. simply set it - // example X=$X:/tmp or X=/tmp - value = parts[1].replace("$" + parts[0], ""); - } - } - } - addToEnvironment(env, parts[0], value); - } - } - } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRBuilderUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRBuilderUtils.java index d710a6f7b88..c429ca55b51 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRBuilderUtils.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRBuilderUtils.java @@ -19,25 +19,27 @@ package org.apache.hadoop.mapreduce.v2.util; 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.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.util.Records; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; public class MRBuilderUtils { + private static final RecordFactory recordFactory = RecordFactoryProvider + .getRecordFactory(null); + public static JobId newJobId(ApplicationId appId, int id) { - JobId jobId = Records.newRecord(JobId.class); + JobId jobId = recordFactory.newRecordInstance(JobId.class); jobId.setAppId(appId); jobId.setId(id); return jobId; } public static TaskId newTaskId(JobId jobId, int id, TaskType taskType) { - TaskId taskId = Records.newRecord(TaskId.class); + TaskId taskId = recordFactory.newRecordInstance(TaskId.class); taskId.setJobId(jobId); taskId.setId(id); taskId.setTaskType(taskType); @@ -46,27 +48,9 @@ public class MRBuilderUtils { public static TaskAttemptId newTaskAttemptId(TaskId taskId, int attemptId) { TaskAttemptId taskAttemptId = - Records.newRecord(TaskAttemptId.class); + recordFactory.newRecordInstance(TaskAttemptId.class); taskAttemptId.setTaskId(taskId); taskAttemptId.setId(attemptId); return taskAttemptId; } - - public static JobReport newJobReport(JobId jobId, String jobName, - String userName, JobState state, long startTime, long finishTime, - float setupProgress, float mapProgress, float reduceProgress, - float cleanupProgress) { - JobReport report = Records.newRecord(JobReport.class); - report.setJobId(jobId); - report.setJobName(jobName); - report.setUser(userName); - report.setJobState(state); - report.setStartTime(startTime); - report.setFinishTime(finishTime); - report.setSetupProgress(setupProgress); - report.setCleanupProgress(cleanupProgress); - report.setMapProgress(mapProgress); - report.setReduceProgress(reduceProgress); - return report; - } } \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/proto/mr_protos.proto b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/proto/mr_protos.proto index 29184da4868..7d8d1b2e0b8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/proto/mr_protos.proto +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/proto/mr_protos.proto @@ -143,8 +143,6 @@ message JobReportProto { optional int64 finish_time = 8; optional string user = 9; optional string jobName = 10; - optional string trackingUrl = 11; - optional string diagnostics = 12; } enum TaskAttemptCompletionEventStatusProto { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java index 1aeae987c80..bda7fb9d658 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java @@ -19,14 +19,11 @@ package org.apache.hadoop.mapreduce; import junit.framework.Assert; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationState; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.QueueInfoPBImpl; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import org.junit.Test; @@ -70,14 +67,4 @@ public class TestTypeConverter { Assert.assertEquals("jobId set incorrectly", 6789, status.getJobID().getId()); Assert.assertEquals("state set incorrectly", JobStatus.State.KILLED, status.getState()); } - - @Test - public void testFromYarnQueueInfo() { - org.apache.hadoop.yarn.api.records.QueueInfo queueInfo = new QueueInfoPBImpl(); - queueInfo.setQueueState(org.apache.hadoop.yarn.api.records.QueueState.STOPPED); - org.apache.hadoop.mapreduce.QueueInfo returned = - TypeConverter.fromYarn(queueInfo, new Configuration()); - Assert.assertEquals("queueInfo translation didn't work.", - returned.getState().toString(), queueInfo.getQueueState().toString().toLowerCase()); - } } 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 11589980625..7a2ee00a92d 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 @@ -25,6 +25,7 @@ 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; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; +import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.util.MRApps; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -114,8 +115,7 @@ public class TestMRApps { @Test public void testGetJobFileWithUser() { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/my/path/to/staging"); - String jobFile = MRApps.getJobFile(conf, "dummy-user", - new JobID("dummy-job", 12345)); + 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.", "/my/path/to/staging/dummy-user/.staging/job_dummy-job_12345/job.xml", jobFile); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/BackupStore.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/BackupStore.java index f409d2298eb..026793c5374 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/BackupStore.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/BackupStore.java @@ -41,7 +41,6 @@ import org.apache.hadoop.mapred.IFile.Reader; import org.apache.hadoop.mapred.IFile.Writer; import org.apache.hadoop.mapred.Merger.Segment; import org.apache.hadoop.mapreduce.MRConfig; -import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TaskAttemptID; /** @@ -561,7 +560,7 @@ public class BackupStore { private Writer createSpillFile() throws IOException { Path tmp = - new Path(MRJobConfig.OUTPUT + "/backup_" + tid.getId() + "_" + new Path(Constants.OUTPUT + "/backup_" + tid.getId() + "_" + (spillNumber++) + ".out"); LOG.info("Created file: " + tmp); 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/RMAppFailedAttemptEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Constants.java similarity index 64% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppFailedAttemptEvent.java rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Constants.java index 111c6acc41b..e8a202ed44b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppFailedAttemptEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Constants.java @@ -16,21 +16,12 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.resourcemanager.rmapp; +package org.apache.hadoop.mapred; -import org.apache.hadoop.yarn.api.records.ApplicationId; - -public class RMAppFailedAttemptEvent extends RMAppEvent { - - private final String diagnostics; - - public RMAppFailedAttemptEvent(ApplicationId appId, RMAppEventType event, - String diagnostics) { - super(appId, event); - this.diagnostics = diagnostics; - } - - public String getDiagnostics() { - return this.diagnostics; - } +public class Constants { + static final String OUTPUT = "output"; + public static final String HADOOP_WORK_DIR = "HADOOP_WORK_DIR"; + public static final String JOBFILE = "job.xml"; + public static final String STDOUT_LOGFILE_ENV = "STDOUT_LOGFILE_ENV"; + public static final String STDERR_LOGFILE_ENV = "STDERR_LOGFILE_ENV"; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java index b489d41b17c..49d12d764d5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java @@ -348,7 +348,6 @@ public class JobConf extends Configuration { */ public static final Level DEFAULT_LOG_LEVEL = Level.INFO; - /** * Construct a map/reduce job configuration. */ diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java index e5add2139f5..90b68872ff4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java @@ -321,10 +321,6 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus { super.setJobACLs(acls); } - public synchronized void setFailureInfo(String failureInfo) { - super.setFailureInfo(failureInfo); - } - /** * Set the priority of the job, defaulting to NORMAL. * @param jp new job priority diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MRConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MRConstants.java index 3d7363e5faa..e2c16fbfac1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MRConstants.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MRConstants.java @@ -17,16 +17,11 @@ */ package org.apache.hadoop.mapred; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceStability.Unstable; - /******************************* * Some handy constants * *******************************/ -@Private -@Unstable -public interface MRConstants { +interface MRConstants { // // Timeouts, constants // @@ -58,6 +53,5 @@ public interface MRConstants { */ public static final String FOR_REDUCE_TASK = "for-reduce-task"; - /** Used in MRv1, mostly in TaskTracker code **/ public static final String WORKDIR = "work"; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java index a9e25f287d4..e81e11d3fb6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java @@ -27,7 +27,6 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.MRConfig; -import org.apache.hadoop.mapreduce.MRJobConfig; /** * Manipulate the working area for the transient store for maps and reduces. @@ -55,7 +54,7 @@ public class MROutputFiles extends MapOutputFile { @Override public Path getOutputFile() throws IOException { - return lDirAlloc.getLocalPathToRead(MRJobConfig.OUTPUT + Path.SEPARATOR + return lDirAlloc.getLocalPathToRead(Constants.OUTPUT + Path.SEPARATOR + MAP_OUTPUT_FILENAME_STRING, getConf()); } @@ -69,7 +68,7 @@ public class MROutputFiles extends MapOutputFile { @Override public Path getOutputFileForWrite(long size) throws IOException { - return lDirAlloc.getLocalPathForWrite(MRJobConfig.OUTPUT + Path.SEPARATOR + return lDirAlloc.getLocalPathForWrite(Constants.OUTPUT + Path.SEPARATOR + MAP_OUTPUT_FILENAME_STRING, size, getConf()); } @@ -90,7 +89,7 @@ public class MROutputFiles extends MapOutputFile { @Override public Path getOutputIndexFile() throws IOException { - return lDirAlloc.getLocalPathToRead(MRJobConfig.OUTPUT + Path.SEPARATOR + return lDirAlloc.getLocalPathToRead(Constants.OUTPUT + Path.SEPARATOR + MAP_OUTPUT_FILENAME_STRING + MAP_OUTPUT_INDEX_SUFFIX_STRING, getConf()); } @@ -105,7 +104,7 @@ public class MROutputFiles extends MapOutputFile { @Override public Path getOutputIndexFileForWrite(long size) throws IOException { - return lDirAlloc.getLocalPathForWrite(MRJobConfig.OUTPUT + Path.SEPARATOR + return lDirAlloc.getLocalPathForWrite(Constants.OUTPUT + Path.SEPARATOR + MAP_OUTPUT_FILENAME_STRING + MAP_OUTPUT_INDEX_SUFFIX_STRING, size, getConf()); } @@ -129,7 +128,7 @@ public class MROutputFiles extends MapOutputFile { @Override public Path getSpillFile(int spillNumber) throws IOException { - return lDirAlloc.getLocalPathToRead(MRJobConfig.OUTPUT + "/spill" + return lDirAlloc.getLocalPathToRead(Constants.OUTPUT + "/spill" + spillNumber + ".out", getConf()); } @@ -144,7 +143,7 @@ public class MROutputFiles extends MapOutputFile { @Override public Path getSpillFileForWrite(int spillNumber, long size) throws IOException { - return lDirAlloc.getLocalPathForWrite(MRJobConfig.OUTPUT + "/spill" + return lDirAlloc.getLocalPathForWrite(Constants.OUTPUT + "/spill" + spillNumber + ".out", size, getConf()); } @@ -158,7 +157,7 @@ public class MROutputFiles extends MapOutputFile { @Override public Path getSpillIndexFile(int spillNumber) throws IOException { - return lDirAlloc.getLocalPathToRead(MRJobConfig.OUTPUT + "/spill" + return lDirAlloc.getLocalPathToRead(Constants.OUTPUT + "/spill" + spillNumber + ".out.index", getConf()); } @@ -173,7 +172,7 @@ public class MROutputFiles extends MapOutputFile { @Override public Path getSpillIndexFileForWrite(int spillNumber, long size) throws IOException { - return lDirAlloc.getLocalPathForWrite(MRJobConfig.OUTPUT + "/spill" + return lDirAlloc.getLocalPathForWrite(Constants.OUTPUT + "/spill" + spillNumber + ".out.index", size, getConf()); } @@ -188,7 +187,7 @@ public class MROutputFiles extends MapOutputFile { public Path getInputFile(int mapId) throws IOException { return lDirAlloc.getLocalPathToRead(String.format( - REDUCE_INPUT_FILE_FORMAT_STRING, MRJobConfig.OUTPUT, Integer + REDUCE_INPUT_FILE_FORMAT_STRING, Constants.OUTPUT, Integer .valueOf(mapId)), getConf()); } @@ -205,7 +204,7 @@ public class MROutputFiles extends MapOutputFile { long size) throws IOException { return lDirAlloc.getLocalPathForWrite(String.format( - REDUCE_INPUT_FILE_FORMAT_STRING, MRJobConfig.OUTPUT, mapId.getId()), + REDUCE_INPUT_FILE_FORMAT_STRING, Constants.OUTPUT, mapId.getId()), size, getConf()); } @@ -213,7 +212,7 @@ public class MROutputFiles extends MapOutputFile { @Override public void removeAll() throws IOException { - ((JobConf)getConf()).deleteLocalFiles(MRJobConfig.OUTPUT); + ((JobConf)getConf()).deleteLocalFiles(Constants.OUTPUT); } @Override diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java index 597b2edaa39..7e978e9cf94 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java @@ -44,7 +44,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.SecureIOUtils; import org.apache.hadoop.mapreduce.JobID; -import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.util.ProcessTree; import org.apache.hadoop.util.Shell; import org.apache.log4j.Appender; @@ -76,18 +75,10 @@ public class TaskLog { } } } - - public static String getMRv2LogDir() { - return System.getProperty(MRJobConfig.TASK_LOG_DIR); - } - + public static File getTaskLogFile(TaskAttemptID taskid, boolean isCleanup, LogName filter) { - if (getMRv2LogDir() != null) { - return new File(getMRv2LogDir(), filter.toString()); - } else { - return new File(getAttemptDir(taskid, isCleanup), filter.toString()); - } + return new File(getAttemptDir(taskid, isCleanup), filter.toString()); } static File getRealTaskLogFileLocation(TaskAttemptID taskid, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java index 0a108d73b63..cb8b476ac75 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java @@ -18,7 +18,6 @@ package org.apache.hadoop.mapred.pipes; -import java.io.BufferedInputStream; import java.io.File; import java.io.IOException; import java.net.ServerSocket; @@ -27,7 +26,6 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Random; import javax.crypto.SecretKey; @@ -113,6 +111,7 @@ class Application jobACLs = new HashMap(); @@ -279,14 +278,6 @@ public class JobStatus implements Writable, Cloneable { this.queue = queue; } - /** - * Set diagnostic information. - * @param failureInfo diagnostic information - */ - protected synchronized void setFailureInfo(String failureInfo) { - this.failureInfo = failureInfo; - } - /** * Get queue name * @return queue name @@ -368,15 +359,6 @@ public class JobStatus implements Writable, Cloneable { */ public synchronized JobPriority getPriority() { return priority; } - /** - * Gets any available info on the reason of failure of the job. - * @return diagnostic information on why a job might have failed. - */ - public synchronized String getFailureInfo() { - return this.failureInfo; - } - - /** * Returns true if the status is for a completed job. */ 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 accfdddc3db..33884bb82e9 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 @@ -210,8 +210,6 @@ public interface MRJobConfig { public static final String REDUCE_LOG_LEVEL = "mapreduce.reduce.log.level"; - public static final String DEFAULT_LOG_LEVEL = "INFO"; - public static final String REDUCE_MERGE_INMEM_THRESHOLD = "mapreduce.reduce.merge.inmem.threshold"; public static final String REDUCE_INPUT_BUFFER_PERCENT = "mapreduce.reduce.input.buffer.percent"; @@ -332,15 +330,9 @@ public interface MRJobConfig { MR_AM_PREFIX+"num-progress-splits"; public static final int DEFAULT_MR_AM_NUM_PROGRESS_SPLITS = 12; - /** - * Upper limit on the number of threads user to launch containers in the app - * master. Expect level config, you shouldn't be needing it in most cases. - */ - public static final String MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT = - MR_AM_PREFIX+"containerlauncher.thread-count-limit"; - - public static final int DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT = - 500; + /** 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 = @@ -408,69 +400,4 @@ public interface MRJobConfig { */ public static final String MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR = MR_AM_PREFIX + "create-intermediate-jh-base-dir"; - - public static final String MAPRED_MAP_ADMIN_JAVA_OPTS = - "mapreduce.admin.map.child.java.opts"; - - public static final String MAPRED_REDUCE_ADMIN_JAVA_OPTS = - "mapreduce.admin.reduce.child.java.opts"; - - public static final String DEFAULT_MAPRED_ADMIN_JAVA_OPTS = - "-Djava.net.preferIPv4Stack=true " + - "-Dhadoop.metrics.log.level=WARN "; - - public static final String MAPRED_ADMIN_USER_SHELL = - "mapreduce.admin.user.shell"; - - public static final String DEFAULT_SHELL = "/bin/bash"; - - public static final String MAPRED_ADMIN_USER_ENV = - "mapreduce.admin.user.env"; - - public static final String DEFAULT_MAPRED_ADMIN_USER_ENV = - "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib"; - - public static final String WORKDIR = "work"; - - public static final String OUTPUT = "output"; - - public static final String HADOOP_WORK_DIR = "HADOOP_WORK_DIR"; - - public static final String STDOUT_LOGFILE_ENV = "STDOUT_LOGFILE_ENV"; - - public static final String STDERR_LOGFILE_ENV = "STDERR_LOGFILE_ENV"; - - // This should be the directory where splits file gets localized on the node - // running ApplicationMaster. - public static final String JOB_SUBMIT_DIR = "jobSubmitDir"; - - // This should be the name of the localized job-configuration file on the node - // running ApplicationMaster and Task - public static final String JOB_CONF_FILE = "job.xml"; - - // This should be the name of the localized job-jar file on the node running - // individual containers/tasks. - public static final String JOB_JAR = "job.jar"; - - public static final String JOB_SPLIT = "job.split"; - - public static final String JOB_SPLIT_METAINFO = "job.splitmetainfo"; - - public static final String APPLICATION_MASTER_CLASS = - "org.apache.hadoop.mapreduce.v2.app.MRAppMaster"; - - // 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 - // Containers. - public static final String APPLICATION_TOKENS_FILE = "appTokens"; - - /** The log directory for the containers */ - public static final String TASK_LOG_DIR = MR_PREFIX + "container.log.dir"; - - public static final String TASK_LOG_SIZE = MR_PREFIX + "container.log.filesize"; - - public static final String MAPREDUCE_V2_CHILD_CLASS = - "org.apache.hadoop.mapred.YarnChild"; - } 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 e86eb279e9a..56f114adc5c 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 @@ -135,7 +135,7 @@ public class HistoryClientService extends AbstractService { webApp = new HsWebApp(history); String bindAddress = conf.get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS, JHAdminConfig.DEFAULT_MR_HISTORY_WEBAPP_ADDRESS); - WebApps.$for("jobhistory", this).at(bindAddress).start(webApp); + WebApps.$for("yarn", this).at(bindAddress).start(webApp); } @Override 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 7e9e67c3c3d..c9f90b9e79e 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 @@ -22,6 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -83,6 +84,25 @@ public class JobHistory extends AbstractService implements HistoryContext { private static final Log SUMMARY_LOG = LogFactory.getLog(JobSummary.class); + /* + * TODO Get rid of this once JobId has it's own comparator + */ + private static final Comparator JOB_ID_COMPARATOR = + new Comparator() { + @Override + public int compare(JobId o1, JobId o2) { + if (o1.getAppId().getClusterTimestamp() > + o2.getAppId().getClusterTimestamp()) { + return 1; + } else if (o1.getAppId().getClusterTimestamp() < + o2.getAppId().getClusterTimestamp()) { + return -1; + } else { + return o1.getId() - o2.getId(); + } + } + }; + private static String DONE_BEFORE_SERIAL_TAIL = JobHistoryUtils.doneSubdirsBeforeSerialTail(); @@ -98,19 +118,19 @@ public class JobHistory extends AbstractService implements HistoryContext { //Maintains minimal details for recent jobs (parsed from history file name). //Sorted on Job Completion Time. private final SortedMap jobListCache = - new ConcurrentSkipListMap(); + new ConcurrentSkipListMap(JOB_ID_COMPARATOR); // Re-use exisiting MetaInfo objects if they exist for the specific JobId. (synchronization on MetaInfo) // Check for existance of the object when using iterators. private final SortedMap intermediateListCache = - new ConcurrentSkipListMap(); + new ConcurrentSkipListMap(JOB_ID_COMPARATOR); //Maintains a list of known done subdirectories. Not currently used. private final Set existingDoneSubdirs = new HashSet(); private final SortedMap loadedJobCache = - new ConcurrentSkipListMap(); + new ConcurrentSkipListMap(JOB_ID_COMPARATOR); /** * Maintains a mapping between intermediate user directories and the last @@ -653,7 +673,7 @@ public class JobHistory extends AbstractService implements HistoryContext { private Map getAllJobsInternal() { //TODO This should ideally be using getAllJobsMetaInfo // or get rid of that method once Job has APIs for user, finishTime etc. - SortedMap result = new TreeMap(); + SortedMap result = new TreeMap(JOB_ID_COMPARATOR); try { scanIntermediateDirectory(); } catch (IOException e) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml index ef388fcd86a..4b605cb2ae5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml @@ -64,12 +64,6 @@ hadoop-yarn-server-resourcemanager test - - org.apache.hadoop - hadoop-yarn-server-resourcemanager - test-jar - test - org.apache.hadoop hadoop-yarn-server-common 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 20c6ce7c000..80c8d91a1b3 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 @@ -1,20 +1,20 @@ /** - * 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.mapred; @@ -42,29 +42,29 @@ public class ClientCache { private final Configuration conf; private final ResourceMgrDelegate rm; - + private static final Log LOG = LogFactory.getLog(ClientCache.class); private Map cache = - new HashMap(); - + new HashMap(); + private MRClientProtocol hsProxy; - public ClientCache(Configuration conf, ResourceMgrDelegate rm) { + ClientCache(Configuration conf, ResourceMgrDelegate rm) { this.conf = conf; this.rm = rm; } //TODO: evict from the cache on some threshold - public synchronized ClientServiceDelegate getClient(JobID jobId) { - if (hsProxy == null) { + synchronized ClientServiceDelegate getClient(JobID jobId) { + if (hsProxy == null) { try { - hsProxy = instantiateHistoryProxy(); - } catch (IOException e) { - LOG.warn("Could not connect to History server.", e); - throw new YarnException("Could not connect to History server.", e); - } - } + hsProxy = instantiateHistoryProxy(); + } catch (IOException e) { + LOG.warn("Could not connect to History server.", e); + throw new YarnException("Could not connect to History server.", e); + } + } ClientServiceDelegate client = cache.get(jobId); if (client == null) { client = new ClientServiceDelegate(conf, rm, jobId, hsProxy); @@ -74,7 +74,7 @@ public class ClientCache { } private MRClientProtocol instantiateHistoryProxy() - throws IOException { + throws IOException { final String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS); if (StringUtils.isEmpty(serviceAddr)) { return null; 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 341e17e9513..605c44e5ed9 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 @@ -70,7 +70,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier; import org.apache.hadoop.yarn.security.SchedulerSecurityInfo; -public class ClientServiceDelegate { +class ClientServiceDelegate { private static final Log LOG = LogFactory.getLog(ClientServiceDelegate.class); // Caches for per-user NotRunningJobs @@ -87,7 +87,7 @@ public class ClientServiceDelegate { private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); private static String UNKNOWN_USER = "Unknown User"; - public ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm, + ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm, JobID jobId, MRClientProtocol historyServerProxy) { this.conf = new Configuration(conf); // Cloning for modifying. // For faster redirects from AM to HS. @@ -101,20 +101,16 @@ public class ClientServiceDelegate { // Get the instance of the NotRunningJob corresponding to the specified // user and state - private NotRunningJob getNotRunningJob(ApplicationReport applicationReport, - JobState state) { + private NotRunningJob getNotRunningJob(String user, JobState state) { synchronized (notRunningJobs) { HashMap map = notRunningJobs.get(state); if (map == null) { map = new HashMap(); notRunningJobs.put(state, map); } - String user = - (applicationReport == null) ? - UNKNOWN_USER : applicationReport.getUser(); NotRunningJob notRunningJob = map.get(user); if (notRunningJob == null) { - notRunningJob = new NotRunningJob(applicationReport, state); + notRunningJob = new NotRunningJob(user, state); map.put(user, notRunningJob); } return notRunningJob; @@ -134,7 +130,7 @@ public class ClientServiceDelegate { if (application == null) { LOG.info("Could not get Job info from RM for job " + jobId + ". Redirecting to job history server."); - return checkAndGetHSProxy(null, JobState.NEW); + return checkAndGetHSProxy(UNKNOWN_USER, JobState.NEW); } try { if (application.getHost() == null || "".equals(application.getHost())) { @@ -175,7 +171,7 @@ public class ClientServiceDelegate { if (application == null) { LOG.info("Could not get Job info from RM for job " + jobId + ". Redirecting to job history server."); - return checkAndGetHSProxy(null, JobState.RUNNING); + return checkAndGetHSProxy(UNKNOWN_USER, JobState.RUNNING); } } catch (InterruptedException e) { LOG.warn("getProxy() call interruped", e); @@ -195,17 +191,17 @@ public class ClientServiceDelegate { if (application.getState() == ApplicationState.NEW || application.getState() == ApplicationState.SUBMITTED) { realProxy = null; - return getNotRunningJob(application, JobState.NEW); + return getNotRunningJob(user, JobState.NEW); } if (application.getState() == ApplicationState.FAILED) { realProxy = null; - return getNotRunningJob(application, JobState.FAILED); + return getNotRunningJob(user, JobState.FAILED); } if (application.getState() == ApplicationState.KILLED) { realProxy = null; - return getNotRunningJob(application, JobState.KILLED); + return getNotRunningJob(user, JobState.KILLED); } //History server can serve a job only if application @@ -213,16 +209,15 @@ public class ClientServiceDelegate { if (application.getState() == ApplicationState.SUCCEEDED) { LOG.info("Application state is completed. " + "Redirecting to job history server"); - realProxy = checkAndGetHSProxy(application, JobState.SUCCEEDED); + realProxy = checkAndGetHSProxy(user, JobState.SUCCEEDED); } return realProxy; } - private MRClientProtocol checkAndGetHSProxy( - ApplicationReport applicationReport, JobState state) { + private MRClientProtocol checkAndGetHSProxy(String user, JobState state) { if (null == historyServerProxy) { LOG.warn("Job History Server is not configured."); - return getNotRunningJob(applicationReport, state); + return getNotRunningJob(user, state); } return historyServerProxy; } @@ -279,7 +274,7 @@ public class ClientServiceDelegate { } } - public org.apache.hadoop.mapreduce.Counters getJobCounters(JobID arg0) throws IOException, + org.apache.hadoop.mapreduce.Counters getJobCounters(JobID arg0) throws IOException, InterruptedException { org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter.toYarn(arg0); GetCountersRequest request = recordFactory.newRecordInstance(GetCountersRequest.class); @@ -290,7 +285,7 @@ public class ClientServiceDelegate { } - public TaskCompletionEvent[] getTaskCompletionEvents(JobID arg0, int arg1, int arg2) + TaskCompletionEvent[] getTaskCompletionEvents(JobID arg0, int arg1, int arg2) throws IOException, InterruptedException { org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter .toYarn(arg0); @@ -308,7 +303,7 @@ public class ClientServiceDelegate { .toArray(new org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent[0])); } - public String[] getTaskDiagnostics(org.apache.hadoop.mapreduce.TaskAttemptID arg0) + String[] getTaskDiagnostics(org.apache.hadoop.mapreduce.TaskAttemptID arg0) throws IOException, InterruptedException { org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID = TypeConverter @@ -326,25 +321,24 @@ public class ClientServiceDelegate { return result; } - public JobStatus getJobStatus(JobID oldJobID) throws YarnRemoteException { + JobStatus getJobStatus(JobID oldJobID) throws YarnRemoteException { org.apache.hadoop.mapreduce.v2.api.records.JobId jobId = TypeConverter.toYarn(oldJobID); - GetJobReportRequest request = - recordFactory.newRecordInstance(GetJobReportRequest.class); + GetJobReportRequest request = recordFactory.newRecordInstance(GetJobReportRequest.class); request.setJobId(jobId); JobReport report = ((GetJobReportResponse) invoke("getJobReport", GetJobReportRequest.class, request)).getJobReport(); String jobFile = MRApps.getJobFile(conf, report.getUser(), oldJobID); - return TypeConverter.fromYarn(report, jobFile); + //TODO: add tracking url in JobReport + return TypeConverter.fromYarn(report, jobFile, ""); } - public org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(JobID oldJobID, TaskType taskType) + org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(JobID oldJobID, TaskType taskType) throws YarnRemoteException, YarnRemoteException { org.apache.hadoop.mapreduce.v2.api.records.JobId jobId = TypeConverter.toYarn(oldJobID); - GetTaskReportsRequest request = - recordFactory.newRecordInstance(GetTaskReportsRequest.class); + GetTaskReportsRequest request = recordFactory.newRecordInstance(GetTaskReportsRequest.class); request.setJobId(jobId); request.setTaskType(TypeConverter.toYarn(taskType)); @@ -356,7 +350,7 @@ public class ClientServiceDelegate { (taskReports).toArray(new org.apache.hadoop.mapreduce.TaskReport[0]); } - public boolean killTask(TaskAttemptID taskAttemptID, boolean fail) + boolean killTask(TaskAttemptID taskAttemptID, boolean fail) throws YarnRemoteException { org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID = TypeConverter.toYarn(taskAttemptID); @@ -372,7 +366,7 @@ public class ClientServiceDelegate { return true; } - public boolean killJob(JobID oldJobID) + boolean killJob(JobID oldJobID) throws YarnRemoteException { org.apache.hadoop.mapreduce.v2.api.records.JobId jobId = TypeConverter.toYarn(oldJobID); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java index 17ad9f62aae..a40fcedda39 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java @@ -22,8 +22,6 @@ import java.util.ArrayList; import java.util.HashMap; import org.apache.commons.lang.NotImplementedException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; 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; @@ -55,41 +53,20 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskReport; import org.apache.hadoop.mapreduce.v2.api.records.TaskState; -import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; public class NotRunningJob implements MRClientProtocol { - private static final Log LOG = LogFactory.getLog(NotRunningJob.class); - private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); private final JobState jobState; - private final ApplicationReport applicationReport; - - - private ApplicationReport getUnknownApplicationReport() { - ApplicationReport unknown = - recordFactory.newRecordInstance(ApplicationReport.class); - unknown.setUser("N/A"); - unknown.setHost("N/A"); - unknown.setName("N/A"); - unknown.setQueue("N/A"); - unknown.setStartTime(0); - unknown.setFinishTime(0); - unknown.setTrackingUrl("N/A"); - unknown.setDiagnostics("N/A"); - LOG.info("getUnknownApplicationReport"); - return unknown; - } - - NotRunningJob(ApplicationReport applicationReport, JobState jobState) { - this.applicationReport = - (applicationReport == null) ? - getUnknownApplicationReport() : applicationReport; + private final String user; + + NotRunningJob(String username, JobState jobState) { + this.user = username; this.jobState = jobState; } @@ -124,19 +101,15 @@ public class NotRunningJob implements MRClientProtocol { @Override public GetJobReportResponse getJobReport(GetJobReportRequest request) throws YarnRemoteException { + GetJobReportResponse resp = + recordFactory.newRecordInstance(GetJobReportResponse.class); JobReport jobReport = recordFactory.newRecordInstance(JobReport.class); jobReport.setJobId(request.getJobId()); - jobReport.setJobState(jobState); - jobReport.setUser(applicationReport.getUser()); - jobReport.setStartTime(applicationReport.getStartTime()); - jobReport.setDiagnostics(applicationReport.getDiagnostics()); - jobReport.setJobName(applicationReport.getName()); - jobReport.setTrackingUrl(applicationReport.getTrackingUrl()); - jobReport.setFinishTime(applicationReport.getFinishTime()); + jobReport.setJobState(this.jobState); - GetJobReportResponse resp = - recordFactory.newRecordInstance(GetJobReportResponse.class); + jobReport.setUser(this.user); + // TODO: Add jobName & other job information that is available resp.setJobReport(jobReport); return resp; } 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 8b7c818b1e3..8e8081abe4d 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 @@ -32,19 +32,19 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.ClusterMetrics; import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.JobStatus; -import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.QueueAclsInfo; import org.apache.hadoop.mapreduce.QueueInfo; import org.apache.hadoop.mapreduce.TaskTrackerInfo; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier; +import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.util.MRApps; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.api.ClientRMProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; @@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; @@ -79,10 +79,6 @@ public class ResourceMgrDelegate { private ApplicationId applicationId; private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); - /** - * Delegate responsible for communicating with the Resource Manager's {@link ClientRMProtocol}. - * @param conf the configuration object. - */ public ResourceMgrDelegate(YarnConfiguration conf) { this.conf = conf; YarnRPC rpc = YarnRPC.create(this.conf); @@ -101,16 +97,6 @@ public class ResourceMgrDelegate { LOG.info("Connected to ResourceManager at " + rmAddress); } - /** - * Used for injecting applicationsManager, mostly for testing. - * @param conf the configuration object - * @param applicationsManager the handle to talk the resource managers {@link ClientRMProtocol}. - */ - public ResourceMgrDelegate(YarnConfiguration conf, ClientRMProtocol applicationsManager) { - this.conf = conf; - this.applicationsManager = applicationsManager; - } - public void cancelDelegationToken(Token arg0) throws IOException, InterruptedException { return; @@ -169,8 +155,8 @@ public class ResourceMgrDelegate { } public JobID getNewJobID() throws IOException, InterruptedException { - GetNewApplicationRequest request = recordFactory.newRecordInstance(GetNewApplicationRequest.class); - applicationId = applicationsManager.getNewApplication(request).getApplicationId(); + GetNewApplicationIdRequest request = recordFactory.newRecordInstance(GetNewApplicationIdRequest.class); + applicationId = applicationsManager.getNewApplicationId(request).getApplicationId(); return TypeConverter.fromYarn(applicationId); } @@ -268,7 +254,7 @@ public class ResourceMgrDelegate { public String getSystemDir() throws IOException, InterruptedException { - Path sysDir = new Path(MRJobConfig.JOB_SUBMIT_DIR); + Path sysDir = new Path(MRConstants.JOB_SUBMIT_DIR); //FileContext.getFileContext(conf).delete(sysDir, true); return sysDir.toString(); } @@ -308,9 +294,9 @@ public class ResourceMgrDelegate { } public void killApplication(ApplicationId applicationId) throws IOException { - KillApplicationRequest request = recordFactory.newRecordInstance(KillApplicationRequest.class); + FinishApplicationRequest request = recordFactory.newRecordInstance(FinishApplicationRequest.class); request.setApplicationId(applicationId); - applicationsManager.forceKillApplication(request); + applicationsManager.finishApplication(request); LOG.info("Killing application " + applicationId); } 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 a11968a16f9..82134c7520f 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 @@ -51,6 +51,7 @@ import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier; +import org.apache.hadoop.mapreduce.v2.MRConstants; import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; import org.apache.hadoop.mapreduce.v2.util.MRApps; import org.apache.hadoop.security.Credentials; @@ -59,7 +60,6 @@ import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationState; @@ -105,22 +105,10 @@ public class YARNRunner implements ClientProtocol { * @param resMgrDelegate the resourcemanager client handle. */ public YARNRunner(Configuration conf, ResourceMgrDelegate resMgrDelegate) { - this(conf, resMgrDelegate, new ClientCache(conf, resMgrDelegate)); - } - - /** - * Similar to {@link YARNRunner#YARNRunner(Configuration, ResourceMgrDelegate)} - * but allowing injecting {@link ClientCache}. Enable mocking and testing. - * @param conf the configuration object - * @param resMgrDelegate the resource manager delegate - * @param clientCache the client cache object. - */ - public YARNRunner(Configuration conf, ResourceMgrDelegate resMgrDelegate, - ClientCache clientCache) { this.conf = conf; try { this.resMgrDelegate = resMgrDelegate; - this.clientCache = clientCache; + this.clientCache = new ClientCache(this.conf, resMgrDelegate); this.defaultFileContext = FileContext.getFileContext(this.conf); } catch (UnsupportedFileSystemException ufe) { throw new RuntimeException("Error in instantiating YarnClient", ufe); @@ -222,7 +210,7 @@ public class YARNRunner implements ClientProtocol { // Upload only in security mode: TODO Path applicationTokensFile = - new Path(jobSubmitDir, MRJobConfig.APPLICATION_TOKENS_FILE); + new Path(jobSubmitDir, MRConstants.APPLICATION_TOKENS_FILE); try { ts.writeTokenStorageFile(applicationTokensFile, conf); } catch (IOException e) { @@ -238,9 +226,7 @@ public class YARNRunner implements ClientProtocol { ApplicationReport appMaster = resMgrDelegate .getApplicationReport(applicationId); - String diagnostics = - (appMaster == null ? - "application report is null" : appMaster.getDiagnostics()); + String diagnostics = (appMaster == null ? "application report is null" : appMaster.getDiagnostics()); if (appMaster == null || appMaster.getState() == ApplicationState.FAILED || appMaster.getState() == ApplicationState.KILLED) { throw new IOException("Failed to run job : " + @@ -277,7 +263,7 @@ public class YARNRunner implements ClientProtocol { Map localResources = new HashMap(); - Path jobConfPath = new Path(jobSubmitDir, MRJobConfig.JOB_CONF_FILE); + Path jobConfPath = new Path(jobSubmitDir, MRConstants.JOB_CONF_FILE); URL yarnUrlForJobSubmitDir = ConverterUtils .getYarnUrlFromPath(defaultFileContext.getDefaultFileSystem() @@ -286,13 +272,13 @@ public class YARNRunner implements ClientProtocol { LOG.debug("Creating setup context, jobSubmitDir url is " + yarnUrlForJobSubmitDir); - localResources.put(MRJobConfig.JOB_CONF_FILE, + localResources.put(MRConstants.JOB_CONF_FILE, createApplicationResource(defaultFileContext, jobConfPath)); if (jobConf.get(MRJobConfig.JAR) != null) { - localResources.put(MRJobConfig.JOB_JAR, + localResources.put(MRConstants.JOB_JAR, createApplicationResource(defaultFileContext, - new Path(jobSubmitDir, MRJobConfig.JOB_JAR))); + new Path(jobSubmitDir, MRConstants.JOB_JAR))); } else { // Job jar may be null. For e.g, for pipes, the job jar is the hadoop // mapreduce jar itself which is already on the classpath. @@ -301,12 +287,10 @@ public class YARNRunner implements ClientProtocol { } // TODO gross hack - for (String s : new String[] { - MRJobConfig.JOB_SPLIT, - MRJobConfig.JOB_SPLIT_METAINFO, - MRJobConfig.APPLICATION_TOKENS_FILE }) { + for (String s : new String[] { "job.split", "job.splitmetainfo", + MRConstants.APPLICATION_TOKENS_FILE }) { localResources.put( - MRJobConfig.JOB_SUBMIT_DIR + "/" + s, + MRConstants.JOB_SUBMIT_DIR + "/" + s, createApplicationResource(defaultFileContext, new Path(jobSubmitDir, s))); } @@ -320,24 +304,22 @@ public class YARNRunner implements ClientProtocol { } // Setup the command to run the AM + String javaHome = "$JAVA_HOME"; Vector vargs = new Vector(8); - vargs.add(Environment.JAVA_HOME.$() + "/bin/java"); - - long logSize = TaskLog.getTaskLogLength(new JobConf(conf)); - vargs.add("-Dlog4j.configuration=container-log4j.properties"); - vargs.add("-D" + MRJobConfig.TASK_LOG_DIR + "=" - + ApplicationConstants.LOG_DIR_EXPANSION_VAR); - vargs.add("-D" + MRJobConfig.TASK_LOG_SIZE + "=" + logSize); + vargs.add(javaHome + "/bin/java"); + vargs.add("-Dhadoop.root.logger=" + + conf.get(MRJobConfig.MR_AM_LOG_OPTS, + MRJobConfig.DEFAULT_MR_AM_LOG_OPTS) + ",console"); vargs.add(conf.get(MRJobConfig.MR_AM_COMMAND_OPTS, MRJobConfig.DEFAULT_MR_AM_COMMAND_OPTS)); - vargs.add(MRJobConfig.APPLICATION_MASTER_CLASS); - vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + - Path.SEPARATOR + ApplicationConstants.STDOUT); - vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + - Path.SEPARATOR + ApplicationConstants.STDERR); - + vargs.add("org.apache.hadoop.mapreduce.v2.app.MRAppMaster"); + vargs.add(String.valueOf(applicationId.getClusterTimestamp())); + vargs.add(String.valueOf(applicationId.getId())); + vargs.add(ApplicationConstants.AM_FAIL_COUNT_STRING); + vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout"); + vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr"); Vector vargsFinal = new Vector(8); // Final commmand @@ -350,13 +332,15 @@ public class YARNRunner implements ClientProtocol { LOG.info("Command to launch container for ApplicationMaster is : " + mergedCommand); - // Setup the CLASSPATH in environment - // i.e. add { job jar, CWD, Hadoop jars} to classpath. + // Setup the environment - Add { job jar, MR app jar } to classpath. Map environment = new HashMap(); - MRApps.setClasspath(environment); - + MRApps.setInitialClasspath(environment); + MRApps.addToClassPath(environment, MRConstants.JOB_JAR); + MRApps.addToClassPath(environment, + MRConstants.YARN_MAPREDUCE_APP_JAR_PATH); + // Parse distributed cache - MRApps.setupDistributedCache(jobConf, localResources); + MRApps.setupDistributedCache(jobConf, localResources, environment); // Setup ContainerLaunchContext for AM container ContainerLaunchContext amContainer = @@ -441,35 +425,9 @@ public class YARNRunner implements ClientProtocol { @Override public void killJob(JobID arg0) throws IOException, InterruptedException { - /* check if the status is not running, if not send kill to RM */ - JobStatus status = clientCache.getClient(arg0).getJobStatus(arg0); - if (status.getState() != JobStatus.State.RUNNING) { - resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId()); - return; - } - - try { - /* send a kill to the AM */ - clientCache.getClient(arg0).killJob(arg0); - long currentTimeMillis = System.currentTimeMillis(); - long timeKillIssued = currentTimeMillis; - while ((currentTimeMillis < timeKillIssued + 10000L) && (status.getState() - != JobStatus.State.KILLED)) { - try { - Thread.sleep(1000L); - } catch(InterruptedException ie) { - /** interrupted, just break */ - break; - } - currentTimeMillis = System.currentTimeMillis(); - status = clientCache.getClient(arg0).getJobStatus(arg0); - } - } catch(IOException io) { - LOG.debug("Error when checking for application status", io); - } - if (status.getState() != JobStatus.State.KILLED) { - resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId()); - } + if (!clientCache.getClient(arg0).killJob(arg0)) { + resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId()); + } } @Override 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 d90e7216941..e2cb1e05ea7 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 @@ -68,8 +68,8 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.api.ClientRMProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; @@ -78,8 +78,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; @@ -245,7 +245,7 @@ public class TestClientRedirect { } @Override - public GetNewApplicationResponse getNewApplication(GetNewApplicationRequest request) throws YarnRemoteException { + public GetNewApplicationIdResponse getNewApplicationId(GetNewApplicationIdRequest request) throws YarnRemoteException { return null; } @@ -267,13 +267,6 @@ public class TestClientRedirect { application.setHost(split[0]); application.setRpcPort(Integer.parseInt(split[1])); application.setUser("TestClientRedirect-user"); - application.setName("N/A"); - application.setQueue("N/A"); - application.setStartTime(0); - application.setFinishTime(0); - application.setTrackingUrl("N/A"); - application.setDiagnostics("N/A"); - GetApplicationReportResponse response = recordFactory .newRecordInstance(GetApplicationReportResponse.class); response.setApplicationReport(application); @@ -288,9 +281,9 @@ public class TestClientRedirect { } @Override - public KillApplicationResponse forceKillApplication( - KillApplicationRequest request) throws YarnRemoteException { - return recordFactory.newRecordInstance(KillApplicationResponse.class); + public FinishApplicationResponse finishApplication( + FinishApplicationRequest request) throws YarnRemoteException { + return null; } @Override @@ -451,7 +444,7 @@ public class TestClientRedirect { @Override public KillJobResponse killJob(KillJobRequest request) throws YarnRemoteException { - return recordFactory.newRecordInstance(KillJobResponse.class); + return null; } @Override diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java index 5b07d4997d7..b7fd6c9475a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java @@ -109,7 +109,7 @@ public class TestClientServiceDelegate { ClientServiceDelegate clientServiceDelegate = getClientServiceDelegate( null, getRMDelegate()); JobStatus jobStatus = clientServiceDelegate.getJobStatus(oldJobId); - Assert.assertEquals("N/A", jobStatus.getUsername()); + Assert.assertEquals("Unknown User", jobStatus.getUsername()); Assert.assertEquals(JobStatus.State.PREP, jobStatus.getState()); //RM has app report and job History Server is not configured @@ -145,13 +145,6 @@ public class TestClientServiceDelegate { .newRecord(ApplicationReport.class); applicationReport.setState(ApplicationState.SUCCEEDED); applicationReport.setUser("root"); - applicationReport.setHost("N/A"); - applicationReport.setName("N/A"); - applicationReport.setQueue("N/A"); - applicationReport.setStartTime(0); - applicationReport.setFinishTime(0); - applicationReport.setTrackingUrl("N/A"); - applicationReport.setDiagnostics("N/A"); return applicationReport; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java deleted file mode 100644 index 2bc9030bf85..00000000000 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java +++ /dev/null @@ -1,59 +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; - -import java.io.IOException; - -import junit.framework.TestCase; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapred.YARNRunner; -import org.apache.hadoop.mapreduce.protocol.ClientProtocol; -import org.junit.Test; - -public class TestYarnClientProtocolProvider extends TestCase { - - @Test - public void testClusterWithYarnClientProtocolProvider() throws Exception { - - Configuration conf = new Configuration(false); - Cluster cluster = null; - - try { - cluster = new Cluster(conf); - fail("Cluster should not be initialized with out any framework name"); - } catch (IOException e) { - - } - - try { - conf = new Configuration(); - conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME); - cluster = new Cluster(conf); - ClientProtocol client = cluster.getClient(); - assertTrue(client instanceof YARNRunner); - } catch (IOException e) { - - } finally { - if (cluster != null) { - cluster.close(); - } - } - } -} 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 49a63db44ba..fcb2a79fafb 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 @@ -43,15 +43,9 @@ import org.apache.hadoop.yarn.service.Service; */ public class MiniMRYarnCluster extends MiniYARNCluster { - public static final String HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME = - "hadoop-mapreduce-client-app-0.24.0-SNAPSHOT.jar"; - - public static final String YARN_MAPREDUCE_APP_JAR_PATH = - "$YARN_HOME/modules/" + HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME; - public static final String APPJAR = "../hadoop-mapreduce-client-app/target/" - + HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME; + + MRConstants.HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME; private static final Log LOG = LogFactory.getLog(MiniMRYarnCluster.class); private JobHistoryServer historyServer; 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 aa832aa1cc2..0a1943c013b 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 @@ -402,7 +402,7 @@ public class TestMRJobs { // both should be reachable via the class loader. Assert.assertNotNull(cl.getResource("distributed.jar.inside2")); Assert.assertNotNull(cl.getResource("distributed.jar.inside3")); - Assert.assertNotNull(cl.getResource("distributed.jar.inside4")); + Assert.assertNull(cl.getResource("distributed.jar.inside4")); // Check that the symlink for the renaming was created in the cwd; File symlinkFile = new File("distributed.first.symlink"); 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 index 346ccd2f0da..bc0dfe5fa4a 100644 --- 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 @@ -22,7 +22,6 @@ 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.verify; import static org.mockito.Mockito.when; import java.io.File; @@ -37,37 +36,15 @@ 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.ClientCache; -import org.apache.hadoop.mapred.ClientServiceDelegate; import org.apache.hadoop.mapred.ResourceMgrDelegate; import org.apache.hadoop.mapred.YARNRunner; import org.apache.hadoop.mapreduce.JobID; -import org.apache.hadoop.mapreduce.JobPriority; -import org.apache.hadoop.mapreduce.JobStatus.State; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.yarn.api.ClientRMProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse; 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.api.records.QueueInfo; -import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; @@ -77,8 +54,9 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; /** - * Test YarnRunner and make sure the client side plugin works - * fine + * 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); @@ -87,22 +65,18 @@ public class TestYARNRunner extends TestCase { private YARNRunner yarnRunner; private ResourceMgrDelegate resourceMgrDelegate; private YarnConfiguration conf; - private ClientCache clientCache; private ApplicationId appId; private JobID jobId; private File testWorkDir = new File("target", TestYARNRunner.class.getName()); private ApplicationSubmissionContext submissionContext; - private ClientServiceDelegate clientDelegate; private static final String failString = "Rejected job"; @Before public void setUp() throws Exception { resourceMgrDelegate = mock(ResourceMgrDelegate.class); conf = new YarnConfiguration(); - clientCache = new ClientCache(conf, resourceMgrDelegate); - clientCache = spy(clientCache); - yarnRunner = new YARNRunner(conf, resourceMgrDelegate, clientCache); + yarnRunner = new YARNRunner(conf, resourceMgrDelegate); yarnRunner = spy(yarnRunner); submissionContext = mock(ApplicationSubmissionContext.class); doAnswer( @@ -127,31 +101,6 @@ public class TestYARNRunner extends TestCase { } - @Test - public void testJobKill() throws Exception { - clientDelegate = mock(ClientServiceDelegate.class); - when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new - org.apache.hadoop.mapreduce.JobStatus(jobId, 0f, 0f, 0f, 0f, - State.PREP, JobPriority.HIGH, "tmp", "tmp", "tmp", "tmp")); - when(clientDelegate.killJob(any(JobID.class))).thenReturn(true); - doAnswer( - new Answer() { - @Override - public ClientServiceDelegate answer(InvocationOnMock invocation) - throws Throwable { - return clientDelegate; - } - } - ).when(clientCache).getClient(any(JobID.class)); - yarnRunner.killJob(jobId); - verify(resourceMgrDelegate).killApplication(appId); - when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new - org.apache.hadoop.mapreduce.JobStatus(jobId, 0f, 0f, 0f, 0f, - State.RUNNING, JobPriority.HIGH, "tmp", "tmp", "tmp", "tmp")); - yarnRunner.killJob(jobId); - verify(clientDelegate).killJob(jobId); - } - @Test public void testJobSubmissionFailure() throws Exception { when(resourceMgrDelegate.submitApplication(any(ApplicationSubmissionContext.class))). @@ -173,66 +122,4 @@ public class TestYARNRunner extends TestCase { assertTrue(io.getLocalizedMessage().contains(failString)); } } - - @Test - public void testResourceMgrDelegate() throws Exception { - /* we not want a mock of resourcemgr deleagte */ - ClientRMProtocol clientRMProtocol = mock(ClientRMProtocol.class); - ResourceMgrDelegate delegate = new ResourceMgrDelegate(conf, clientRMProtocol); - /* make sure kill calls finish application master */ - when(clientRMProtocol.forceKillApplication(any(KillApplicationRequest.class))) - .thenReturn(null); - delegate.killApplication(appId); - verify(clientRMProtocol).forceKillApplication(any(KillApplicationRequest.class)); - - /* make sure getalljobs calls get all applications */ - when(clientRMProtocol.getAllApplications(any(GetAllApplicationsRequest.class))). - thenReturn(recordFactory.newRecordInstance(GetAllApplicationsResponse.class)); - delegate.getAllJobs(); - verify(clientRMProtocol).getAllApplications(any(GetAllApplicationsRequest.class)); - - /* make sure getapplication report is called */ - when(clientRMProtocol.getApplicationReport(any(GetApplicationReportRequest.class))) - .thenReturn(recordFactory.newRecordInstance(GetApplicationReportResponse.class)); - delegate.getApplicationReport(appId); - verify(clientRMProtocol).getApplicationReport(any(GetApplicationReportRequest.class)); - - /* make sure metrics is called */ - GetClusterMetricsResponse clusterMetricsResponse = recordFactory.newRecordInstance - (GetClusterMetricsResponse.class); - clusterMetricsResponse.setClusterMetrics(recordFactory.newRecordInstance( - YarnClusterMetrics.class)); - when(clientRMProtocol.getClusterMetrics(any(GetClusterMetricsRequest.class))) - .thenReturn(clusterMetricsResponse); - delegate.getClusterMetrics(); - verify(clientRMProtocol).getClusterMetrics(any(GetClusterMetricsRequest.class)); - - when(clientRMProtocol.getClusterNodes(any(GetClusterNodesRequest.class))). - thenReturn(recordFactory.newRecordInstance(GetClusterNodesResponse.class)); - delegate.getActiveTrackers(); - verify(clientRMProtocol).getClusterNodes(any(GetClusterNodesRequest.class)); - - GetNewApplicationResponse newAppResponse = recordFactory.newRecordInstance( - GetNewApplicationResponse.class); - newAppResponse.setApplicationId(appId); - when(clientRMProtocol.getNewApplication(any(GetNewApplicationRequest.class))). - thenReturn(newAppResponse); - delegate.getNewJobID(); - verify(clientRMProtocol).getNewApplication(any(GetNewApplicationRequest.class)); - - GetQueueInfoResponse queueInfoResponse = recordFactory.newRecordInstance( - GetQueueInfoResponse.class); - queueInfoResponse.setQueueInfo(recordFactory.newRecordInstance(QueueInfo.class)); - when(clientRMProtocol.getQueueInfo(any(GetQueueInfoRequest.class))). - thenReturn(queueInfoResponse); - delegate.getQueues(); - verify(clientRMProtocol).getQueueInfo(any(GetQueueInfoRequest.class)); - - GetQueueUserAclsInfoResponse aclResponse = recordFactory.newRecordInstance( - GetQueueUserAclsInfoResponse.class); - when(clientRMProtocol.getQueueUserAcls(any(GetQueueUserAclsInfoRequest.class))) - .thenReturn(aclResponse); - delegate.getQueueAclsForCurrentUser(); - verify(clientRMProtocol).getQueueUserAcls(any(GetQueueUserAclsInfoRequest.class)); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml index 2a5cef3cbc9..ab1ffcca988 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml @@ -88,12 +88,6 @@ hadoop-yarn-server-resourcemanager ${yarn.version} - - org.apache.hadoop - hadoop-yarn-server-resourcemanager - ${yarn.version} - test-jar - org.apache.hadoop hadoop-mapreduce-client-core diff --git a/hadoop-mapreduce-project/hadoop-yarn/README b/hadoop-mapreduce-project/hadoop-yarn/README index 713871ab768..8c4f43454ea 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/README +++ b/hadoop-mapreduce-project/hadoop-yarn/README @@ -30,6 +30,7 @@ clean and test: mvn clean install run selected test after compile: mvn test -Dtest=TestClassName (combined: mvn clean install -Dtest=TestClassName) create runnable binaries after install: mvn assembly:assembly (combined: mvn clean install assembly:assembly) + Eclipse Projects ---------------- http://maven.apache.org/guides/mini/guide-ide-eclipse.html @@ -70,16 +71,3 @@ hadoop-yarn-server - Implementation of the hadoop-yarn-api hadoop-yarn-server-common - APIs shared between resourcemanager and nodemanager hadoop-yarn-server-nodemanager (TaskTracker replacement) hadoop-yarn-server-resourcemanager (JobTracker replacement) - -Utilities for understanding the code ------------------------------------- -Almost all of the yarn components as well as the mapreduce framework use -state-machines for all the data objects. To understand those central pieces of -the code, a visual representation of the state-machines helps much. You can first -convert the state-machines into graphviz(.gv) format by -running: - mvn compile -Pvisualize -Then you can use the dot program for generating directed graphs and convert the above -.gv files to images. The graphviz package has the needed dot program and related -utilites.For e.g., to generate png files you can run: - dot -Tpng NodeManager.gv > NodeManager.png 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 7e34ff5487d..219fd1eb579 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -49,10 +49,6 @@ - - - - diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java index 99f145fbdc3..212ca671c89 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java @@ -37,11 +37,8 @@ public interface ApplicationConstants { public static final String APPLICATION_CLIENT_SECRET_ENV_NAME = "AppClientTokenEnv"; - /** - * The environmental variable for APPLICATION_ATTEMPT_ID. Set in - * ApplicationMaster's environment only. - */ - public static final String APPLICATION_ATTEMPT_ID_ENV = "APPLICATION_ATTEMPT_ID"; + // TODO: Weird. This is part of AM command line. Instead it should be a env. + public static final String AM_FAIL_COUNT_STRING = ""; public static final String CONTAINER_TOKEN_FILE_ENV_NAME = UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION; @@ -49,117 +46,4 @@ public interface ApplicationConstants { public static final String LOCAL_DIR_ENV = "YARN_LOCAL_DIRS"; public static final String LOG_DIR_EXPANSION_VAR = ""; - - public static final String STDERR = "stderr"; - - public static final String STDOUT = "stdout"; - - /** - * Classpath for typical applications. - */ - public static final String[] APPLICATION_CLASSPATH = - new String[] { - "$HADOOP_CONF_DIR", - "$HADOOP_COMMON_HOME/share/hadoop/common/*", - "$HADOOP_COMMON_HOME/share/hadoop/common/lib/*", - "$HADOOP_HDFS_HOME/share/hadoop/hdfs/*", - "$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*", - "$YARN_HOME/modules/*", - "$YARN_HOME/lib/*" - }; - - /** - * Environment for Applications. - * - * Some of the environment variables for applications are final - * i.e. they cannot be modified by the applications. - */ - public enum Environment { - /** - * $USER - * Final, non-modifiable. - */ - USER("USER"), - - /** - * $LOGNAME - * Final, non-modifiable. - */ - LOGNAME("LOGNAME"), - - /** - * $HOME - * Final, non-modifiable. - */ - HOME("HOME"), - - /** - * $PWD - * Final, non-modifiable. - */ - PWD("PWD"), - - /** - * $PATH - */ - PATH("PATH"), - - /** - * $SHELL - */ - SHELL("SHELL"), - - /** - * $JAVA_HOME - */ - JAVA_HOME("JAVA_HOME"), - - /** - * $CLASSPATH - */ - CLASSPATH("CLASSPATH"), - - /** - * $LD_LIBRARY_PATH - */ - LD_LIBRARY_PATH("LD_LIBRARY_PATH"), - - /** - * $HADOOP_CONF_DIR - * Final, non-modifiable. - */ - HADOOP_CONF_DIR("HADOOP_CONF_DIR"), - - /** - * $HADOOP_COMMON_HOME - */ - HADOOP_COMMON_HOME("HADOOP_COMMON_HOME"), - - /** - * $HADOOP_HDFS_HOME - */ - HADOOP_HDFS_HOME("HADOOP_HDFS_HOME"), - - /** - * $YARN_HOME - */ - YARN_HOME("YARN_HOME"); - - private final String variable; - private Environment(String variable) { - this.variable = variable; - } - - public String key() { - return variable; - } - - public String toString() { - return variable; - } - - public String $() { - return "$" + variable; - } - } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java index fb934591354..db4c4790cf0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java @@ -21,8 +21,8 @@ package org.apache.hadoop.yarn.api; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; @@ -31,8 +31,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; @@ -62,18 +62,14 @@ public interface ClientRMProtocol { *

The ResourceManager responds with a new, monotonically * increasing, {@link ApplicationId} which is used by the client to submit * a new application.

- * - *

The ResourceManager also responds with details such - * as minimum and maximum resource capabilities in the cluster as specified in - * {@link GetNewApplicationResponse}.

- * + * * @param request request to get a new ApplicationId * @return new ApplicationId to be used to submit an application * @throws YarnRemoteException * @see #submitApplication(SubmitApplicationRequest) */ - public GetNewApplicationResponse getNewApplication( - GetNewApplicationRequest request) + public GetNewApplicationIdResponse getNewApplicationId( + GetNewApplicationIdRequest request) throws YarnRemoteException; /** @@ -96,7 +92,7 @@ public interface ClientRMProtocol { * @param request request to submit a new application * @return (empty) response on accepting the submission * @throws YarnRemoteException - * @see #getNewApplication(GetNewApplicationRequest) + * @see #getNewApplicationId(GetNewApplicationIdRequest) */ public SubmitApplicationResponse submitApplication( SubmitApplicationRequest request) @@ -106,7 +102,7 @@ public interface ClientRMProtocol { *

The interface used by clients to request the * ResourceManager to abort submitted application.

* - *

The client, via {@link KillApplicationRequest} provides the + *

The client, via {@link FinishApplicationRequest} provides the * {@link ApplicationId} of the application to be aborted.

* *

In secure mode,the ResourceManager verifies access to the @@ -121,8 +117,8 @@ public interface ClientRMProtocol { * @throws YarnRemoteException * @see #getQueueUserAcls(GetQueueUserAclsInfoRequest) */ - public KillApplicationResponse forceKillApplication( - KillApplicationRequest request) + public FinishApplicationResponse finishApplication( + FinishApplicationRequest request) throws YarnRemoteException; /** diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationRequest.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationRequest.java similarity index 94% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationRequest.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationRequest.java index c033e64bb20..023ee3c4ac7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationRequest.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationRequest.java @@ -32,11 +32,11 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; *

The request includes the {@link ApplicationId} of the application to be * aborted.

* - * @see ClientRMProtocol#forceKillApplication(KillApplicationRequest) + * @see ClientRMProtocol#finishApplication(FinishApplicationRequest) */ @Public @Stable -public interface KillApplicationRequest { +public interface FinishApplicationRequest { /** * Get the ApplicationId of the application to be aborted. * @return ApplicationId of the application to be aborted diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationResponse.java similarity index 91% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationResponse.java index 2a8d0f06d29..cd0c728e536 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationResponse.java @@ -28,10 +28,10 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol; * *

Currently it's empty.

* - * @see ClientRMProtocol#forceKillApplication(KillApplicationRequest) + * @see ClientRMProtocol#finishApplication(FinishApplicationRequest) */ @Public @Stable -public interface KillApplicationResponse { +public interface FinishApplicationResponse { } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationIdRequest.java similarity index 91% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationIdRequest.java index a70989f1aab..c841070080d 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationRequest.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationIdRequest.java @@ -27,10 +27,10 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; *

The request sent by clients to get a new {@link ApplicationId} for * submitting an application.

* - * @see ClientRMProtocol#getNewApplication(GetNewApplicationRequest) + * @see ClientRMProtocol#getNewApplicationId(GetNewApplicationIdRequest) */ @Public @Stable -public interface GetNewApplicationRequest { +public interface GetNewApplicationIdRequest { } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationIdResponse.java similarity index 66% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationIdResponse.java index 4e7a7e565be..93a1ab680b3 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationResponse.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewApplicationIdResponse.java @@ -24,17 +24,16 @@ import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ClientRMProtocol; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Resource; /** *

The response sent by the ResourceManager to the client for * a request to a new {@link ApplicationId} for submitting applications.

* - * @see ClientRMProtocol#getNewApplication(GetNewApplicationRequest) + * @see ClientRMProtocol#getNewApplicationId(GetNewApplicationIdRequest) */ @Public @Stable -public interface GetNewApplicationResponse { +public interface GetNewApplicationIdResponse { /** * Get the new ApplicationId allocated by the * ResourceManager. @@ -48,30 +47,4 @@ public interface GetNewApplicationResponse { @Private @Unstable public abstract void setApplicationId(ApplicationId applicationId); - - /** - * Get the minimum capability for any {@link Resource} allocated by the - * ResourceManager in the cluster. - * @return minimum capability of allocated resources in the cluster - */ - @Public - @Stable - public Resource getMinimumResourceCapability(); - - @Private - @Unstable - public void setMinimumResourceCapability(Resource capability); - - /** - * Get the maximum capability for any {@link Resource} allocated by the - * ResourceManager in the cluster. - * @return maximum capability of allocated resources in the cluster - */ - @Public - @Stable - public Resource getMaximumResourceCapability(); - - @Private - @Unstable - public void setMaximumResourceCapability(Resource capability); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/KillApplicationRequestPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/FinishApplicationRequestPBImpl.java similarity index 74% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/KillApplicationRequestPBImpl.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/FinishApplicationRequestPBImpl.java index e2761a090be..044382bddeb 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/KillApplicationRequestPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/FinishApplicationRequestPBImpl.java @@ -19,34 +19,34 @@ package org.apache.hadoop.yarn.api.protocolrecords.impl.pb; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationRequest; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ProtoBase; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationRequestProtoOrBuilder; -public class KillApplicationRequestPBImpl extends ProtoBase implements KillApplicationRequest { - KillApplicationRequestProto proto = KillApplicationRequestProto.getDefaultInstance(); - KillApplicationRequestProto.Builder builder = null; +public class FinishApplicationRequestPBImpl extends ProtoBase implements FinishApplicationRequest { + FinishApplicationRequestProto proto = FinishApplicationRequestProto.getDefaultInstance(); + FinishApplicationRequestProto.Builder builder = null; boolean viaProto = false; private ApplicationId applicationId = null; - public KillApplicationRequestPBImpl() { - builder = KillApplicationRequestProto.newBuilder(); + public FinishApplicationRequestPBImpl() { + builder = FinishApplicationRequestProto.newBuilder(); } - public KillApplicationRequestPBImpl(KillApplicationRequestProto proto) { + public FinishApplicationRequestPBImpl(FinishApplicationRequestProto proto) { this.proto = proto; viaProto = true; } - public KillApplicationRequestProto getProto() { + public FinishApplicationRequestProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -69,7 +69,7 @@ public class KillApplicationRequestPBImpl extends ProtoBase implements KillApplicationResponse { - KillApplicationResponseProto proto = KillApplicationResponseProto.getDefaultInstance(); - KillApplicationResponseProto.Builder builder = null; +public class FinishApplicationResponsePBImpl extends ProtoBase implements FinishApplicationResponse { + FinishApplicationResponseProto proto = FinishApplicationResponseProto.getDefaultInstance(); + FinishApplicationResponseProto.Builder builder = null; boolean viaProto = false; - public KillApplicationResponsePBImpl() { - builder = KillApplicationResponseProto.newBuilder(); + public FinishApplicationResponsePBImpl() { + builder = FinishApplicationResponseProto.newBuilder(); } - public KillApplicationResponsePBImpl(KillApplicationResponseProto proto) { + public FinishApplicationResponsePBImpl(FinishApplicationResponseProto proto) { this.proto = proto; viaProto = true; } - public KillApplicationResponseProto getProto() { + public FinishApplicationResponseProto getProto() { proto = viaProto ? proto : builder.build(); viaProto = true; return proto; @@ -47,7 +47,7 @@ public class KillApplicationResponsePBImpl extends ProtoBase implements GetNewApplicationRequest { - GetNewApplicationRequestProto proto = GetNewApplicationRequestProto.getDefaultInstance(); - GetNewApplicationRequestProto.Builder builder = null; +public class GetNewApplicationIdRequestPBImpl extends ProtoBase implements GetNewApplicationIdRequest { + GetNewApplicationIdRequestProto proto = GetNewApplicationIdRequestProto.getDefaultInstance(); + GetNewApplicationIdRequestProto.Builder builder = null; boolean viaProto = false; - public GetNewApplicationRequestPBImpl() { - builder = GetNewApplicationRequestProto.newBuilder(); + public GetNewApplicationIdRequestPBImpl() { + builder = GetNewApplicationIdRequestProto.newBuilder(); } - public GetNewApplicationRequestPBImpl(GetNewApplicationRequestProto proto) { + public GetNewApplicationIdRequestPBImpl(GetNewApplicationIdRequestProto proto) { this.proto = proto; viaProto = true; } - public GetNewApplicationRequestProto getProto() { + public GetNewApplicationIdRequestProto getProto() { proto = viaProto ? proto : builder.build(); viaProto = true; return proto; @@ -46,7 +47,7 @@ public class GetNewApplicationRequestPBImpl extends ProtoBase implements GetNewApplicationIdResponse { + GetNewApplicationIdResponseProto proto = GetNewApplicationIdResponseProto.getDefaultInstance(); + GetNewApplicationIdResponseProto.Builder builder = null; + boolean viaProto = false; + + private ApplicationId applicationId = null; + + + public GetNewApplicationIdResponsePBImpl() { + builder = GetNewApplicationIdResponseProto.newBuilder(); + } + + public GetNewApplicationIdResponsePBImpl(GetNewApplicationIdResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetNewApplicationIdResponseProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + private void mergeLocalToBuilder() { + if (applicationId != null) { + builder.setApplicationId(convertToProtoFormat(this.applicationId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) + maybeInitBuilder(); + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetNewApplicationIdResponseProto.newBuilder(proto); + } + viaProto = false; + } + + + @Override + public ApplicationId getApplicationId() { + GetNewApplicationIdResponseProtoOrBuilder p = viaProto ? proto : builder; + if (this.applicationId != null) { + return this.applicationId; + } + if (!p.hasApplicationId()) { + return null; + } + this.applicationId = convertFromProtoFormat(p.getApplicationId()); + return this.applicationId; + } + + @Override + public void setApplicationId(ApplicationId applicationId) { + maybeInitBuilder(); + if (applicationId == null) + builder.clearApplicationId(); + this.applicationId = applicationId; + } + + private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) { + return new ApplicationIdPBImpl(p); + } + + private ApplicationIdProto convertToProtoFormat(ApplicationId t) { + return ((ApplicationIdPBImpl)t).getProto(); + } + + + +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNewApplicationResponsePBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNewApplicationResponsePBImpl.java deleted file mode 100644 index d15f1b75274..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNewApplicationResponsePBImpl.java +++ /dev/null @@ -1,173 +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.api.protocolrecords.impl.pb; - - -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ProtoBase; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl; -import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationResponseProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationResponseProtoOrBuilder; - -public class GetNewApplicationResponsePBImpl extends ProtoBase implements GetNewApplicationResponse { - GetNewApplicationResponseProto proto = GetNewApplicationResponseProto.getDefaultInstance(); - GetNewApplicationResponseProto.Builder builder = null; - boolean viaProto = false; - - private ApplicationId applicationId = null; - private Resource minimumResourceCapability = null; - private Resource maximumResourceCapability = null; - - public GetNewApplicationResponsePBImpl() { - builder = GetNewApplicationResponseProto.newBuilder(); - } - - public GetNewApplicationResponsePBImpl(GetNewApplicationResponseProto proto) { - this.proto = proto; - viaProto = true; - } - - public GetNewApplicationResponseProto getProto() { - mergeLocalToProto(); - proto = viaProto ? proto : builder.build(); - viaProto = true; - return proto; - } - - private void mergeLocalToBuilder() { - if (applicationId != null) { - builder.setApplicationId(convertToProtoFormat(this.applicationId)); - } - if (minimumResourceCapability != null) { - builder.setMinimumCapability(convertToProtoFormat(this.minimumResourceCapability)); - } - if (maximumResourceCapability != null) { - builder.setMaximumCapability(convertToProtoFormat(this.maximumResourceCapability)); - } - } - - private void mergeLocalToProto() { - if (viaProto) - maybeInitBuilder(); - mergeLocalToBuilder(); - proto = builder.build(); - viaProto = true; - } - - private void maybeInitBuilder() { - if (viaProto || builder == null) { - builder = GetNewApplicationResponseProto.newBuilder(proto); - } - viaProto = false; - } - - - @Override - public ApplicationId getApplicationId() { - if (this.applicationId != null) { - return this.applicationId; - } - - GetNewApplicationResponseProtoOrBuilder p = viaProto ? proto : builder; - if (!p.hasApplicationId()) { - return null; - } - - this.applicationId = convertFromProtoFormat(p.getApplicationId()); - return this.applicationId; - } - - @Override - public void setApplicationId(ApplicationId applicationId) { - maybeInitBuilder(); - if (applicationId == null) - builder.clearApplicationId(); - this.applicationId = applicationId; - } - - @Override - public Resource getMaximumResourceCapability() { - if (this.maximumResourceCapability != null) { - return this.maximumResourceCapability; - } - - GetNewApplicationResponseProtoOrBuilder p = viaProto ? proto : builder; - if (!p.hasMaximumCapability()) { - return null; - } - - this.maximumResourceCapability = convertFromProtoFormat(p.getMaximumCapability()); - return this.maximumResourceCapability; - } - - @Override - public Resource getMinimumResourceCapability() { - if (this.minimumResourceCapability != null) { - return this.minimumResourceCapability; - } - - GetNewApplicationResponseProtoOrBuilder p = viaProto ? proto : builder; - if (!p.hasMinimumCapability()) { - return null; - } - - this.minimumResourceCapability = convertFromProtoFormat(p.getMinimumCapability()); - return this.minimumResourceCapability; - } - - @Override - public void setMaximumResourceCapability(Resource capability) { - maybeInitBuilder(); - if(maximumResourceCapability == null) { - builder.clearMaximumCapability(); - } - this.maximumResourceCapability = capability; - } - - @Override - public void setMinimumResourceCapability(Resource capability) { - maybeInitBuilder(); - if(minimumResourceCapability == null) { - builder.clearMinimumCapability(); - } - this.minimumResourceCapability = capability; - } - - private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) { - return new ApplicationIdPBImpl(p); - } - - private ApplicationIdProto convertToProtoFormat(ApplicationId t) { - return ((ApplicationIdPBImpl)t).getProto(); - } - - private Resource convertFromProtoFormat(ResourceProto resource) { - return new ResourcePBImpl(resource); - } - - private ResourceProto convertToProtoFormat(Resource resource) { - return ((ResourcePBImpl)resource).getProto(); - } - -} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java index ffb920d5b90..ca7a6f415a1 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java @@ -186,16 +186,4 @@ public interface ApplicationReport { @Private @Unstable void setStartTime(long startTime); - - /** - * Get the finish time of the application. - * @return finish time of the application - */ - @Public - @Stable - long getFinishTime(); - - @Private - @Unstable - void setFinishTime(long finishTime); } 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 ff054b22ac5..97c84e4d10a 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 @@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.api.ContainerManager; * *
  • HTTP uri of the node.
  • *
  • {@link Resource} allocated to the container.
  • - *
  • {@link Priority} at which the container was allocated.
  • *
  • {@link ContainerState} of the container.
  • *
  • * {@link ContainerToken} of the container, used to securely verify @@ -112,18 +111,6 @@ public interface Container extends Comparable { @Private @Unstable void setResource(Resource resource); - - /** - * Get the Priority at which the Container was - * allocated. - * @return Priority at which the Container was - * allocated - */ - Priority getPriority(); - - @Private - @Unstable - void setPriority(Priority priority); /** * Get the current ContainerState of the container. diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java index 2ea2ddbcdb2..b1e80fc7598 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java @@ -240,30 +240,6 @@ implements ApplicationReport { return proto; } - @Override - public long getStartTime() { - ApplicationReportProtoOrBuilder p = viaProto ? proto : builder; - return p.getStartTime(); - } - - @Override - public void setStartTime(long startTime) { - maybeInitBuilder(); - builder.setStartTime(startTime); - } - - @Override - public long getFinishTime() { - ApplicationReportProtoOrBuilder p = viaProto ? proto : builder; - return p.getFinishTime(); - } - - @Override - public void setFinishTime(long finishTime) { - maybeInitBuilder(); - builder.setFinishTime(finishTime); - } - private void mergeLocalToBuilder() { if (this.applicationId != null && !((ApplicationIdPBImpl) this.applicationId).getProto().equals( @@ -303,4 +279,16 @@ implements ApplicationReport { ApplicationIdProto applicationId) { return new ApplicationIdPBImpl(applicationId); } + + @Override + public long getStartTime() { + ApplicationReportProtoOrBuilder p = viaProto ? proto : builder; + return p.getStartTime(); + } + + @Override + public void setStartTime(long startTime) { + maybeInitBuilder(); + builder.setStartTime(startTime); + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java index 39b15e0cefd..388cad0f4d8 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java @@ -25,7 +25,6 @@ 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; import org.apache.hadoop.yarn.api.records.ProtoBase; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; @@ -35,7 +34,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerTokenProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; -import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; import org.apache.hadoop.yarn.util.ProtoUtils; @@ -50,7 +48,6 @@ public class ContainerPBImpl extends ProtoBase implements Contai private ContainerId containerId = null; private NodeId nodeId = null; private Resource resource = null; - private Priority priority = null; private ContainerToken containerToken = null; private ContainerStatus containerStatus = null; @@ -87,11 +84,6 @@ public class ContainerPBImpl extends ProtoBase implements Contai builder.getResource())) { builder.setResource(convertToProtoFormat(this.resource)); } - if (this.priority != null && - !((PriorityPBImpl) this.priority).getProto().equals( - builder.getPriority())) { - builder.setPriority(convertToProtoFormat(this.priority)); - } if (this.containerToken != null && !((ContainerTokenPBImpl) this.containerToken).getProto().equals( builder.getContainerToken())) { @@ -219,29 +211,6 @@ public class ContainerPBImpl extends ProtoBase implements Contai builder.clearResource(); this.resource = resource; } - - @Override - public Priority getPriority() { - ContainerProtoOrBuilder p = viaProto ? proto : builder; - if (this.priority != null) { - return this.priority; - } - if (!p.hasPriority()) { - return null; - } - this.priority = convertFromProtoFormat(p.getPriority()); - return this.priority; - } - - @Override - public void setPriority(Priority priority) { - maybeInitBuilder(); - if (priority == null) { - builder.clearPriority(); - } - this.priority = priority; - } - @Override public ContainerToken getContainerToken() { ContainerProtoOrBuilder p = viaProto ? proto : builder; @@ -316,14 +285,6 @@ public class ContainerPBImpl extends ProtoBase implements Contai return ((ResourcePBImpl)t).getProto(); } - private PriorityPBImpl convertFromProtoFormat(PriorityProto p) { - return new PriorityPBImpl(p); - } - - private PriorityProto convertToProtoFormat(Priority p) { - return ((PriorityPBImpl)p).getProto(); - } - private ContainerTokenPBImpl convertFromProtoFormat(ContainerTokenProto p) { return new ContainerTokenPBImpl(p); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/client_RM_protocol.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/client_RM_protocol.proto index fb5f5f6e741..cfb14ff3518 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/client_RM_protocol.proto +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/client_RM_protocol.proto @@ -24,10 +24,10 @@ option java_generate_equals_and_hash = true; import "yarn_service_protos.proto"; service ClientRMProtocolService { - rpc getNewApplication (GetNewApplicationRequestProto) returns (GetNewApplicationResponseProto); + rpc getNewApplicationId (GetNewApplicationIdRequestProto) returns (GetNewApplicationIdResponseProto); rpc getApplicationReport (GetApplicationReportRequestProto) returns (GetApplicationReportResponseProto); rpc submitApplication (SubmitApplicationRequestProto) returns (SubmitApplicationResponseProto); - rpc forceKillApplication (KillApplicationRequestProto) returns (KillApplicationResponseProto); + rpc finishApplication (FinishApplicationRequestProto) returns (FinishApplicationResponseProto); rpc getClusterMetrics (GetClusterMetricsRequestProto) returns (GetClusterMetricsResponseProto); rpc getAllApplications (GetAllApplicationsRequestProto) returns (GetAllApplicationsResponseProto); rpc getClusterNodes (GetClusterNodesRequestProto) returns (GetClusterNodesResponseProto); 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 704c7109964..cdcd1a747b8 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 @@ -48,10 +48,6 @@ message ResourceProto { optional int32 memory = 1; } -message PriorityProto { - optional int32 priority = 1; -} - enum ContainerStateProto { C_NEW = 1; C_RUNNING = 2; @@ -70,10 +66,9 @@ message ContainerProto { optional NodeIdProto nodeId = 2; optional string node_http_address = 3; optional ResourceProto resource = 4; - optional PriorityProto priority = 5; - optional ContainerStateProto state = 6; - optional ContainerTokenProto container_token = 7; - optional ContainerStatusProto container_status = 8; + optional ContainerStateProto state = 5; + optional ContainerTokenProto container_token = 6; + optional ContainerStatusProto container_status = 7; } enum ApplicationStateProto { @@ -145,7 +140,6 @@ message ApplicationReportProto { optional string trackingUrl = 11; optional string diagnostics = 12 [default = "N/A"]; optional int64 startTime = 13; - optional int64 finishTime = 14; } message NodeIdProto { @@ -258,6 +252,10 @@ message ContainerStatusProto { //////////////////////////////////////////////////////////////////////// ////// From common////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////// +message PriorityProto { + optional int32 priority = 1; +} + message StringURLMapProto { optional string key = 1; optional URLProto value = 2; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto index 1a992ad578e..753c6b8c9a8 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto @@ -66,13 +66,11 @@ message AllocateResponseProto { /////// client_RM_Protocol /////////////////////////// ////////////////////////////////////////////////////// -message GetNewApplicationRequestProto { +message GetNewApplicationIdRequestProto { } -message GetNewApplicationResponseProto { +message GetNewApplicationIdResponseProto { optional ApplicationIdProto application_id = 1; - optional ResourceProto minimumCapability = 2; - optional ResourceProto maximumCapability = 3; } message GetApplicationReportRequestProto { @@ -90,11 +88,11 @@ message SubmitApplicationRequestProto { message SubmitApplicationResponseProto { } -message KillApplicationRequestProto { +message FinishApplicationRequestProto { optional ApplicationIdProto application_id = 1; } -message KillApplicationResponseProto { +message FinishApplicationResponseProto { } message GetClusterMetricsRequestProto { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java index b4f2dc46e0f..8972c656d91 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java @@ -25,6 +25,8 @@ import java.net.InetSocketAddress; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; @@ -33,16 +35,16 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAllApplicationsRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAllApplicationsResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportRequestPBImpl; @@ -51,28 +53,27 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsReque import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesResponsePBImpl; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationRequestPBImpl; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationIdRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationIdResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoResponsePBImpl; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRequestPBImpl; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine; import org.apache.hadoop.yarn.proto.ClientRMProtocol.ClientRMProtocolService; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterMetricsRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodesRequestProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationRequestProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationIdRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoResponseProto; import com.google.protobuf.ServiceException; @@ -87,11 +88,11 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol { } @Override - public KillApplicationResponse forceKillApplication( - KillApplicationRequest request) throws YarnRemoteException { - KillApplicationRequestProto requestProto = ((KillApplicationRequestPBImpl)request).getProto(); + public FinishApplicationResponse finishApplication( + FinishApplicationRequest request) throws YarnRemoteException { + FinishApplicationRequestProto requestProto = ((FinishApplicationRequestPBImpl)request).getProto(); try { - return new KillApplicationResponsePBImpl(proxy.forceKillApplication(null, requestProto)); + return new FinishApplicationResponsePBImpl(proxy.finishApplication(null, requestProto)); } catch (ServiceException e) { if (e.getCause() instanceof YarnRemoteException) { throw (YarnRemoteException)e.getCause(); @@ -138,11 +139,11 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol { } @Override - public GetNewApplicationResponse getNewApplication( - GetNewApplicationRequest request) throws YarnRemoteException { - GetNewApplicationRequestProto requestProto = ((GetNewApplicationRequestPBImpl)request).getProto(); + public GetNewApplicationIdResponse getNewApplicationId( + GetNewApplicationIdRequest request) throws YarnRemoteException { + GetNewApplicationIdRequestProto requestProto = ((GetNewApplicationIdRequestPBImpl)request).getProto(); try { - return new GetNewApplicationResponsePBImpl(proxy.getNewApplication(null, requestProto)); + return new GetNewApplicationIdResponsePBImpl(proxy.getNewApplicationId(null, requestProto)); } catch (ServiceException e) { if (e.getCause() instanceof YarnRemoteException) { throw (YarnRemoteException)e.getCause(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java index 342d864ca5d..35e4be53984 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ClientRMProtocolPBServiceImpl.java @@ -19,15 +19,17 @@ package org.apache.hadoop.yarn.api.impl.pb.service; import org.apache.hadoop.yarn.api.ClientRMProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAllApplicationsRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAllApplicationsResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportRequestPBImpl; @@ -36,18 +38,18 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsReque import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesResponsePBImpl; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationRequestPBImpl; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationIdRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationIdResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoResponsePBImpl; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRequestPBImpl; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.proto.ClientRMProtocol.ClientRMProtocolService.BlockingInterface; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto; @@ -56,14 +58,12 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterMetricsRequestPr import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterMetricsResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodesRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodesResponseProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationRequestProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationIdRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationIdResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoResponseProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseProto; @@ -79,12 +79,12 @@ public class ClientRMProtocolPBServiceImpl implements BlockingInterface { } @Override - public KillApplicationResponseProto forceKillApplication(RpcController arg0, - KillApplicationRequestProto proto) throws ServiceException { - KillApplicationRequestPBImpl request = new KillApplicationRequestPBImpl(proto); + public FinishApplicationResponseProto finishApplication(RpcController arg0, + FinishApplicationRequestProto proto) throws ServiceException { + FinishApplicationRequestPBImpl request = new FinishApplicationRequestPBImpl(proto); try { - KillApplicationResponse response = real.forceKillApplication(request); - return ((KillApplicationResponsePBImpl)response).getProto(); + FinishApplicationResponse response = real.finishApplication(request); + return ((FinishApplicationResponsePBImpl)response).getProto(); } catch (YarnRemoteException e) { throw new ServiceException(e); } @@ -116,13 +116,13 @@ public class ClientRMProtocolPBServiceImpl implements BlockingInterface { } @Override - public GetNewApplicationResponseProto getNewApplication( - RpcController arg0, GetNewApplicationRequestProto proto) + public GetNewApplicationIdResponseProto getNewApplicationId( + RpcController arg0, GetNewApplicationIdRequestProto proto) throws ServiceException { - GetNewApplicationRequestPBImpl request = new GetNewApplicationRequestPBImpl(proto); + GetNewApplicationIdRequestPBImpl request = new GetNewApplicationIdRequestPBImpl(proto); try { - GetNewApplicationResponse response = real.getNewApplication(request); - return ((GetNewApplicationResponsePBImpl)response).getProto(); + GetNewApplicationIdResponse response = real.getNewApplicationId(request); + return ((GetNewApplicationIdResponsePBImpl)response).getProto(); } catch (YarnRemoteException e) { throw new ServiceException(e); } 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 cb955af8c43..ba23134170f 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 @@ -357,12 +357,6 @@ public class YarnConfiguration extends Configuration { public static final String NM_AUX_SERVICE_FMT = NM_PREFIX + "aux-services.%s.class"; - public static final String NM_USER_HOME_DIR = - NM_PREFIX + "user-home-dir"; - - public static final String DEFAULT_NM_USER_HOME_DIR= "/home/"; - - public static final int INVALID_CONTAINER_EXIT_STATUS = -1000; public static final int ABORTED_CONTAINER_EXIT_STATUS = -100; @@ -386,6 +380,6 @@ public class YarnConfiguration extends Configuration { // Use apps manager address to figure out the host for webapp addr = conf.get(YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS); String host = ADDR_SPLITTER.split(addr).iterator().next(); - return JOINER.join("http://", host, ":", port); + return JOINER.join("http://", host, ":", port, "/"); } } 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 9d8b846a3b6..9a623a1a8a7 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 @@ -320,12 +320,6 @@ public class ProtoOverHadoopRpcEngine implements RpcEngine { + methodName); MethodDescriptor methodDescriptor = service.getDescriptorForType() .findMethodByName(methodName); - if (methodDescriptor == null) { - String msg = "Unknown method " + methodName + " called on " - + protocol + " protocol."; - LOG.warn(msg); - return handleException(new IOException(msg)); - } Message prototype = service.getRequestPrototype(methodDescriptor); Message param = prototype.newBuilderForType() .mergeFrom(rpcRequest.getRequestProto()).build(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java index 71c829ac582..2a5244d6512 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java @@ -20,14 +20,10 @@ package org.apache.hadoop.yarn.state; import java.util.EnumMap; import java.util.HashMap; -import java.util.Iterator; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; import java.util.Stack; -import org.apache.hadoop.yarn.util.Graph; - /** * State machine topology. * This object is semantically immutable. If you have a @@ -445,39 +441,4 @@ final public class StateMachineFactory return currentState; } } - - /** - * Generate a graph represents the state graph of this StateMachine - * @param name graph name - * @return Graph object generated - */ - public Graph generateStateGraph(String name) { - maybeMakeStateMachineTable(); - Graph g = new Graph(name); - for (STATE startState : stateMachineTable.keySet()) { - Map> transitions - = stateMachineTable.get(startState); - for (Entry> entry : - transitions.entrySet()) { - Transition transition = entry.getValue(); - if (transition instanceof StateMachineFactory.SingleInternalArc) { - StateMachineFactory.SingleInternalArc sa - = (StateMachineFactory.SingleInternalArc) transition; - Graph.Node fromNode = g.getNode(startState.toString()); - Graph.Node toNode = g.getNode(sa.postState.toString()); - fromNode.addEdge(toNode, entry.getKey().toString()); - } else if (transition instanceof StateMachineFactory.MultipleInternalArc) { - StateMachineFactory.MultipleInternalArc ma - = (StateMachineFactory.MultipleInternalArc) transition; - Iterator iter = ma.validPostStates.iterator(); - while (iter.hasNext()) { - Graph.Node fromNode = g.getNode(startState.toString()); - Graph.Node toNode = g.getNode(iter.next().toString()); - fromNode.addEdge(toNode, entry.getKey().toString()); - } - } - } - } - return g; - } } 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 7ec367292e1..4eb63c04470 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 @@ -20,9 +20,7 @@ package org.apache.hadoop.yarn.util; import java.net.URI; import java.util.Comparator; -import java.util.List; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; @@ -186,31 +184,32 @@ public class BuilderUtils { return id; } - public static NodeId newNodeId(String host, int port) { - NodeId nodeId = recordFactory.newRecordInstance(NodeId.class); - nodeId.setHost(host); - nodeId.setPort(port); - return nodeId; + public static Container clone(Container c) { + Container container = recordFactory.newRecordInstance(Container.class); + container.setId(c.getId()); + container.setContainerToken(c.getContainerToken()); + container.setNodeId(c.getNodeId()); + container.setNodeHttpAddress(c.getNodeHttpAddress()); + container.setResource(c.getResource()); + container.setState(c.getState()); + return container; } public static Container newContainer(RecordFactory recordFactory, ApplicationAttemptId appAttemptId, int containerId, NodeId nodeId, - String nodeHttpAddress, Resource resource, Priority priority) { + String nodeHttpAddress, Resource resource) { ContainerId containerID = newContainerId(recordFactory, appAttemptId, containerId); - return newContainer(containerID, nodeId, nodeHttpAddress, - resource, priority); + return newContainer(containerID, nodeId, nodeHttpAddress, resource); } public static Container newContainer(ContainerId containerId, - NodeId nodeId, String nodeHttpAddress, - Resource resource, Priority priority) { + NodeId nodeId, String nodeHttpAddress, Resource resource) { Container container = recordFactory.newRecordInstance(Container.class); container.setId(containerId); container.setNodeId(nodeId); container.setNodeHttpAddress(nodeHttpAddress); container.setResource(resource); - container.setPriority(priority); container.setState(ContainerState.NEW); ContainerStatus containerStatus = Records.newRecord(ContainerStatus.class); containerStatus.setContainerId(containerId); @@ -243,7 +242,7 @@ public class BuilderUtils { public static ApplicationReport newApplicationReport( ApplicationId applicationId, String user, String queue, String name, String host, int rpcPort, String clientToken, ApplicationState state, - String diagnostics, String url, long startTime, long finishTime) { + String diagnostics, String url, long startTime) { ApplicationReport report = recordFactory .newRecordInstance(ApplicationReport.class); report.setApplicationId(applicationId); @@ -257,7 +256,6 @@ public class BuilderUtils { report.setDiagnostics(diagnostics); report.setTrackingUrl(url); report.setStartTime(startTime); - report.setFinishTime(finishTime); return report; } @@ -275,18 +273,5 @@ public class BuilderUtils { url.setFile(file); return url; } - - public static AllocateRequest newAllocateRequest( - ApplicationAttemptId applicationAttemptId, int responseID, - float appProgress, List resourceAsk, - List containersToBeReleased) { - AllocateRequest allocateRequest = recordFactory - .newRecordInstance(AllocateRequest.class); - allocateRequest.setApplicationAttemptId(applicationAttemptId); - allocateRequest.setResponseId(responseID); - allocateRequest.setProgress(appProgress); - allocateRequest.addAllAsks(resourceAsk); - allocateRequest.addAllReleases(containersToBeReleased); - return allocateRequest; - } + } 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 6f5e9043192..ab6bd7395dc 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 @@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.util; import static org.apache.hadoop.yarn.util.StringHelper._split; -import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; import java.text.NumberFormat; @@ -46,8 +45,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; public class ConverterUtils { public static final String APPLICATION_PREFIX = "application"; - public static final String CONTAINER_PREFIX = "container"; - public static final String APPLICATION_ATTEMPT_PREFIX = "appattempt"; /** * return a hadoop path from a given url @@ -135,12 +132,14 @@ public class ConverterUtils { } private static ApplicationAttemptId toApplicationAttemptId( - Iterator it) throws NumberFormatException { - ApplicationId appId = Records.newRecord(ApplicationId.class); + RecordFactory recordFactory, + Iterator it) { + ApplicationId appId = + recordFactory.newRecordInstance(ApplicationId.class); appId.setClusterTimestamp(Long.parseLong(it.next())); appId.setId(Integer.parseInt(it.next())); - ApplicationAttemptId appAttemptId = Records - .newRecord(ApplicationAttemptId.class); + ApplicationAttemptId appAttemptId = + recordFactory.newRecordInstance(ApplicationAttemptId.class); appAttemptId.setApplicationId(appId); appAttemptId.setAttemptId(Integer.parseInt(it.next())); return appAttemptId; @@ -150,35 +149,16 @@ public class ConverterUtils { return cId.toString(); } - public static ContainerId toContainerId(String containerIdStr) - throws IOException { + public static ContainerId toContainerId(RecordFactory recordFactory, + String containerIdStr) { Iterator it = _split(containerIdStr).iterator(); - if (!it.next().equals(CONTAINER_PREFIX)) { - throw new IOException("Invalid ContainerId prefix: " + containerIdStr); - } - try { - ApplicationAttemptId appAttemptID = toApplicationAttemptId(it); - ContainerId containerId = Records.newRecord(ContainerId.class); - containerId.setApplicationAttemptId(appAttemptID); - containerId.setId(Integer.parseInt(it.next())); - return containerId; - } catch (NumberFormatException n) { - throw new IOException("Invalid ContainerId: " + containerIdStr, n); - } - } - - public static ApplicationAttemptId toApplicationAttemptId( - String applicationAttmeptIdStr) throws IOException { - Iterator it = _split(applicationAttmeptIdStr).iterator(); - if (!it.next().equals(APPLICATION_ATTEMPT_PREFIX)) { - throw new IOException("Invalid AppAttemptId prefix: " - + applicationAttmeptIdStr); - } - try { - return toApplicationAttemptId(it); - } catch (NumberFormatException n) { - throw new IOException("Invalid AppAttemptId: " - + applicationAttmeptIdStr, n); - } + it.next(); // prefix. TODO: Validate container prefix + ApplicationAttemptId appAttemptID = + toApplicationAttemptId(recordFactory, it); + ContainerId containerId = + recordFactory.newRecordInstance(ContainerId.class); + 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/Graph.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Graph.java deleted file mode 100644 index aa3604fa87a..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Graph.java +++ /dev/null @@ -1,210 +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.util; - -import java.io.FileWriter; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.apache.commons.lang.StringEscapeUtils; - -public class Graph { - public class Edge { - Node from; - Node to; - String label; - - public Edge(Node from, Node to, String info) { - this.from = from; - this.to = to; - this.label = info; - } - - public boolean sameAs(Edge rhs) { - if (this.from == rhs.from && - this.to == rhs.to) { - return true; - } - return false; - } - - public Edge combine(Edge rhs) { - String newlabel = this.label + "," + rhs.label; - return new Edge(this.from, this.to, newlabel); - } - } - - public class Node { - Graph parent; - String id; - List ins; - List outs; - - public Node(String id) { - this.id = id; - this.parent = Graph.this; - this.ins = new ArrayList(); - this.outs = new ArrayList(); - } - - public Graph getParent() { - return parent; - } - - public Node addEdge(Node to, String info) { - Edge e = new Edge(this, to, info); - outs.add(e); - to.ins.add(e); - return this; - } - - public String getUniqueId() { - return Graph.this.name + "." + id; - } - } - - private String name; - private Graph parent; - private Set nodes = new HashSet(); - private Set subgraphs = new HashSet(); - - public Graph(String name, Graph parent) { - this.name = name; - this.parent = parent; - } - - public Graph(String name) { - this(name, null); - } - - public Graph() { - this("graph", null); - } - - public String getName() { - return name; - } - - public Graph getParent() { - return parent; - } - - private Node newNode(String id) { - Node ret = new Node(id); - nodes.add(ret); - return ret; - } - - public Node getNode(String id) { - for (Node node : nodes) { - if (node.id.equals(id)) { - return node; - } - } - return newNode(id); - } - - public Graph newSubGraph(String name) { - Graph ret = new Graph(name, this); - subgraphs.add(ret); - return ret; - } - - public void addSubGraph(Graph graph) { - subgraphs.add(graph); - graph.parent = this; - } - - private static String wrapSafeString(String label) { - if (label.indexOf(',') >= 0) { - if (label.length()>14) { - label = label.replaceAll(",", ",\n"); - } - } - label = "\"" + StringEscapeUtils.escapeJava(label) + "\""; - return label; - } - - public String generateGraphViz(String indent) { - StringBuilder sb = new StringBuilder(); - if (this.parent == null) { - sb.append("digraph " + name + " {\n"); - sb.append(String.format("graph [ label=%s, fontsize=24, fontname=Helvetica];\n", - wrapSafeString(name))); - sb.append("node [fontsize=12, fontname=Helvetica];\n"); - sb.append("edge [fontsize=9, fontcolor=blue, fontname=Arial];\n"); - } else { - sb.append("subgraph cluster_" + name + " {\nlabel=\"" + name + "\"\n"); - } - for (Graph g : subgraphs) { - String ginfo = g.generateGraphViz(indent+" "); - sb.append(ginfo); - sb.append("\n"); - } - for (Node n : nodes) { - sb.append(String.format( - "%s%s [ label = %s ];\n", - indent, - wrapSafeString(n.getUniqueId()), - n.id)); - List combinedOuts = combineEdges(n.outs); - for (Edge e : combinedOuts) { - sb.append(String.format( - "%s%s -> %s [ label = %s ];\n", - indent, - wrapSafeString(e.from.getUniqueId()), - wrapSafeString(e.to.getUniqueId()), - wrapSafeString(e.label))); - } - } - sb.append("}\n"); - return sb.toString(); - } - - public String generateGraphViz() { - return generateGraphViz(""); - } - - public void save(String filepath) throws IOException { - FileWriter fout = new FileWriter(filepath); - fout.write(generateGraphViz()); - fout.close(); - } - - public static List combineEdges(List edges) { - List ret = new ArrayList(); - for (Edge edge : edges) { - boolean found = false; - for (int i = 0; i < ret.size(); i++) { - Edge current = ret.get(i); - if (edge.sameAs(current)) { - ret.set(i, current.combine(edge)); - found = true; - break; - } - } - if (!found) { - ret.add(edge); - } - } - return ret; - } -} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/VisualizeStateMachine.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/VisualizeStateMachine.java deleted file mode 100644 index 0fb9a48b098..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/VisualizeStateMachine.java +++ /dev/null @@ -1,73 +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.util; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.yarn.state.StateMachineFactory; - -public class VisualizeStateMachine { - - /** - * @param classes list of classes which have static field - * stateMachineFactory of type StateMachineFactory - * @return graph represent this StateMachine - */ - public static Graph getGraphFromClasses(String graphName, List classes) - throws Exception { - Graph ret = null; - if (classes.size() != 1) { - ret = new Graph(graphName); - } - for (String className : classes) { - Class clz = Class.forName(className); - Field factoryField = clz.getDeclaredField("stateMachineFactory"); - factoryField.setAccessible(true); - StateMachineFactory factory = (StateMachineFactory) factoryField.get(null); - if (classes.size() == 1) { - return factory.generateStateGraph(graphName); - } - String gname = clz.getSimpleName(); - if (gname.endsWith("Impl")) { - gname = gname.substring(0, gname.length()-4); - } - ret.addSubGraph(factory.generateStateGraph(gname)); - } - return ret; - } - - public static void main(String [] args) throws Exception { - if (args.length < 3) { - System.err.printf("Usage: %s \n", - VisualizeStateMachine.class.getName()); - System.exit(1); - } - String [] classes = args[1].split(","); - ArrayList validClasses = new ArrayList(); - for (String c : classes) { - String vc = c.trim(); - if (vc.length()>0) { - validClasses.add(vc); - } - } - Graph g = getGraphFromClasses(args[0], validClasses); - g.save(args[2]); - } -} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Dispatcher.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Dispatcher.java index e404fe5a723..ef8ab976ef8 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Dispatcher.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Dispatcher.java @@ -84,15 +84,6 @@ public class Dispatcher extends HttpServlet { prepareToExit(); return; } - // if they provide a redirectPath go there instead of going to - // "/" so that filters can differentiate the webapps. - if (uri.equals("/")) { - String redirectPath = webApp.getRedirectPath(); - if (redirectPath != null && !redirectPath.isEmpty()) { - res.sendRedirect(redirectPath); - return; - } - } String method = req.getMethod(); if (method.equals("OPTIONS")) { doOptions(req, res); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java index f83843e97e6..b9afe81ca85 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java @@ -26,7 +26,6 @@ import com.google.inject.Provides; import com.google.inject.servlet.GuiceFilter; import com.google.inject.servlet.ServletModule; -import java.util.ArrayList; import java.util.List; import org.apache.hadoop.conf.Configuration; @@ -45,9 +44,6 @@ public abstract class WebApp extends ServletModule { public enum HTTP { GET, POST, HEAD, PUT, DELETE }; private volatile String name; - private volatile List servePathSpecs = new ArrayList(); - // path to redirect to if user goes to "/" - private volatile String redirectPath; private volatile Configuration conf; private volatile HttpServer httpServer; private volatile GuiceFilter guiceFilter; @@ -102,22 +98,6 @@ public abstract class WebApp extends ServletModule { public String name() { return this.name; } - void addServePathSpec(String path) { this.servePathSpecs.add(path); } - - public String[] getServePathSpecs() { - return this.servePathSpecs.toArray(new String[this.servePathSpecs.size()]); - } - - /** - * Set a path to redirect the user to if they just go to "/". For - * instance "/" goes to "/yarn/apps". This allows the filters to - * more easily differentiate the different webapps. - * @param path the path to redirect to - */ - void setRedirectPath(String path) { this.redirectPath = path; } - - public String getRedirectPath() { return this.redirectPath; } - void setHostClass(Class cls) { router.setHostClass(cls); } @@ -129,10 +109,7 @@ public abstract class WebApp extends ServletModule { @Override public void configureServlets() { setup(); - serve("/", "/__stop").with(Dispatcher.class); - for (String path : this.servePathSpecs) { - serve(path).with(Dispatcher.class); - } + serve("/", "/__stop", StringHelper.join('/', name, '*')).with(Dispatcher.class); } /** 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 b5217999687..85b88d16cc4 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 @@ -113,14 +113,6 @@ public class WebApps { }; } webapp.setName(name); - String basePath = "/" + name; - webapp.setRedirectPath(basePath); - if (basePath.equals("/")) { - webapp.addServePathSpec("/*"); - } else { - webapp.addServePathSpec(basePath); - webapp.addServePathSpec(basePath + "/*"); - } if (conf == null) { conf = new Configuration(); } @@ -150,8 +142,7 @@ public class WebApps { } } HttpServer server = - new HttpServer(name, bindAddress, port, findPort, conf, - webapp.getServePathSpecs()); + new HttpServer(name, bindAddress, port, findPort, conf); server.addGlobalFilter("guice", GuiceFilter.class.getName(), null); webapp.setConf(conf); webapp.setHttpServer(server); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/cluster/.keep b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/cluster/.keep deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/jobhistory/.keep b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/jobhistory/.keep deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/mapreduce/.keep b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/mapreduce/.keep deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/node/.keep b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/node/.keep deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/MockApps.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/MockApps.java index 7d233e2d9fc..65f6c548fbc 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/MockApps.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/MockApps.java @@ -167,16 +167,6 @@ public class MockApps { // TODO Auto-generated method stub } - @Override - public long getFinishTime() { - // TODO Auto-generated method stub - return 0; - } - @Override - public void setFinishTime(long finishTime) { - // TODO Auto-generated method stub - - } }; } 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 a855cc6f218..58efcc42307 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 @@ -25,11 +25,9 @@ import junit.framework.Assert; import org.apache.avro.ipc.Server; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.yarn.api.ClientRMProtocol; import org.apache.hadoop.yarn.api.ContainerManager; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; @@ -49,7 +47,6 @@ import org.apache.hadoop.yarn.factory.providers.YarnRemoteExceptionFactoryProvid import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC; import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.util.Records; import org.junit.Test; public class TestRPC { @@ -68,35 +65,6 @@ public class TestRPC { // test(HadoopYarnRPC.class.getName()); // } - @Test - public void testUnknownCall() { - Configuration conf = new Configuration(); - conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class - .getName()); - YarnRPC rpc = YarnRPC.create(conf); - String bindAddr = "localhost:0"; - InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); - Server server = rpc.getServer(ContainerManager.class, - new DummyContainerManager(), addr, conf, null, 1); - server.start(); - - // Any unrelated protocol would do - ClientRMProtocol proxy = (ClientRMProtocol) rpc.getProxy( - ClientRMProtocol.class, NetUtils.createSocketAddr("localhost:" - + server.getPort()), conf); - - try { - proxy.getNewApplication(Records - .newRecord(GetNewApplicationRequest.class)); - Assert.fail("Excepted RPC call to fail with unknown method."); - } catch (YarnRemoteException e) { - Assert.assertTrue(e.getMessage().matches( - "Unknown method getNewApplication called on.*" - + "org.apache.hadoop.yarn.proto.ClientRMProtocol" - + "\\$ClientRMProtocolService\\$BlockingInterface protocol.")); - } - } - @Test public void testHadoopProtoRPC() throws Exception { test(HadoopYarnProtoRPC.class.getName()); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java deleted file mode 100644 index 3d2a5769097..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java +++ /dev/null @@ -1,54 +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.conf; - -import java.net.InetSocketAddress; - -import junit.framework.Assert; - -import org.apache.avro.ipc.Server; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.junit.Test; - -public class TestYarnConfiguration { - - @Test - public void testDefaultRMWebUrl() throws Exception { - YarnConfiguration conf = new YarnConfiguration(); - String rmWebUrl = YarnConfiguration.getRMWebAppURL(conf); - // shouldn't have a "/" on the end of the url as all the other uri routinnes - // specifically add slashes and Jetty doesn't handle double slashes. - Assert.assertEquals("RM Web Url is not correct", "http://0.0.0.0:8088", - rmWebUrl); - } - - @Test - public void testRMWebUrlSpecified() throws Exception { - YarnConfiguration conf = new YarnConfiguration(); - // seems a bit odd but right now we are forcing webapp for RM to be RM_ADDRESS - // for host and use the port from the RM_WEBAPP_ADDRESS - conf.set(YarnConfiguration.RM_WEBAPP_ADDRESS, "footesting:99110"); - conf.set(YarnConfiguration.RM_ADDRESS, "rmtesting:9999"); - String rmWebUrl = YarnConfiguration.getRMWebAppURL(conf); - Assert.assertEquals("RM Web Url is not correct", "http://rmtesting:99110", - rmWebUrl); - } - -} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java index 31b2aaa2ed6..db84f32cf64 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.webapp; -import org.apache.commons.lang.ArrayUtils; import org.apache.hadoop.yarn.MockApps; import org.apache.hadoop.yarn.webapp.Controller; import org.apache.hadoop.yarn.webapp.WebApp; @@ -149,32 +148,6 @@ public class TestWebApp { app.stop(); } - @Test public void testServePaths() { - WebApp app = WebApps.$for("test", this).start(); - assertEquals("/test", app.getRedirectPath()); - String[] expectedPaths = { "/test", "/test/*" }; - String[] pathSpecs = app.getServePathSpecs(); - - assertEquals(2, pathSpecs.length); - for(int i = 0; i < expectedPaths.length; i++) { - assertTrue(ArrayUtils.contains(pathSpecs, expectedPaths[i])); - } - app.stop(); - } - - @Test public void testServePathsNoName() { - WebApp app = WebApps.$for("", this).start(); - assertEquals("/", app.getRedirectPath()); - String[] expectedPaths = { "/*" }; - String[] pathSpecs = app.getServePathSpecs(); - - assertEquals(1, pathSpecs.length); - for(int i = 0; i < expectedPaths.length; i++) { - assertTrue(ArrayUtils.contains(pathSpecs, expectedPaths[i])); - } - app.stop(); - } - @Test public void testDefaultRoutes() throws Exception { WebApp app = WebApps.$for("test", this).start(); String baseUrl = baseUrl(app); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml index 1d7b9cb2d1f..98959644cf6 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml @@ -103,39 +103,6 @@ true - - visualize - - false - - - - - org.codehaus.mojo - exec-maven-plugin - 1.2 - - - compile - - java - - - org.apache.hadoop.yarn.util.VisualizeStateMachine - - NodeManager - org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl, - org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl, - org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizedResource - NodeManager.gv - - - - - - - - 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 83872876797..a7e82a2d41a 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 @@ -133,10 +133,8 @@ public class DefaultContainerExecutor extends ContainerExecutor { String[] command = new String[] { "bash", "-c", launchDst.toUri().getPath().toString() }; LOG.info("launchContainer: " + Arrays.toString(command)); - shExec = new ShellCommandExecutor( - command, - new File(containerWorkDir.toUri().getPath()), - container.getLaunchContext().getEnvironment()); // sanitized env + shExec = new ShellCommandExecutor(command, + new File(containerWorkDir.toUri().getPath())); launchCommandObjs.put(containerId, shExec); shExec.execute(); } catch (IOException e) { 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 0779d3b1581..97721f72a36 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 @@ -161,11 +161,7 @@ public class LinuxContainerExecutor extends ContainerExecutor { nmPrivateCotainerScriptPath.toUri().getPath().toString(), nmPrivateTokensPath.toUri().getPath().toString())); String[] commandArray = command.toArray(new String[command.size()]); - ShellCommandExecutor shExec = - new ShellCommandExecutor( - commandArray, - null, // NM's cwd - container.getLaunchContext().getEnvironment()); // sanitized env + ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray); launchCommandObjs.put(containerId, shExec); // DEBUG LOG.info("launchContainer: " + Arrays.toString(commandArray)); 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 1b1fd46b9e7..641e74b8018 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 @@ -101,7 +101,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements public synchronized void init(Configuration conf) { this.rmAddress = conf.get(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS, - YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS); + YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS); this.heartBeatInterval = conf.getLong(YarnConfiguration.NM_TO_RM_HEARTBEAT_INTERVAL_MS, YarnConfiguration.DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS); 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 8d3f3fe0842..4e02c3adede 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 @@ -158,12 +158,10 @@ public class ContainerImpl implements Container { ContainerEventType.CONTAINER_LAUNCHED, new LaunchTransition()) .addTransition(ContainerState.LOCALIZED, ContainerState.EXITED_WITH_FAILURE, ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, - new ExitedWithFailureTransition(true)) + new ExitedWithFailureTransition()) .addTransition(ContainerState.LOCALIZED, ContainerState.LOCALIZED, ContainerEventType.UPDATE_DIAGNOSTICS_MSG, UPDATE_DIAGNOSTICS_TRANSITION) - // TODO race: Can lead to a CONTAINER_LAUNCHED event at state KILLING, - // and a container which will never be killed by the NM. .addTransition(ContainerState.LOCALIZED, ContainerState.KILLING, ContainerEventType.KILL_CONTAINER, new KillTransition()) @@ -171,19 +169,16 @@ public class ContainerImpl implements Container { .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_SUCCESS, ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS, - new ExitedWithSuccessTransition(true)) + new ExitedWithSuccessTransition()) .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE, ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, - new ExitedWithFailureTransition(true)) + new ExitedWithFailureTransition()) .addTransition(ContainerState.RUNNING, ContainerState.RUNNING, ContainerEventType.UPDATE_DIAGNOSTICS_MSG, UPDATE_DIAGNOSTICS_TRANSITION) .addTransition(ContainerState.RUNNING, ContainerState.KILLING, ContainerEventType.KILL_CONTAINER, new KillTransition()) - .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE, - ContainerEventType.CONTAINER_KILLED_ON_REQUEST, - new KilledExternallyTransition()) // From CONTAINER_EXITED_WITH_SUCCESS State .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE, @@ -225,10 +220,10 @@ public class ContainerImpl implements Container { ContainerEventType.KILL_CONTAINER) .addTransition(ContainerState.KILLING, ContainerState.EXITED_WITH_SUCCESS, ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS, - new ExitedWithSuccessTransition(false)) + new ExitedWithSuccessTransition()) .addTransition(ContainerState.KILLING, ContainerState.EXITED_WITH_FAILURE, ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, - new ExitedWithFailureTransition(false)) + new ExitedWithFailureTransition()) .addTransition(ContainerState.KILLING, ContainerState.DONE, ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP, @@ -556,41 +551,18 @@ public class ContainerImpl implements Container { } } - @SuppressWarnings("unchecked") // dispatcher not typed static class ExitedWithSuccessTransition extends ContainerTransition { - - boolean clCleanupRequired; - - public ExitedWithSuccessTransition(boolean clCleanupRequired) { - this.clCleanupRequired = clCleanupRequired; - } - @Override public void transition(ContainerImpl container, ContainerEvent event) { - // Set exit code to 0 on success - container.exitCode = 0; - // TODO: Add containerWorkDir to the deletion service. - if (clCleanupRequired) { - container.dispatcher.getEventHandler().handle( - new ContainersLauncherEvent(container, - ContainersLauncherEventType.CLEANUP_CONTAINER)); - } - + // Inform the localizer to decrement reference counts and cleanup + // resources. container.cleanup(); } } - @SuppressWarnings("unchecked") // dispatcher not typed static class ExitedWithFailureTransition extends ContainerTransition { - - boolean clCleanupRequired; - - public ExitedWithFailureTransition(boolean clCleanupRequired) { - this.clCleanupRequired = clCleanupRequired; - } - @Override public void transition(ContainerImpl container, ContainerEvent event) { ContainerExitEvent exitEvent = (ContainerExitEvent) event; @@ -599,28 +571,12 @@ public class ContainerImpl implements Container { // TODO: Add containerWorkDir to the deletion service. // TODO: Add containerOuputDir to the deletion service. - if (clCleanupRequired) { - container.dispatcher.getEventHandler().handle( - new ContainersLauncherEvent(container, - ContainersLauncherEventType.CLEANUP_CONTAINER)); - } - + // Inform the localizer to decrement reference counts and cleanup + // resources. container.cleanup(); } } - static class KilledExternallyTransition extends ExitedWithFailureTransition { - KilledExternallyTransition() { - super(true); - } - - @Override - public void transition(ContainerImpl container, ContainerEvent event) { - super.transition(container, event); - container.diagnostics.append("Killed by external signal\n"); - } - } - static class ResourceFailedTransition implements SingleArcTransition { @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/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 43afa4cb85e..497460d3e7d 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 @@ -44,7 +44,6 @@ import org.apache.hadoop.security.Credentials; import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; @@ -90,6 +89,7 @@ public class ContainerLaunch implements Callable { final Map localResources = container.getLocalizedResources(); String containerIdStr = ConverterUtils.toString(container.getContainerID()); final String user = launchContext.getUser(); + final Map env = launchContext.getEnvironment(); final List command = launchContext.getCommands(); int ret = -1; @@ -109,16 +109,16 @@ public class ContainerLaunch implements Callable { } launchContext.setCommands(newCmds); - Map environment = launchContext.getEnvironment(); - // Make a copy of env to iterate & do variable expansion - for (Entry entry : environment.entrySet()) { - String value = entry.getValue(); - entry.setValue( - value.replace( + Map envs = launchContext.getEnvironment(); + Map newEnvs = new HashMap(envs.size()); + for (Entry entry : envs.entrySet()) { + newEnvs.put( + entry.getKey(), + entry.getValue().replace( ApplicationConstants.LOG_DIR_EXPANSION_VAR, - containerLogDir.toUri().getPath()) - ); + containerLogDir.toUri().getPath())); } + launchContext.setEnvironment(newEnvs); // /////////////////////////// End of variable expansion FileContext lfs = FileContext.getLocalFSFileContext(); @@ -164,18 +164,11 @@ public class ContainerLaunch implements Callable { EnumSet.of(CREATE, OVERWRITE)); // Set the token location too. - environment.put( - ApplicationConstants.CONTAINER_TOKEN_FILE_ENV_NAME, - new Path(containerWorkDir, - FINAL_CONTAINER_TOKENS_FILE).toUri().getPath()); + env.put(ApplicationConstants.CONTAINER_TOKEN_FILE_ENV_NAME, new Path( + containerWorkDir, FINAL_CONTAINER_TOKENS_FILE).toUri().getPath()); - // Sanitize the container's environment - sanitizeEnv(environment, containerWorkDir, appDirs); - - // Write out the environment - writeLaunchEnv(containerScriptOutStream, environment, localResources, - launchContext.getCommands()); - + writeLaunchEnv(containerScriptOutStream, env, localResources, + launchContext.getCommands(), appDirs); // /////////// End of writing out container-script // /////////// Write out the container-tokens in the nmPrivate space. @@ -282,71 +275,19 @@ public class ContainerLaunch implements Callable { } - private static void putEnvIfNotNull( - Map environment, String variable, String value) { - if (value != null) { - environment.put(variable, value); - } - } - - private static void putEnvIfAbsent( - Map environment, String variable) { - if (environment.get(variable) == null) { - putEnvIfNotNull(environment, variable, System.getenv(variable)); - } - } - - public void sanitizeEnv(Map environment, - Path pwd, List appDirs) { - /** - * Non-modifiable environment variables - */ - - putEnvIfNotNull(environment, Environment.USER.name(), container.getUser()); - - putEnvIfNotNull(environment, - Environment.LOGNAME.name(),container.getUser()); - - putEnvIfNotNull(environment, - Environment.HOME.name(), - conf.get( - YarnConfiguration.NM_USER_HOME_DIR, - YarnConfiguration.DEFAULT_NM_USER_HOME_DIR - ) - ); - - putEnvIfNotNull(environment, Environment.PWD.name(), pwd.toString()); - - putEnvIfNotNull(environment, - Environment.HADOOP_CONF_DIR.name(), - System.getenv(Environment.HADOOP_CONF_DIR.name()) - ); - - putEnvIfNotNull(environment, - ApplicationConstants.LOCAL_DIR_ENV, - StringUtils.join(",", appDirs) - ); - - if (!Shell.WINDOWS) { - environment.put("JVM_PID", "$$"); - } - - /** - * Modifiable environment variables - */ - - putEnvIfAbsent(environment, Environment.JAVA_HOME.name()); - putEnvIfAbsent(environment, Environment.HADOOP_COMMON_HOME.name()); - putEnvIfAbsent(environment, Environment.HADOOP_HDFS_HOME.name()); - putEnvIfAbsent(environment, Environment.YARN_HOME.name()); - - } - private static void writeLaunchEnv(OutputStream out, Map environment, Map resources, - List command) + List command, List appDirs) throws IOException { ShellScriptBuilder sb = new ShellScriptBuilder(); + if (System.getenv("YARN_HOME") != null) { + // TODO: Get from whitelist. + sb.env("YARN_HOME", System.getenv("YARN_HOME")); + } + sb.env(ApplicationConstants.LOCAL_DIR_ENV, StringUtils.join(",", appDirs)); + if (!Shell.WINDOWS) { + sb.env("JVM_PID", "$$"); + } if (environment != null) { for (Map.Entry env : environment.entrySet()) { sb.env(env.getKey().toString(), env.getValue().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/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 e0795613b65..68b0686a254 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 @@ -31,6 +31,8 @@ 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.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState; @@ -54,26 +56,22 @@ public class ContainerLogsPage extends NMView { private final Configuration conf; private final LocalDirAllocator logsSelector; private final Context nmContext; + private final RecordFactory recordFactory; @Inject public ContainersLogsBlock(Configuration conf, Context context) { this.conf = conf; this.logsSelector = new LocalDirAllocator(YarnConfiguration.NM_LOG_DIRS); this.nmContext = context; + this.recordFactory = RecordFactoryProvider.getRecordFactory(conf); } @Override protected void render(Block html) { DIV div = html.div("#content"); - ContainerId containerId; - try { - containerId = ConverterUtils.toContainerId($(CONTAINER_ID)); - } catch (IOException e) { - div.h1("Invalid containerId " + $(CONTAINER_ID))._(); - return; - } - + ContainerId containerId = + ConverterUtils.toContainerId(this.recordFactory, $(CONTAINER_ID)); Container container = this.nmContext.getContainers().get(containerId); if (container == null) { 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 de76b84e277..27be38a0299 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 @@ -18,21 +18,20 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp; -import static org.apache.hadoop.yarn.util.StringHelper.ujoin; import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; +import static org.apache.hadoop.yarn.util.StringHelper.ujoin; -import java.io.IOException; - +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; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.webapp.SubView; -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 org.apache.hadoop.yarn.webapp.view.InfoBlock; @@ -54,30 +53,22 @@ public class ContainerPage extends NMView implements NMWebParams { public static class ContainerBlock extends HtmlBlock implements NMWebParams { + private final Configuration conf; private final Context nmContext; + private final RecordFactory recordFactory; @Inject - public ContainerBlock(Context nmContext) { + public ContainerBlock(Configuration conf, Context nmContext) { + this.conf = conf; this.nmContext = nmContext; + this.recordFactory = RecordFactoryProvider.getRecordFactory(this.conf); } @Override protected void render(Block html) { - ContainerId containerID; - try { - containerID = ConverterUtils.toContainerId($(CONTAINER_ID)); - } catch (IOException e) { - html.p()._("Invalid containerId " + $(CONTAINER_ID))._(); - return; - } - - DIV div = html.div("#content"); + ContainerId containerID = + ConverterUtils.toContainerId(this.recordFactory, $(CONTAINER_ID)); Container container = this.nmContext.getContainers().get(containerID); - if (container == null) { - div.h1("Unknown Container. Container might have completed, " - + "please go back to the previous page and retry.")._(); - return; - } ContainerStatus containerData = container.cloneAndGetContainerStatus(); int exitCode = containerData.getExitStatus(); String exiStatus = 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 a043a37f594..307e87eccd6 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 @@ -57,7 +57,7 @@ public class WebServer extends AbstractService { LOG.info("Instantiating NMWebApp at " + bindAddress); try { this.webApp = - WebApps.$for("node", Context.class, this.nmContext) + WebApps.$for("yarn", Context.class, this.nmContext) .at(bindAddress).with(getConfig()) .start(new NMWebApp(this.resourceView)); } catch (Exception e) { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties index 96108ab9656..897bca3f414 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties @@ -12,12 +12,12 @@ log4j.threshold=ALL # #Default values -yarn.app.mapreduce.container.log.dir=null -yarn.app.mapreduce.container.log.filesize=100 +hadoop.yarn.mr.containerLogDir=null +hadoop.yarn.mr.totalLogFileSize=100 log4j.appender.CLA=org.apache.hadoop.yarn.ContainerLogAppender -log4j.appender.CLA.containerLogDir=${yarn.app.mapreduce.container.log.dir} -log4j.appender.CLA.totalLogFileSize=${yarn.app.mapreduce.container.log.filesize} +log4j.appender.CLA.containerLogDir=${hadoop.yarn.mr.containerLogDir} +log4j.appender.CLA.totalLogFileSize=${hadoop.yarn.mr.totalLogFileSize} log4j.appender.CLA.layout=org.apache.log4j.PatternLayout log4j.appender.CLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n 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 44328dbe0aa..6ee220b674a 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 @@ -105,31 +105,7 @@ public class TestContainerManagerWithLCE extends TestContainerManager { LOG.info("Running testContainerLaunchAndStop"); super.testContainerLaunchAndStop(); } - - @Override - public void testContainerLaunchAndExitSuccess() throws IOException, - InterruptedException { - // Don't run the test if the binary is not available. - if (!shouldRunTest()) { - LOG.info("LCE binary path is not passed. Not running the test"); - return; - } - LOG.info("Running testContainerLaunchAndExitSuccess"); - super.testContainerLaunchAndExitSuccess(); - } - @Override - public void testContainerLaunchAndExitFailure() throws IOException, - InterruptedException { - // Don't run the test if the binary is not available. - if (!shouldRunTest()) { - LOG.info("LCE binary path is not passed. Not running the test"); - return; - } - LOG.info("Running testContainerLaunchAndExitFailure"); - super.testContainerLaunchAndExitFailure(); - } - @Override public void testLocalFilesCleanup() throws InterruptedException, IOException { 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 2de0428cb6b..87460d045ce 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 @@ -287,95 +287,7 @@ public class TestContainerManager extends BaseContainerManagerTest { exec.signalContainer(user, pid, Signal.NULL)); } - - private void testContainerLaunchAndExit(int exitCode) throws IOException, InterruptedException { - File scriptFile = new File(tmpDir, "scriptFile.sh"); - 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("\necho Hello World! > " + processStartFile); - fileWriter.write("\necho $$ >> " + processStartFile); - - // Have script throw an exit code at the end - if (exitCode != 0) { - fileWriter.write("\nexit "+exitCode); - } - - fileWriter.close(); - - ContainerLaunchContext containerLaunchContext = - recordFactory.newRecordInstance(ContainerLaunchContext.class); - - // ////// Construct the Container-id - ContainerId cId = createContainerId(); - containerLaunchContext.setContainerId(cId); - - containerLaunchContext.setUser(user); - - URL resource_alpha = - ConverterUtils.getYarnUrlFromPath(localFS - .makeQualified(new Path(scriptFile.getAbsolutePath()))); - 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"; - Map localResources = - new HashMap(); - localResources.put(destinationFile, rsrc_alpha); - containerLaunchContext.setLocalResources(localResources); - containerLaunchContext.setUser(containerLaunchContext.getUser()); - 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); - - StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class); - startRequest.setContainerLaunchContext(containerLaunchContext); - containerManager.startContainer(startRequest); - - BaseContainerManagerTest.waitForContainerState(containerManager, cId, - ContainerState.COMPLETE); - - GetContainerStatusRequest gcsRequest = - recordFactory.newRecordInstance(GetContainerStatusRequest.class); - gcsRequest.setContainerId(cId); - ContainerStatus containerStatus = - containerManager.getContainerStatus(gcsRequest).getStatus(); - - // Verify exit status matches exit state of script - Assert.assertEquals(exitCode, - containerStatus.getExitStatus()); - } - - @Test - public void testContainerLaunchAndExitSuccess() throws IOException, InterruptedException { - containerManager.start(); - int exitCode = 0; - - // launch context for a command that will return exit code 0 - // and verify exit code returned - testContainerLaunchAndExit(exitCode); - } - - @Test - public void testContainerLaunchAndExitFailure() throws IOException, InterruptedException { - containerManager.start(); - int exitCode = 50; - - // launch context for a command that will return exit code 0 - // and verify exit code returned - testContainerLaunchAndExit(exitCode); - } - @Test public void testLocalFilesCleanup() throws InterruptedException, IOException { 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 48c745457a7..04d400ad18d 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 @@ -38,6 +38,8 @@ import java.util.Map.Entry; 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; @@ -133,28 +135,6 @@ public class TestContainer { } } - @Test - @SuppressWarnings("unchecked") // mocked generic - public void testExternalKill() throws Exception { - WrappedContainer wc = null; - try { - wc = new WrappedContainer(13, 314159265358979L, 4344, "yak"); - wc.initContainer(); - wc.localizeResources(); - wc.launchContainer(); - reset(wc.localizerBus); - wc.containerKilledOnRequest(); - assertEquals(ContainerState.EXITED_WITH_FAILURE, - wc.c.getContainerState()); - verifyCleanupCall(wc); - } - finally { - if (wc != null) { - wc.finished(); - } - } - } - @Test @SuppressWarnings("unchecked") // mocked generic public void testCleanupOnFailure() throws Exception { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml index d94f5973144..b4c398f70f0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml @@ -37,20 +37,6 @@ - - - - maven-jar-plugin - - - - test-jar - - test-compile - - - - maven-antrun-plugin @@ -112,41 +98,4 @@ - - - - visualize - - false - - - - - org.codehaus.mojo - exec-maven-plugin - 1.2 - - - compile - - java - - - org.apache.hadoop.yarn.util.VisualizeStateMachine - - ResourceManager - org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl, - org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl, - org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl, - org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl - ResourceManager.gv - - - - - - - - - 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 6237f8961f0..1fc34f0dfd0 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 @@ -84,7 +84,7 @@ public class AdminService extends AbstractService implements RMAdminProtocol { super.init(conf); String bindAddress = conf.get(YarnConfiguration.RM_ADMIN_ADDRESS, - YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS); + YarnConfiguration.RM_ADMIN_ADDRESS); masterServiceAddress = NetUtils.createSocketAddr(bindAddress); adminAcl = new AccessControlList( 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 2cf19000985..a31bef8af9d 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 @@ -36,8 +36,8 @@ import org.apache.hadoop.security.SecurityInfo; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.yarn.api.ClientRMProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; @@ -46,8 +46,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; @@ -165,17 +165,11 @@ public class ClientRMService extends AbstractService implements } @Override - public GetNewApplicationResponse getNewApplication( - GetNewApplicationRequest request) throws YarnRemoteException { - GetNewApplicationResponse response = recordFactory - .newRecordInstance(GetNewApplicationResponse.class); + public GetNewApplicationIdResponse getNewApplicationId( + GetNewApplicationIdRequest request) throws YarnRemoteException { + GetNewApplicationIdResponse response = recordFactory + .newRecordInstance(GetNewApplicationIdResponse.class); response.setApplicationId(getNewApplicationId()); - // Pick up min/max resource from scheduler... - response.setMinimumResourceCapability(scheduler - .getMinimumResourceCapability()); - response.setMaximumResourceCapability(scheduler - .getMaximumResourceCapability()); - return response; } @@ -234,8 +228,8 @@ public class ClientRMService extends AbstractService implements @SuppressWarnings("unchecked") @Override - public KillApplicationResponse forceKillApplication( - KillApplicationRequest request) throws YarnRemoteException { + public FinishApplicationResponse finishApplication( + FinishApplicationRequest request) throws YarnRemoteException { ApplicationId applicationId = request.getApplicationId(); @@ -268,8 +262,8 @@ public class ClientRMService extends AbstractService implements RMAuditLogger.logSuccess(callerUGI.getShortUserName(), AuditConstants.KILL_APP_REQUEST, "ClientRMService" , applicationId); - KillApplicationResponse response = recordFactory - .newRecordInstance(KillApplicationResponse.class); + FinishApplicationResponse response = recordFactory + .newRecordInstance(FinishApplicationResponse.class); return response; } 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 3f175a34a0a..d0cd0a7ff86 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 @@ -30,7 +30,6 @@ 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.ipc.RPCUtil; import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier; import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; @@ -251,10 +250,13 @@ public class RMAppManager implements EventHandler { if (rmContext.getRMApps().putIfAbsent(applicationId, application) != null) { - String message = "Application with id " + applicationId - + " is already present! Cannot add a duplicate!"; - LOG.info(message); - throw RPCUtil.getRemoteException(message); + LOG.info("Application with id " + applicationId + + " is already present! Cannot add a duplicate!"); + // don't send event through dispatcher as it will be handled by app + // already present with this id. + application.handle(new RMAppRejectedEvent(applicationId, + "Application with this id is already present! " + + "Cannot add a duplicate!")); } else { this.rmContext.getDispatcher().getEventHandler().handle( new RMAppEvent(applicationId, RMAppEventType.START)); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java index 997906a62e4..85cd8825daa 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java @@ -22,6 +22,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore; @@ -30,6 +31,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; public class RMContextImpl implements RMContext { 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 8a56d504d69..179b56a4af4 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 @@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager; import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager; -import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store; @@ -98,7 +97,7 @@ public class ResourceManager extends CompositeService implements Recoverable { private ContainerAllocationExpirer containerAllocationExpirer; protected NMLivelinessMonitor nmLivelinessMonitor; protected NodesListManager nodesListManager; - private EventHandler schedulerDispatcher; + private SchedulerEventDispatcher schedulerDispatcher; protected RMAppManager rmAppManager; private WebApp webApp; @@ -119,7 +118,7 @@ public class ResourceManager extends CompositeService implements Recoverable { @Override public synchronized void init(Configuration conf) { - this.rmDispatcher = createDispatcher(); + this.rmDispatcher = new AsyncDispatcher(); addIfService(this.rmDispatcher); this.containerAllocationExpirer = new ContainerAllocationExpirer( @@ -138,8 +137,8 @@ public class ResourceManager extends CompositeService implements Recoverable { this.conf = new YarnConfiguration(conf); // Initialize the scheduler this.scheduler = createScheduler(); - this.schedulerDispatcher = createSchedulerEventDispatcher(); - addIfService(this.schedulerDispatcher); + this.schedulerDispatcher = new SchedulerEventDispatcher(this.scheduler); + addService(this.schedulerDispatcher); this.rmDispatcher.register(SchedulerEventType.class, this.schedulerDispatcher); @@ -187,22 +186,11 @@ public class ResourceManager extends CompositeService implements Recoverable { addService(adminService); this.applicationMasterLauncher = createAMLauncher(); - this.rmDispatcher.register(AMLauncherEventType.class, - this.applicationMasterLauncher); - addService(applicationMasterLauncher); super.init(conf); } - protected EventHandler createSchedulerEventDispatcher() { - return new SchedulerEventDispatcher(this.scheduler); - } - - protected Dispatcher createDispatcher() { - return new AsyncDispatcher(); - } - protected void addIfService(Object object) { if (object instanceof Service) { addService((Service) object); @@ -393,7 +381,7 @@ public class ResourceManager extends CompositeService implements Recoverable { } protected void startWepApp() { - webApp = WebApps.$for("cluster", masterService).at( + webApp = WebApps.$for("yarn", masterService).at( conf.get(YarnConfiguration.RM_WEBAPP_ADDRESS, YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS)). start(new RMWebApp(this)); 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 337f4816890..b394faa85d2 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 @@ -136,7 +136,7 @@ public class AMLauncher implements Runnable { containerMgrProxy.stopContainer(stopRequest); } - protected ContainerManager getContainerMgrProxy( + private ContainerManager getContainerMgrProxy( final ApplicationId applicationID) throws IOException { Container container = application.getMasterContainer(); @@ -173,11 +173,23 @@ public class AMLauncher implements Runnable { // Construct the actual Container ContainerLaunchContext container = applicationMasterContext.getAMContainerSpec(); - LOG.info("Command to launch container " - + containerID - + " : " - + StringUtils.arrayToString(container.getCommands().toArray( - new String[0]))); + StringBuilder mergedCommand = new StringBuilder(); + String failCount = Integer.toString(application.getAppAttemptId() + .getAttemptId()); + List commandList = new ArrayList(); + 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, + failCount); + mergedCommand.append(result).append(" "); + commandList.add(result); + } + container.setCommands(commandList); + /** add the failed count to the app master command line */ + + LOG.info("Command to launch container " + + containerID + " : " + mergedCommand); // Finalize the container container.setContainerId(containerID); @@ -191,11 +203,6 @@ public class AMLauncher implements Runnable { ContainerLaunchContext container) throws IOException { Map environment = container.getEnvironment(); - - // Set the AppAttemptId to be consumable by the AM. - environment.put(ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV, - application.getAppAttemptId().toString()); - if (UserGroupInformation.isSecurityEnabled()) { // TODO: Security enabled/disabled info should come from RM. 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 a25a4312b17..d1ef1d14004 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 @@ -42,16 +42,17 @@ public class ApplicationMasterLauncher extends AbstractService implements private final BlockingQueue masterEvents = new LinkedBlockingQueue(); - protected ApplicationTokenSecretManager applicationTokenSecretManager; + private ApplicationTokenSecretManager applicationTokenSecretManager; private ClientToAMSecretManager clientToAMSecretManager; - protected final RMContext context; + private final RMContext context; - public ApplicationMasterLauncher( - ApplicationTokenSecretManager applicationTokenSecretManager, - ClientToAMSecretManager clientToAMSecretManager, + public ApplicationMasterLauncher(ApplicationTokenSecretManager + applicationTokenSecretManager, ClientToAMSecretManager clientToAMSecretManager, RMContext context) { super(ApplicationMasterLauncher.class.getName()); this.context = context; + /* register to dispatcher */ + this.context.getDispatcher().register(AMLauncherEventType.class, this); this.launcherPool = new ThreadPoolExecutor(1, 10, 1, TimeUnit.HOURS, new LinkedBlockingQueue()); this.launcherHandlingThread = new LauncherThread(); 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/RMApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java index 6e63e2248d6..2e739a98b99 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java @@ -24,6 +24,7 @@ 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.event.EventHandler; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; @@ -32,7 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; * look at {@link RMAppImpl} for its implementation. This interface * exposes methods to access various updates in application status/report. */ -public interface RMApp extends EventHandler { +public interface RMApp extends EventHandler{ /** * The application id for this {@link RMApp}. 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 94d04a8d127..9246d1838c7 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 @@ -39,7 +39,6 @@ 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.event.EventHandler; import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType; @@ -87,8 +86,7 @@ public class RMAppImpl implements RMApp { private long startTime; private long finishTime; private RMAppAttempt currentAttempt; - @SuppressWarnings("rawtypes") - private EventHandler handler; + private static final FinalTransition FINAL_TRANSITION = new FinalTransition(); private static final StateMachineFactory(RMAppState.NEW) + // TODO - ATTEMPT_KILLED not sent right now but should handle if + // attempt starts sending + // Transitions from NEW state .addTransition(RMAppState.NEW, RMAppState.SUBMITTED, RMAppEventType.START, new StartAppAttemptTransition()) @@ -115,7 +116,7 @@ public class RMAppImpl implements RMApp { .addTransition(RMAppState.SUBMITTED, RMAppState.ACCEPTED, RMAppEventType.APP_ACCEPTED) .addTransition(RMAppState.SUBMITTED, RMAppState.KILLED, - RMAppEventType.KILL, new KillAppAndAttemptTransition()) + RMAppEventType.KILL, new AppKilledTransition()) // Transitions from ACCEPTED state .addTransition(RMAppState.ACCEPTED, RMAppState.RUNNING, @@ -125,7 +126,7 @@ public class RMAppImpl implements RMApp { RMAppEventType.ATTEMPT_FAILED, new AttemptFailedTransition(RMAppState.SUBMITTED)) .addTransition(RMAppState.ACCEPTED, RMAppState.KILLED, - RMAppEventType.KILL, new KillAppAndAttemptTransition()) + RMAppEventType.KILL, new AppKilledTransition()) // Transitions from RUNNING state .addTransition(RMAppState.RUNNING, RMAppState.FINISHED, @@ -135,7 +136,7 @@ public class RMAppImpl implements RMApp { RMAppEventType.ATTEMPT_FAILED, new AttemptFailedTransition(RMAppState.SUBMITTED)) .addTransition(RMAppState.RUNNING, RMAppState.KILLED, - RMAppEventType.KILL, new KillAppAndAttemptTransition()) + RMAppEventType.KILL, new AppKilledTransition()) // Transitions from FINISHED state .addTransition(RMAppState.FINISHED, RMAppState.FINISHED, @@ -167,7 +168,6 @@ public class RMAppImpl implements RMApp { this.name = name; this.rmContext = rmContext; this.dispatcher = rmContext.getDispatcher(); - this.handler = dispatcher.getEventHandler(); this.conf = config; this.user = user; this.queue = queue; @@ -310,8 +310,7 @@ public class RMAppImpl implements RMApp { return BuilderUtils.newApplicationReport(this.applicationId, this.user, this.queue, this.name, host, rpcPort, clientToken, createApplicationState(this.stateMachine.getCurrentState()), - this.diagnostics.toString(), trackingUrl, - this.startTime, this.finishTime); + this.diagnostics.toString(), trackingUrl, this.startTime); } finally { this.readLock.unlock(); } @@ -403,7 +402,7 @@ public class RMAppImpl implements RMApp { submissionContext); attempts.put(appAttemptId, attempt); currentAttempt = attempt; - handler.handle( + dispatcher.getEventHandler().handle( new RMAppAttemptEvent(appAttemptId, RMAppAttemptEventType.START)); } @@ -420,23 +419,13 @@ public class RMAppImpl implements RMApp { }; } - private static class AppKilledTransition extends FinalTransition { - @Override + private static final class AppKilledTransition extends FinalTransition { public void transition(RMAppImpl app, RMAppEvent event) { app.diagnostics.append("Application killed by user."); super.transition(app, event); }; } - private static class KillAppAndAttemptTransition extends AppKilledTransition { - @SuppressWarnings("unchecked") - @Override - public void transition(RMAppImpl app, RMAppEvent event) { - app.handler.handle(new RMAppAttemptEvent(app.currentAttempt.getAppAttemptId(), - RMAppAttemptEventType.KILL)); - super.transition(app, event); - } - } private static final class AppRejectedTransition extends FinalTransition{ public void transition(RMAppImpl app, RMAppEvent event) { @@ -460,11 +449,11 @@ public class RMAppImpl implements RMApp { public void transition(RMAppImpl app, RMAppEvent event) { Set nodes = getNodesOnWhichAttemptRan(app); for (NodeId nodeId : nodes) { - app.handler.handle( + app.dispatcher.getEventHandler().handle( new RMNodeCleanAppEvent(nodeId, app.applicationId)); } app.finishTime = System.currentTimeMillis(); - app.handler.handle( + app.dispatcher.getEventHandler().handle( new RMAppManagerEvent(app.applicationId, RMAppManagerEventType.APP_COMPLETED)); }; @@ -481,13 +470,11 @@ public class RMAppImpl implements RMApp { @Override public RMAppState transition(RMAppImpl app, RMAppEvent event) { - - RMAppFailedAttemptEvent failedEvent = ((RMAppFailedAttemptEvent)event); + if (app.attempts.size() == app.maxRetries) { String msg = "Application " + app.getApplicationId() + " failed " + app.maxRetries - + " times due to " + failedEvent.getDiagnostics() - + ". Failing the application."; + + " times. Failing the application."; LOG.info(msg); app.diagnostics.append(msg); // Inform the node for app-finish 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 aeb3d2af045..70747deacba 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 @@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; * {@link YarnConfiguration#RM_AM_MAX_RETRIES}. For specific * implementation take a look at {@link RMAppAttemptImpl}. */ -public interface RMAppAttempt extends EventHandler { +public interface RMAppAttempt extends EventHandler{ /** * Get the application attempt id for this {@link RMAppAttempt}. @@ -79,7 +79,7 @@ public interface RMAppAttempt extends EventHandler { * Diagnostics information for the application attempt. * @return diagnostics information for the application attempt. */ - String getDiagnostics(); + StringBuilder getDiagnostics(); /** * Progress for the application attempt. 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 7f8ff82d6a1..7c6357defab 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 @@ -31,7 +31,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -48,7 +47,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; 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.RMAppFailedAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent; @@ -106,10 +104,10 @@ public class RMAppAttemptImpl implements RMAppAttempt { private Container masterContainer; private float progress = 0; - private String host = "N/A"; + private String host; private int rpcPort; - private String trackingUrl = "N/A"; - private String finalState = "N/A"; + private String trackingUrl; + private String finalState; private final StringBuilder diagnostics = new StringBuilder(); private static final StateMachineFactory { this.httpPort = httpPort; this.totalCapability = capability; this.nodeAddress = hostName + ":" + cmPort; - this.httpAddress = hostName + ":" + httpPort; + this.httpAddress = hostName + ":" + httpPort;; this.node = node; this.nodeHealthStatus.setIsNodeHealthy(true); - this.nodeHealthStatus.setHealthReport("Healthy"); this.nodeHealthStatus.setLastHealthReportTime(System.currentTimeMillis()); this.latestHeartBeatResponse.setResponseId(0); @@ -223,18 +222,6 @@ public class RMNodeImpl implements RMNode, EventHandler { } } - private void setNodeHealthStatus(NodeHealthStatus status) - { - this.writeLock.lock(); - try { - this.nodeHealthStatus.setHealthReport(status.getHealthReport()); - this.nodeHealthStatus.setIsNodeHealthy(status.getIsNodeHealthy()); - this.nodeHealthStatus.setLastHealthReportTime(status.getLastHealthReportTime()); - } finally { - this.writeLock.unlock(); - } - } - @Override public RMNodeState getState() { this.readLock.lock(); @@ -358,10 +345,7 @@ public class RMNodeImpl implements RMNode, EventHandler { // Switch the last heartbeatresponse. rmNode.latestHeartBeatResponse = statusEvent.getLatestResponse(); - NodeHealthStatus remoteNodeHealthStatus = - statusEvent.getNodeHealthStatus(); - rmNode.setNodeHealthStatus(remoteNodeHealthStatus); - if (!remoteNodeHealthStatus.getIsNodeHealthy()) { + if (!statusEvent.getNodeHealthStatus().getIsNodeHealthy()) { // Inform the scheduler rmNode.context.getDispatcher().getEventHandler().handle( new NodeRemovedSchedulerEvent(rmNode)); @@ -408,9 +392,8 @@ public class RMNodeImpl implements RMNode, EventHandler { // Switch the last heartbeatresponse. rmNode.latestHeartBeatResponse = statusEvent.getLatestResponse(); - NodeHealthStatus remoteNodeHealthStatus = statusEvent.getNodeHealthStatus(); - rmNode.setNodeHealthStatus(remoteNodeHealthStatus); - if (remoteNodeHealthStatus.getIsNodeHealthy()) { + + if (statusEvent.getNodeHealthStatus().getIsNodeHealthy()) { rmNode.context.getDispatcher().getEventHandler().handle( new NodeAddedSchedulerEvent(rmNode)); 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/scheduler/QueueMetrics.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java index 6928cdb19d0..61c829507e2 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java @@ -32,8 +32,10 @@ import static org.apache.hadoop.metrics2.lib.Interns.info; import org.apache.hadoop.metrics2.lib.MetricsRegistry; import org.apache.hadoop.metrics2.lib.MutableCounterInt; import org.apache.hadoop.metrics2.lib.MutableGaugeInt; +import org.apache.hadoop.yarn.api.records.ApplicationState; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.util.Self; import static org.apache.hadoop.yarn.server.resourcemanager.resource.Resources.*; import org.slf4j.LoggerFactory; @@ -280,56 +282,4 @@ public class QueueMetrics { parent.unreserveResource(user, res); } } - - public int getAppsSubmitted() { - return appsSubmitted.value(); - } - - public int getAppsRunning() { - return appsRunning.value(); - } - - public int getAppsPending() { - return appsPending.value(); - } - - public int getAppsCompleted() { - return appsCompleted.value(); - } - - public int getAppsKilled() { - return appsKilled.value(); - } - - public int getAppsFailed() { - return appsFailed.value(); - } - - public int getAllocatedGB() { - return allocatedGB.value(); - } - - public int getAllocatedContainers() { - return allocatedContainers.value(); - } - - public int getAvailableGB() { - return availableGB.value(); - } - - public int getPendingGB() { - return pendingGB.value(); - } - - public int getPendingContainers() { - return pendingContainers.value(); - } - - public int getReservedGB() { - return reservedGB.value(); - } - - public int getReservedContainers() { - return reservedContainers.value(); - } } 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 10913e09999..b4037aaeaf7 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 @@ -207,18 +207,13 @@ public class SchedulerApp { .getDispatcher().getEventHandler(), this.rmContext .getContainerAllocationExpirer()); - // Add it to allContainers list. - newlyAllocatedContainers.add(rmContainer); - liveContainers.put(container.getId(), rmContainer); - // Update consumption and track allocations - appSchedulingInfo.allocate(type, node, priority, request, container); - Resources.addTo(currentConsumption, container.getResource()); - + // Inform the container rmContainer.handle( new RMContainerEvent(container.getId(), RMContainerEventType.START)); + Resources.addTo(currentConsumption, container.getResource()); if (LOG.isDebugEnabled()) { LOG.debug("allocate: applicationAttemptId=" + container.getId().getApplicationAttemptId() @@ -228,6 +223,12 @@ public class SchedulerApp { RMAuditLogger.logSuccess(getUser(), AuditConstants.ALLOC_CONTAINER, "SchedulerApp", getApplicationId(), container.getId()); + + // Add it to allContainers list. + newlyAllocatedContainers.add(rmContainer); + liveContainers.put(container.getId(), rmContainer); + + appSchedulingInfo.allocate(type, node, priority, request, container); return rmContainer; } 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 e67d371ee61..9a3b1c4da35 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 @@ -1046,20 +1046,19 @@ public class LeafQueue implements CSQueue { } private Container getContainer(RMContainer rmContainer, - SchedulerApp application, SchedulerNode node, - Resource capability, Priority priority) { + SchedulerApp application, SchedulerNode node, Resource capability) { return (rmContainer != null) ? rmContainer.getContainer() : - createContainer(application, node, capability, priority); + createContainer(application, node, capability); } public Container createContainer(SchedulerApp application, SchedulerNode node, - Resource capability, Priority priority) { + Resource capability) { Container container = BuilderUtils.newContainer(this.recordFactory, application.getApplicationAttemptId(), application.getNewContainerId(), - node.getNodeID(), node.getHttpAddress(), - capability, priority); + node.getNodeID(), + node.getHttpAddress(), capability); // If security is enabled, send the container-tokens too. if (UserGroupInformation.isSecurityEnabled()) { @@ -1100,7 +1099,7 @@ public class LeafQueue implements CSQueue { // Create the container if necessary Container container = - getContainer(rmContainer, application, node, capability, priority); + getContainer(rmContainer, application, node, capability); // Can we allocate a container on this node? int availableContainers = @@ -1153,17 +1152,14 @@ public class LeafQueue implements CSQueue { private void reserve(SchedulerApp application, Priority priority, SchedulerNode node, RMContainer rmContainer, Container container) { + rmContainer = application.reserve(node, priority, rmContainer, container); + node.reserveResource(application, priority, rmContainer); + // Update reserved metrics if this is the first reservation if (rmContainer == null) { getMetrics().reserveResource( application.getUser(), container.getResource()); } - - // Inform the application - rmContainer = application.reserve(node, priority, rmContainer, container); - - // Update the node - node.reserveResource(application, priority, rmContainer); } private void unreserve(SchedulerApp application, 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/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 ff51d62d910..9f3bc1cce7a 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 @@ -19,7 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event; 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.server.resourcemanager.rmnode.RMNode; 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 7a90c5b6fac..752b81ce5de 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 @@ -291,7 +291,7 @@ public class FifoScheduler implements ResourceScheduler { @SuppressWarnings("unchecked") private synchronized void addApplication(ApplicationAttemptId appAttemptId, - String user) { + String queueName, String user) { // TODO: Fix store SchedulerApp schedulerApp = new SchedulerApp(appAttemptId, user, DEFAULT_QUEUE, @@ -528,8 +528,7 @@ public class FifoScheduler implements ResourceScheduler { application.getApplicationAttemptId(), application.getNewContainerId(), node.getRMNode().getNodeID(), - node.getRMNode().getHttpAddress(), - capability, priority); + node.getRMNode().getHttpAddress(), capability); // If security is enabled, send the container-tokens too. if (UserGroupInformation.isSecurityEnabled()) { @@ -628,7 +627,7 @@ public class FifoScheduler implements ResourceScheduler { { AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event; addApplication(appAddedEvent.getApplicationAttemptId(), appAddedEvent - .getUser()); + .getQueue(), appAddedEvent.getUser()); } break; case APP_REMOVED: 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 1d074e3160b..a621cc10472 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 @@ -76,7 +76,7 @@ class NodesPage extends RmView { // TODO: FIXME Vinodkv // td(String.valueOf(ni.getUsedResource().getMemory())). // td(String.valueOf(ni.getAvailableResource().getMemory())). - td("n/a")._(); + _(); } tbody._()._(); } @@ -100,7 +100,7 @@ class NodesPage extends RmView { // rack, nodeid, host, healthStatus, health update ts, health report, // containers, memused, memavail append(", aoColumns:[null, null, null, null, null, null, "). - append("{sType:'title-numeric', bSearchable:false}]}"). + append("{bSearchable:false},{bSearchable:false},{bSearchable:false}]}"). toString(); } } 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/RmController.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java index 698bc3c933a..234f93e2f0c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java @@ -102,7 +102,7 @@ public class RmController extends Controller { .getMasterContainer(); if (masterContainer != null) { String url = join("http://", masterContainer.getNodeHttpAddress(), - "/node", "/containerlogs/", + "/yarn", "/containerlogs/", ConverterUtils.toString(masterContainer.getId())); info._("AM container logs:", url, url); } else { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java index 727cd1a2323..9a9ae2f51cd 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java @@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; -import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.Records; public class MockAM { @@ -129,7 +128,7 @@ public class MockAM { req.setHostName(resource); req.setNumContainers(containers); Priority pri = Records.newRecord(Priority.class); - pri.setPriority(priority); + pri.setPriority(1); req.setPriority(pri); Resource capability = Records.newRecord(Resource.class); capability.setMemory(memory); @@ -140,8 +139,11 @@ public class MockAM { public AMResponse allocate( List resourceRequest, List releases) throws Exception { - AllocateRequest req = BuilderUtils.newAllocateRequest(attemptId, - ++responseId, 0F, resourceRequest, releases); + AllocateRequest req = Records.newRecord(AllocateRequest.class); + req.setResponseId(++responseId); + req.setApplicationAttemptId(attemptId); + req.addAllAsks(resourceRequest); + req.addAllReleases(releases); AllocateResponse resp = amRMProtocol.allocate(req); return resp.getAMResponse(); } 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 d2a9a11182f..4be27399672 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 @@ -22,9 +22,9 @@ import junit.framework.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.ClientRMProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdResponse; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -74,17 +74,11 @@ public class MockRM extends ResourceManager { Assert.assertEquals("App state is not correct (timedout)", finalState, app.getState()); } - - // get new application id - public GetNewApplicationResponse getNewAppId() throws Exception { - ClientRMProtocol client = getClientRMService(); - return client.getNewApplication(Records.newRecord(GetNewApplicationRequest.class)); - } //client public RMApp submitApp(int masterMemory) throws Exception { ClientRMProtocol client = getClientRMService(); - GetNewApplicationResponse resp = client.getNewApplication(Records.newRecord(GetNewApplicationRequest.class)); + GetNewApplicationIdResponse resp = client.getNewApplicationId(Records.newRecord(GetNewApplicationIdRequest.class)); ApplicationId appId = resp.getApplicationId(); SubmitApplicationRequest req = Records.newRecord(SubmitApplicationRequest.class); @@ -95,7 +89,7 @@ public class MockRM extends ResourceManager { sub.setUser(""); ContainerLaunchContext clc = Records.newRecord(ContainerLaunchContext.class); - Resource capability = Records.newRecord(Resource.class); + Resource capability = Records.newRecord(Resource.class); capability.setMemory(masterMemory); clc.setResource(capability); sub.setAMContainerSpec(clc); @@ -115,9 +109,9 @@ public class MockRM extends ResourceManager { public void killApp(ApplicationId appId) throws Exception { ClientRMProtocol client = getClientRMService(); - KillApplicationRequest req = Records.newRecord(KillApplicationRequest.class); + FinishApplicationRequest req = Records.newRecord(FinishApplicationRequest.class); req.setApplicationId(appId); - client.forceKillApplication(req); + client.finishApplication(req); } //from AMLauncher @@ -201,7 +195,6 @@ public class MockRM extends ResourceManager { }; } - @Override protected AdminService createAdminService() { return new AdminService(getConfig(), scheduler, getRMContext(), this.nodesListManager){ 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 72ade5c1da8..a7b5d02c914 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 @@ -45,7 +45,6 @@ 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.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.factories.RecordFactory; @@ -185,9 +184,7 @@ public class NodeManager implements ContainerManager { Container container = BuilderUtils.newContainer(containerLaunchContext.getContainerId(), this.nodeId, nodeHttpAddress, - containerLaunchContext.getResource(), - null // DKDC - Doesn't matter - ); + containerLaunchContext.getResource()); applicationContainers.add(container); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java deleted file mode 100644 index a12049f9e82..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java +++ /dev/null @@ -1,159 +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 java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ContainerManager; -import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; -import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; -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.exceptions.YarnRemoteException; -import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher; -import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; -import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; -import org.apache.log4j.Level; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.junit.Assert; -import org.junit.Test; - -public class TestApplicationMasterLauncher { - - private static final Log LOG = LogFactory - .getLog(TestApplicationMasterLauncher.class); - - private static final class MyContainerManagerImpl implements - ContainerManager { - - boolean launched = false; - boolean cleanedup = false; - String attemptIdAtContainerManager = null; - - @Override - public StartContainerResponse - startContainer(StartContainerRequest request) - throws YarnRemoteException { - LOG.info("Container started by MyContainerManager: " + request); - launched = true; - attemptIdAtContainerManager = request.getContainerLaunchContext() - .getEnvironment().get( - ApplicationConstants.APPLICATION_ATTEMPT_ID_ENV); - return null; - } - - @Override - public StopContainerResponse stopContainer(StopContainerRequest request) - throws YarnRemoteException { - LOG.info("Container cleaned up by MyContainerManager"); - cleanedup = true; - return null; - } - - @Override - public GetContainerStatusResponse getContainerStatus( - GetContainerStatusRequest request) throws YarnRemoteException { - return null; - } - - } - - private static final class MockRMWithCustomAMLauncher extends MockRM { - - private final ContainerManager containerManager; - - public MockRMWithCustomAMLauncher(ContainerManager containerManager) { - this.containerManager = containerManager; - } - - @Override - protected ApplicationMasterLauncher createAMLauncher() { - return new ApplicationMasterLauncher(super.appTokenSecretManager, - super.clientToAMSecretManager, getRMContext()) { - @Override - protected Runnable createRunnableLauncher(RMAppAttempt application, - AMLauncherEventType event) { - return new AMLauncher(context, application, event, - applicationTokenSecretManager, clientToAMSecretManager, - getConfig()) { - @Override - protected ContainerManager getContainerMgrProxy( - ApplicationId applicationID) throws IOException { - return containerManager; - } - }; - } - }; - } - } - - @Test - public void testAMLaunchAndCleanup() throws Exception { - Logger rootLogger = LogManager.getRootLogger(); - rootLogger.setLevel(Level.DEBUG); - MyContainerManagerImpl containerManager = new MyContainerManagerImpl(); - MockRMWithCustomAMLauncher rm = new MockRMWithCustomAMLauncher( - containerManager); - rm.start(); - MockNM nm1 = rm.registerNode("h1:1234", 5120); - - RMApp app = rm.submitApp(2000); - - // kick the scheduling - nm1.nodeHeartbeat(true); - - int waitCount = 0; - while (containerManager.launched == false && waitCount++ < 20) { - LOG.info("Waiting for AM Launch to happen.."); - Thread.sleep(1000); - } - Assert.assertTrue(containerManager.launched); - - RMAppAttempt attempt = app.getCurrentAppAttempt(); - ApplicationAttemptId appAttemptId = attempt.getAppAttemptId(); - Assert.assertEquals(appAttemptId.toString(), - containerManager.attemptIdAtContainerManager); - - MockAM am = new MockAM(rm.getRMContext(), rm - .getApplicationMasterService(), appAttemptId); - am.registerAppAttempt(); - am.unregisterAppAttempt(); - - waitCount = 0; - while (containerManager.cleanedup == false && waitCount++ < 20) { - LOG.info("Waiting for AM Cleanup to happen.."); - Thread.sleep(1000); - } - Assert.assertTrue(containerManager.cleanedup); - - am.waitForState(RMAppAttemptState.FINISHED); - rm.stop(); - } -} 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 3bba11e1fb1..03941e3625d 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 @@ -25,7 +25,6 @@ import junit.framework.Assert; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -41,20 +40,6 @@ public class TestRM { private static final Log LOG = LogFactory.getLog(TestRM.class); - @Test - public void testGetNewAppId() throws Exception { - Logger rootLogger = LogManager.getRootLogger(); - rootLogger.setLevel(Level.DEBUG); - MockRM rm = new MockRM(); - rm.start(); - - GetNewApplicationResponse resp = rm.getNewAppId(); - assert (resp.getApplicationId().getId() != 0); - assert (resp.getMinimumResourceCapability().getMemory() > 0); - assert (resp.getMaximumResourceCapability().getMemory() > 0); - rm.stop(); - } - @Test public void testAppWithNoContainers() throws Exception { Logger rootLogger = LogManager.getRootLogger(); @@ -134,7 +119,6 @@ public class TestRM { public static void main(String[] args) throws Exception { TestRM t = new TestRM(); - t.testGetNewAppId(); t.testAppWithNoContainers(); t.testAppOnMultiNode(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java index 536aa672d79..60a227bc6d4 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java @@ -18,16 +18,12 @@ package org.apache.hadoop.yarn.server.resourcemanager; -import static org.junit.Assert.assertNotNull; - import java.io.IOException; -import java.util.Collection; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetworkTopology; -import org.apache.hadoop.yarn.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store; @@ -157,23 +153,6 @@ public class TestResourceManager { LOG.info("--- END: testResourceAllocation ---"); } - - @Test - public void testNodeHealthReportIsNotNull() throws Exception{ - String host1 = "host1"; - final int memory = 4 * 1024; - org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm1 = - registerNode(host1, 1234, 2345, NetworkTopology.DEFAULT_RACK, memory); - nm1.heartbeat(); - nm1.heartbeat(); - Collection values = resourceManager.getRMContext().getRMNodes().values(); - for (RMNode ni : values) - { - NodeHealthStatus nodeHealthStatus = ni.getNodeHealthStatus(); - String healthReport = nodeHealthStatus.getHealthReport(); - assertNotNull(healthReport); - } - } private void checkResourceUsage( org.apache.hadoop.yarn.server.resourcemanager.NodeManager... nodes ) { 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/TestAMRMRPCResponseId.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java index 3bc55473423..61d678ea01c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java @@ -32,7 +32,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; -import org.apache.hadoop.yarn.util.BuilderUtils; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -78,14 +77,13 @@ public class TestAMRMRPCResponseId { am.registerAppAttempt(); - AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(attempt - .getAppAttemptId(), 0, 0F, null, null); + AllocateRequest allocateRequest = recordFactory.newRecordInstance(AllocateRequest.class); + allocateRequest.setApplicationAttemptId(attempt.getAppAttemptId()); AMResponse response = amService.allocate(allocateRequest).getAMResponse(); Assert.assertEquals(1, response.getResponseId()); Assert.assertFalse(response.getReboot()); - allocateRequest = BuilderUtils.newAllocateRequest(attempt - .getAppAttemptId(), response.getResponseId(), 0F, null, null); + allocateRequest.setResponseId(response.getResponseId()); response = amService.allocate(allocateRequest).getAMResponse(); Assert.assertEquals(2, response.getResponseId()); @@ -93,9 +91,8 @@ public class TestAMRMRPCResponseId { response = amService.allocate(allocateRequest).getAMResponse(); Assert.assertEquals(2, response.getResponseId()); - /** try sending old request again **/ - allocateRequest = BuilderUtils.newAllocateRequest(attempt - .getAppAttemptId(), 0, 0F, null, null); + /** try sending old **/ + allocateRequest.setResponseId(0); response = amService.allocate(allocateRequest).getAMResponse(); Assert.assertTrue(response.getReboot()); } 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/TestApplicationMasterLauncher.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterLauncher.java new file mode 100644 index 00000000000..8cc948400e1 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterLauncher.java @@ -0,0 +1,193 @@ +/** +* 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.applicationsmanager; + +import java.util.concurrent.atomic.AtomicInteger; + +import junit.framework.Assert; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationState; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager; +import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; +import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent; +import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; +import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemStore; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory; +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.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl; +import org.apache.hadoop.yarn.util.Records; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Testing the applications manager launcher. + * + */ +public class TestApplicationMasterLauncher { +// private static final Log LOG = LogFactory.getLog(TestApplicationMasterLauncher.class); +// private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); +// private ApplicationMasterLauncher amLauncher; +// private DummyEventHandler asmHandle; +// private final ApplicationTokenSecretManager applicationTokenSecretManager = +// new ApplicationTokenSecretManager(); +// private final ClientToAMSecretManager clientToAMSecretManager = +// new ClientToAMSecretManager(); +// +// Object doneLaunching = new Object(); +// AtomicInteger launched = new AtomicInteger(); +// AtomicInteger cleanedUp = new AtomicInteger(); +// private RMContext context = new RMContextImpl(new MemStore(), null, null, +// null); +// +// private Configuration conf = new Configuration(); +// +// private class DummyEventHandler implements EventHandler { +// @Override +// public void handle(ApplicationEvent appEvent) { +// ApplicationEventType event = appEvent.getType(); +// switch (event) { +// case FINISH: +// synchronized(doneLaunching) { +// doneLaunching.notify(); +// } +// break; +// +// default: +// break; +// } +// } +// } +// +// private class DummyLaunch implements Runnable { +// public void run() { +// launched.incrementAndGet(); +// } +// } +// +// private class DummyCleanUp implements Runnable { +// private EventHandler eventHandler; +// +// public DummyCleanUp(EventHandler eventHandler) { +// this.eventHandler = eventHandler; +// } +// public void run() { +// cleanedUp.incrementAndGet(); +// eventHandler.handle(new AMFinishEvent(null, +// ApplicationState.COMPLETED, "", "")); +// } +// } +// +// private class DummyApplicationMasterLauncher extends +// ApplicationMasterLauncher { +// private EventHandler eventHandler; +// +// public DummyApplicationMasterLauncher( +// ApplicationTokenSecretManager applicationTokenSecretManager, +// ClientToAMSecretManager clientToAMSecretManager, +// EventHandler eventHandler) { +// super(applicationTokenSecretManager, clientToAMSecretManager, context); +// this.eventHandler = eventHandler; +// } +// +// @Override +// protected Runnable createRunnableLauncher(RMAppAttempt application, +// AMLauncherEventType event) { +// Runnable r = null; +// switch (event) { +// case LAUNCH: +// r = new DummyLaunch(); +// break; +// case CLEANUP: +// r = new DummyCleanUp(eventHandler); +// default: +// break; +// } +// return r; +// } +// } +// +// @Before +// public void setUp() { +// asmHandle = new DummyEventHandler(); +// amLauncher = new DummyApplicationMasterLauncher(applicationTokenSecretManager, +// clientToAMSecretManager, asmHandle); +// context.getDispatcher().init(conf); +// amLauncher.init(conf); +// context.getDispatcher().start(); +// amLauncher.start(); +// +// } +// +// @After +// public void tearDown() { +// amLauncher.stop(); +// } +// +// @Test +// public void testAMLauncher() throws Exception { +// +// // Creat AppId +// ApplicationId appId = recordFactory +// .newRecordInstance(ApplicationId.class); +// appId.setClusterTimestamp(System.currentTimeMillis()); +// appId.setId(1); +// +// ApplicationAttemptId appAttemptId = Records +// .newRecord(ApplicationAttemptId.class); +// appAttemptId.setApplicationId(appId); +// appAttemptId.setAttemptId(1); +// +// // Create submissionContext +// ApplicationSubmissionContext submissionContext = recordFactory +// .newRecordInstance(ApplicationSubmissionContext.class); +// submissionContext.setApplicationId(appId); +// submissionContext.setUser("dummyuser"); +// +// RMAppAttempt appAttempt = new RMAppAttemptImpl(appAttemptId, +// "dummyclienttoken", context, null, submissionContext); +// +// // Tell AMLauncher to launch the appAttempt +// amLauncher.handle(new AMLauncherEvent(AMLauncherEventType.LAUNCH, +// appAttempt)); +// +// // Tell AMLauncher to cleanup the appAttempt +// amLauncher.handle(new AMLauncherEvent(AMLauncherEventType.CLEANUP, +// appAttempt)); +// +// synchronized (doneLaunching) { +// doneLaunching.wait(10000); +// } +// Assert.assertEquals(1, launched.get()); +// Assert.assertEquals(1, cleanedUp.get()); +// } +} \ No newline at end of file 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/InlineDispatcher.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/InlineDispatcher.java index d771a61d864..51eb8cf2ec7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/InlineDispatcher.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/InlineDispatcher.java @@ -1,57 +1,50 @@ /** -* 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.yarn.server.resourcemanager.resourcetracker; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; -public class InlineDispatcher extends AsyncDispatcher { - private static final Log LOG = LogFactory.getLog(InlineDispatcher.class); - - private class TestEventHandler implements EventHandler { +class InlineDispatcher extends AsyncDispatcher { + private class InlineEventHandler implements EventHandler { + private final InlineDispatcher dispatcher; + public InlineEventHandler(InlineDispatcher dispatcher) { + this.dispatcher = dispatcher; + } @Override public void handle(Event event) { - dispatch(event); + this.dispatcher.dispatch(event); } } - @Override - protected void dispatch(Event event) { - LOG.info("Dispatching the event " + event.getClass().getName() + "." - + event.toString()); - - Class type = event.getType().getDeclaringClass(); - if (eventDispatchers.get(type) != null) { - eventDispatchers.get(type).handle(event); - } + public void dispatch(Event event) { + super.dispatch(event); } @Override public EventHandler getEventHandler() { - return new TestEventHandler(); + return new InlineEventHandler(this); } - + static class EmptyEventHandler implements EventHandler { @Override public void handle(Event event) { - //do nothing - } + ; // ignore + } } } \ No newline at end of file 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/MockRMApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java index 03229c34b48..4fb6486c2c7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java @@ -162,7 +162,6 @@ public class MockRMApp implements RMApp { this.diagnostics = new StringBuilder(diag); } - @Override public void handle(RMAppEvent event) { } 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 24408821e2d..56b3f4b18af 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 @@ -1,27 +1,26 @@ /** - * 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.yarn.server.resourcemanager.rmapp; import static org.mockito.Mockito.mock; import java.io.IOException; -import java.util.List; import junit.framework.Assert; @@ -33,62 +32,46 @@ 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.AsyncDispatcher; +import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; -import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemStore; -import org.apache.hadoop.yarn.server.resourcemanager.resourcetracker.InlineDispatcher; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore; 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.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; - import org.junit.Before; -import org.junit.After; import org.junit.Test; public class TestRMAppTransitions { - static final Log LOG = LogFactory.getLog(TestRMAppTransitions.class); - + private static final Log LOG = LogFactory.getLog(TestRMAppTransitions.class); + private RMContext rmContext; private static int maxRetries = 4; private static int appId = 1; - private AsyncDispatcher rmDispatcher; // ignore all the RM application attempt events private static final class TestApplicationAttemptEventDispatcher implements - EventHandler { + EventHandler { - private final RMContext rmContext; - public TestApplicationAttemptEventDispatcher(RMContext rmContext) { - this.rmContext = rmContext; + public TestApplicationAttemptEventDispatcher() { } @Override public void handle(RMAppAttemptEvent event) { - ApplicationId appId = event.getApplicationAttemptId().getApplicationId(); - RMApp rmApp = this.rmContext.getRMApps().get(appId); - if (rmApp != null) { - try { - rmApp.getRMAppAttempt(event.getApplicationAttemptId()).handle(event); - } catch (Throwable t) { - LOG.error("Error in handling event type " + event.getType() - + " for application " + appId, t); - } - } } } // handle all the RM application events - same as in ResourceManager.java private static final class TestApplicationEventDispatcher implements - EventHandler { + EventHandler { private final RMContext rmContext; public TestApplicationEventDispatcher(RMContext rmContext) { @@ -112,23 +95,19 @@ public class TestRMAppTransitions { @Before public void setUp() throws Exception { - AsyncDispatcher rmDispatcher = new AsyncDispatcher(); Configuration conf = new Configuration(); - rmDispatcher = new InlineDispatcher(); + Dispatcher rmDispatcher = new AsyncDispatcher(); - ContainerAllocationExpirer containerAllocationExpirer = - mock(ContainerAllocationExpirer.class); + ContainerAllocationExpirer containerAllocationExpirer = mock(ContainerAllocationExpirer.class); AMLivelinessMonitor amLivelinessMonitor = mock(AMLivelinessMonitor.class); this.rmContext = new RMContextImpl(new MemStore(), rmDispatcher, - containerAllocationExpirer, amLivelinessMonitor); + containerAllocationExpirer, amLivelinessMonitor); rmDispatcher.register(RMAppAttemptEventType.class, - new TestApplicationAttemptEventDispatcher(this.rmContext)); + new TestApplicationAttemptEventDispatcher()); rmDispatcher.register(RMAppEventType.class, new TestApplicationEventDispatcher(rmContext)); - rmDispatcher.init(conf); - rmDispatcher.start(); } protected RMApp createNewTestApp() { @@ -143,23 +122,22 @@ public class TestRMAppTransitions { String clientTokenStr = "bogusstring"; ApplicationStore appStore = mock(ApplicationStore.class); YarnScheduler scheduler = mock(YarnScheduler.class); - ApplicationMasterService masterService = - new ApplicationMasterService(rmContext, - new ApplicationTokenSecretManager(), scheduler); + ApplicationMasterService masterService = new ApplicationMasterService(rmContext, + new ApplicationTokenSecretManager(), scheduler); RMApp application = new RMAppImpl(applicationId, rmContext, - conf, name, user, - queue, submissionContext, clientTokenStr, - appStore, scheduler, - masterService); + conf, name, user, + queue, submissionContext, clientTokenStr, + appStore, scheduler, + masterService); testAppStartState(applicationId, user, name, queue, application); return application; } // Test expected newly created app state - private static void testAppStartState(ApplicationId applicationId, - String user, String name, String queue, RMApp application) { + private static void testAppStartState(ApplicationId applicationId, String user, + String name, String queue, RMApp application) { Assert.assertTrue("application start time is not greater then 0", application.getStartTime() > 0); Assert.assertTrue("application start time is before currentTime", @@ -213,14 +191,6 @@ public class TestRMAppTransitions { "Application killed by user.", diag.toString()); } - private static void assertAppAndAttemptKilled(RMApp application) { - assertKilled(application); - /* also check if the attempt is killed */ - Assert.assertEquals( RMAppAttemptState.KILLED, - application.getCurrentAppAttempt().getAppAttemptState() - ); - } - private static void assertFailed(RMApp application, String regex) { assertTimesAtFinish(application); assertAppState(RMAppState.FAILED, application); @@ -232,8 +202,7 @@ public class TestRMAppTransitions { protected RMApp testCreateAppSubmitted() throws IOException { RMApp application = createNewTestApp(); // NEW => SUBMITTED event RMAppEventType.START - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), RMAppEventType.START); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.START); application.handle(event); assertStartTimeSet(application); assertAppState(RMAppState.SUBMITTED, application); @@ -243,9 +212,7 @@ public class TestRMAppTransitions { protected RMApp testCreateAppAccepted() throws IOException { RMApp application = testCreateAppSubmitted(); // SUBMITTED => ACCEPTED event RMAppEventType.APP_ACCEPTED - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), - RMAppEventType.APP_ACCEPTED); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.APP_ACCEPTED); application.handle(event); assertStartTimeSet(application); assertAppState(RMAppState.ACCEPTED, application); @@ -255,9 +222,7 @@ public class TestRMAppTransitions { protected RMApp testCreateAppRunning() throws IOException { RMApp application = testCreateAppAccepted(); // ACCEPTED => RUNNING event RMAppEventType.ATTEMPT_REGISTERED - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_REGISTERED); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.ATTEMPT_REGISTERED); application.handle(event); assertStartTimeSet(application); assertAppState(RMAppState.RUNNING, application); @@ -267,9 +232,7 @@ public class TestRMAppTransitions { protected RMApp testCreateAppFinished() throws IOException { RMApp application = testCreateAppRunning(); // RUNNING => FINISHED event RMAppEventType.ATTEMPT_FINISHED - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_FINISHED); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.ATTEMPT_FINISHED); application.handle(event); assertAppState(RMAppState.FINISHED, application); assertTimesAtFinish(application); @@ -288,8 +251,7 @@ public class TestRMAppTransitions { RMApp application = createNewTestApp(); // NEW => KILLED event RMAppEventType.KILL - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); application.handle(event); assertKilled(application); } @@ -301,8 +263,7 @@ public class TestRMAppTransitions { RMApp application = createNewTestApp(); // NEW => FAILED event RMAppEventType.APP_REJECTED String rejectedText = "Test Application Rejected"; - RMAppEvent event = - new RMAppRejectedEvent(application.getApplicationId(), rejectedText); + RMAppEvent event = new RMAppRejectedEvent(application.getApplicationId(), rejectedText); application.handle(event); assertFailed(application, rejectedText); } @@ -314,8 +275,7 @@ public class TestRMAppTransitions { RMApp application = testCreateAppSubmitted(); // SUBMITTED => FAILED event RMAppEventType.APP_REJECTED String rejectedText = "app rejected"; - RMAppEvent event = - new RMAppRejectedEvent(application.getApplicationId(), rejectedText); + RMAppEvent event = new RMAppRejectedEvent(application.getApplicationId(), rejectedText); application.handle(event); assertFailed(application, rejectedText); } @@ -327,9 +287,8 @@ public class TestRMAppTransitions { RMApp application = testCreateAppAccepted(); // SUBMITTED => KILLED event RMAppEventType.KILL RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); - this.rmContext.getRMApps().putIfAbsent(application.getApplicationId(), application); application.handle(event); - assertAppAndAttemptKilled(application); + assertKilled(application); } @Test @@ -339,26 +298,18 @@ public class TestRMAppTransitions { RMApp application = testCreateAppAccepted(); // ACCEPTED => ACCEPTED event RMAppEventType.RMAppEventType.ATTEMPT_FAILED for (int i=1; i FAILED event RMAppEventType.RMAppEventType.ATTEMPT_FAILED - // after max retries - String message = "Test fail"; - RMAppEvent event = - new RMAppFailedAttemptEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_FAILED, message); + // ACCEPTED => FAILED event RMAppEventType.RMAppEventType.ATTEMPT_FAILED after max retries + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED); application.handle(event); - assertFailed(application, ".*" + message + ".*Failing the application.*"); + assertFailed(application, ".*Failing the application.*"); } @Test @@ -367,8 +318,7 @@ public class TestRMAppTransitions { RMApp application = testCreateAppAccepted(); // ACCEPTED => KILLED event RMAppEventType.KILL - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); application.handle(event); assertKilled(application); } @@ -379,8 +329,7 @@ public class TestRMAppTransitions { RMApp application = testCreateAppRunning(); // RUNNING => KILLED event RMAppEventType.KILL - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); application.handle(event); assertKilled(application); } @@ -392,35 +341,25 @@ public class TestRMAppTransitions { RMApp application = testCreateAppRunning(); RMAppAttempt appAttempt = application.getCurrentAppAttempt(); int expectedAttemptId = 1; - Assert.assertEquals(expectedAttemptId, - appAttempt.getAppAttemptId().getAttemptId()); + Assert.assertEquals(expectedAttemptId, appAttempt.getAppAttemptId().getAttemptId()); // RUNNING => FAILED/RESTARTING event RMAppEventType.ATTEMPT_FAILED for (int i=1; i FAILED/RESTARTING event RMAppEventType.ATTEMPT_FAILED - // after max retries - RMAppEvent event = - new RMAppFailedAttemptEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_FAILED, ""); + // RUNNING => FAILED/RESTARTING event RMAppEventType.ATTEMPT_FAILED after max retries + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED); application.handle(event); assertFailed(application, ".*Failing the application.*"); @@ -437,8 +376,7 @@ public class TestRMAppTransitions { RMApp application = testCreateAppFinished(); // FINISHED => FINISHED event RMAppEventType.KILL - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); application.handle(event); assertTimesAtFinish(application); assertAppState(RMAppState.FINISHED, application); @@ -454,32 +392,25 @@ public class TestRMAppTransitions { RMApp application = testCreateAppRunning(); // RUNNING => KILLED event RMAppEventType.KILL - RMAppEvent event = - new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); + RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL); application.handle(event); assertTimesAtFinish(application); assertAppState(RMAppState.KILLED, application); // KILLED => KILLED event RMAppEventType.ATTEMPT_FINISHED - event = - new RMAppEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_FINISHED); + event = new RMAppEvent(application.getApplicationId(), RMAppEventType.ATTEMPT_FINISHED); application.handle(event); assertTimesAtFinish(application); assertAppState(RMAppState.KILLED, application); // KILLED => KILLED event RMAppEventType.ATTEMPT_FAILED - event = - new RMAppFailedAttemptEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_FAILED, ""); + event = new RMAppEvent(application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED); application.handle(event); assertTimesAtFinish(application); assertAppState(RMAppState.KILLED, application); // KILLED => KILLED event RMAppEventType.ATTEMPT_KILLED - event = - new RMAppEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_KILLED); + event = new RMAppEvent(application.getApplicationId(), RMAppEventType.ATTEMPT_KILLED); application.handle(event); assertTimesAtFinish(application); assertAppState(RMAppState.KILLED, application); 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/attempt/TestRMAppAttemptTransitions.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java deleted file mode 100644 index 03a4ba07441..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java +++ /dev/null @@ -1,403 +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.rmapp.attempt; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.mockito.Matchers.*; -import static org.mockito.Mockito.*; - -import java.util.Collections; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.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.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.event.AsyncDispatcher; -import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService; -import org.apache.hadoop.yarn.server.resourcemanager.RMContext; -import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; -import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent; -import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; -import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; -import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemStore; -import org.apache.hadoop.yarn.server.resourcemanager.resourcetracker.InlineDispatcher; -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.RMAppFailedAttemptEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class TestRMAppAttemptTransitions { - - private static final Log LOG = - LogFactory.getLog(TestRMAppAttemptTransitions.class); - - private static final String EMPTY_DIAGNOSTICS = ""; - - private RMContext rmContext; - private YarnScheduler scheduler; - private ApplicationMasterService masterService; - private ApplicationMasterLauncher applicationMasterLauncher; - - private RMApp application; - private RMAppAttempt applicationAttempt; - - private final class TestApplicationAttemptEventDispatcher implements - EventHandler { - - @Override - public void handle(RMAppAttemptEvent event) { - ApplicationAttemptId appID = event.getApplicationAttemptId(); - assertEquals(applicationAttempt.getAppAttemptId(), appID); - try { - applicationAttempt.handle(event); - } catch (Throwable t) { - LOG.error("Error in handling event type " + event.getType() - + " for application " + appID, t); - } - } - } - - // handle all the RM application events - same as in ResourceManager.java - private final class TestApplicationEventDispatcher implements - EventHandler { - @Override - public void handle(RMAppEvent event) { - assertEquals(application.getApplicationId(), event.getApplicationId()); - try { - application.handle(event); - } catch (Throwable t) { - LOG.error("Error in handling event type " + event.getType() - + " for application " + application.getApplicationId(), t); - } - } - } - - private final class TestSchedulerEventDispatcher implements - EventHandler { - @Override - public void handle(SchedulerEvent event) { - scheduler.handle(event); - } - } - - private final class TestAMLauncherEventDispatcher implements - EventHandler { - @Override - public void handle(AMLauncherEvent event) { - applicationMasterLauncher.handle(event); - } - } - - private static int appId = 1; - - @Before - public void setUp() throws Exception { - InlineDispatcher rmDispatcher = new InlineDispatcher(); - - ContainerAllocationExpirer containerAllocationExpirer = - mock(ContainerAllocationExpirer.class); - AMLivelinessMonitor amLivelinessMonitor = mock(AMLivelinessMonitor.class); - rmContext = new RMContextImpl(new MemStore(), rmDispatcher, - containerAllocationExpirer, amLivelinessMonitor); - - scheduler = mock(YarnScheduler.class); - masterService = mock(ApplicationMasterService.class); - applicationMasterLauncher = mock(ApplicationMasterLauncher.class); - - rmDispatcher.register(RMAppAttemptEventType.class, - new TestApplicationAttemptEventDispatcher()); - - rmDispatcher.register(RMAppEventType.class, - new TestApplicationEventDispatcher()); - - rmDispatcher.register(SchedulerEventType.class, - new TestSchedulerEventDispatcher()); - - rmDispatcher.register(AMLauncherEventType.class, - new TestAMLauncherEventDispatcher()); - - rmDispatcher.init(new Configuration()); - rmDispatcher.start(); - - - ApplicationId applicationId = MockApps.newAppID(appId++); - ApplicationAttemptId applicationAttemptId = - MockApps.newAppAttemptID(applicationId, 0); - - final String user = MockApps.newUserName(); - final String queue = MockApps.newQueue(); - ApplicationSubmissionContext submissionContext = - mock(ApplicationSubmissionContext.class); - when(submissionContext.getUser()).thenReturn(user); - when(submissionContext.getQueue()).thenReturn(queue); - ContainerLaunchContext amContainerSpec = mock(ContainerLaunchContext.class); - Resource resource = mock(Resource.class); - when(amContainerSpec.getResource()).thenReturn(resource); - when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec); - - application = mock(RMApp.class); - applicationAttempt = - new RMAppAttemptImpl(applicationAttemptId, null, rmContext, scheduler, - masterService, submissionContext); - when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt); - when(application.getApplicationId()).thenReturn(applicationId); - - testAppAttemptNewState(); - } - - @After - public void tearDown() throws Exception { - ((AsyncDispatcher)this.rmContext.getDispatcher()).stop(); - } - - - /** - * {@link RMAppAttemptState#NEW} - */ - private void testAppAttemptNewState() { - assertEquals(RMAppAttemptState.NEW, - applicationAttempt.getAppAttemptState()); - assertEquals(0, applicationAttempt.getDiagnostics().length()); - assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); - assertNull(applicationAttempt.getMasterContainer()); - assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001); - assertEquals(0, applicationAttempt.getRanNodes().size()); - } - - /** - * {@link RMAppAttemptState#SUBMITTED} - */ - private void testAppAttemptSubmittedState() { - assertEquals(RMAppAttemptState.SUBMITTED, - applicationAttempt.getAppAttemptState()); - assertEquals(0, applicationAttempt.getDiagnostics().length()); - assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); - assertNull(applicationAttempt.getMasterContainer()); - assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001); - assertEquals(0, applicationAttempt.getRanNodes().size()); - - // Check events - verify(masterService). - registerAppAttempt(applicationAttempt.getAppAttemptId()); - verify(scheduler).handle(any(AppAddedSchedulerEvent.class)); - } - - /** - * {@link RMAppAttemptState#SUBMITTED} -> {@link RMAppAttemptState#FAILED} - */ - private void testAppAttemptSubmittedToFailedState(String diagnostics) { - assertEquals(RMAppAttemptState.FAILED, - applicationAttempt.getAppAttemptState()); - assertEquals(diagnostics, applicationAttempt.getDiagnostics()); - assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); - assertNull(applicationAttempt.getMasterContainer()); - assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001); - assertEquals(0, applicationAttempt.getRanNodes().size()); - - // Check events - verify(application).handle(any(RMAppRejectedEvent.class)); - } - - /** - * {@link RMAppAttemptState#KILLED} - */ - private void testAppAttemptKilledState(Container amContainer, - String diagnostics) { - assertEquals(RMAppAttemptState.KILLED, - applicationAttempt.getAppAttemptState()); - assertEquals(diagnostics, applicationAttempt.getDiagnostics()); - assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); - assertEquals(amContainer, applicationAttempt.getMasterContainer()); - assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001); - assertEquals(0, applicationAttempt.getRanNodes().size()); - } - - /** - * {@link RMAppAttemptState#SCHEDULED} - */ - private void testAppAttemptScheduledState() { - assertEquals(RMAppAttemptState.SCHEDULED, - applicationAttempt.getAppAttemptState()); - assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); - assertNull(applicationAttempt.getMasterContainer()); - assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001); - assertEquals(0, applicationAttempt.getRanNodes().size()); - - // Check events - verify(application).handle(any(RMAppEvent.class)); - verify(scheduler). - allocate(any(ApplicationAttemptId.class), - any(List.class), any(List.class)); - } - - /** - * {@link RMAppAttemptState#ALLOCATED} - */ - private void testAppAttemptAllocatedState(Container amContainer) { - assertEquals(RMAppAttemptState.ALLOCATED, - applicationAttempt.getAppAttemptState()); - assertEquals(amContainer, applicationAttempt.getMasterContainer()); - - // Check events - verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class)); - verify(scheduler, times(2)). - allocate( - any(ApplicationAttemptId.class), any(List.class), any(List.class)); - } - - /** - * {@link RMAppAttemptState#FAILED} - */ - private void testAppAttemptFailedState(Container container, - String diagnostics) { - assertEquals(RMAppAttemptState.FAILED, - applicationAttempt.getAppAttemptState()); - assertEquals(diagnostics, applicationAttempt.getDiagnostics()); - assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); - assertEquals(container, applicationAttempt.getMasterContainer()); - assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001); - assertEquals(0, applicationAttempt.getRanNodes().size()); - - // Check events - verify(application, times(2)).handle(any(RMAppFailedAttemptEvent.class)); - } - - private void submitApplicationAttempt() { - ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId(); - applicationAttempt.handle( - new RMAppAttemptEvent(appAttemptId, RMAppAttemptEventType.START)); - testAppAttemptSubmittedState(); - } - - private void scheduleApplicationAttempt() { - submitApplicationAttempt(); - applicationAttempt.handle( - new RMAppAttemptEvent( - applicationAttempt.getAppAttemptId(), - RMAppAttemptEventType.APP_ACCEPTED)); - testAppAttemptScheduledState(); - } - - private Container allocateApplicationAttempt() { - scheduleApplicationAttempt(); - - // Mock the allocation of AM container - Container container = mock(Container.class); - Allocation allocation = mock(Allocation.class); - when(allocation.getContainers()). - thenReturn(Collections.singletonList(container)); - when( - scheduler.allocate( - any(ApplicationAttemptId.class), - any(List.class), - any(List.class))). - thenReturn(allocation); - - applicationAttempt.handle( - new RMAppAttemptContainerAllocatedEvent( - applicationAttempt.getAppAttemptId(), - container)); - - testAppAttemptAllocatedState(container); - - return container; - } - - @Test - public void testNewToKilled() { - applicationAttempt.handle( - new RMAppAttemptEvent( - applicationAttempt.getAppAttemptId(), - RMAppAttemptEventType.KILL)); - testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS); - } - - @Test - public void testSubmittedToFailed() { - submitApplicationAttempt(); - String message = "Rejected"; - applicationAttempt.handle( - new RMAppAttemptRejectedEvent( - applicationAttempt.getAppAttemptId(), message)); - testAppAttemptSubmittedToFailedState(message); - } - - @Test - public void testSubmittedToKilled() { - submitApplicationAttempt(); - applicationAttempt.handle( - new RMAppAttemptEvent( - applicationAttempt.getAppAttemptId(), - RMAppAttemptEventType.KILL)); - testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS); - } - - @Test - public void testScheduledToKilled() { - scheduleApplicationAttempt(); - applicationAttempt.handle( - new RMAppAttemptEvent( - applicationAttempt.getAppAttemptId(), - RMAppAttemptEventType.KILL)); - testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS); - } - - @Test - public void testAllocatedToKilled() { - Container amContainer = allocateApplicationAttempt(); - applicationAttempt.handle( - new RMAppAttemptEvent( - applicationAttempt.getAppAttemptId(), - RMAppAttemptEventType.KILL)); - testAppAttemptKilledState(amContainer, EMPTY_DIAGNOSTICS); - } - - @Test - public void testAllocatedToFailed() { - Container amContainer = allocateApplicationAttempt(); - String diagnostics = "Launch Failed"; - applicationAttempt.handle( - new RMAppAttemptLaunchFailedEvent( - applicationAttempt.getAppAttemptId(), - diagnostics)); - testAppAttemptFailedState(amContainer, diagnostics); - } - -} 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 639daf9e5ac..3ea01003320 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 @@ -135,8 +135,7 @@ public class TestLeafQueue { Container container = TestUtils.getMockContainer( containerId, ((SchedulerNode)(invocation.getArguments()[1])).getNodeID(), - (Resource)(invocation.getArguments()[2]), - ((Priority)invocation.getArguments()[3])); + (Resource)(invocation.getArguments()[2])); return container; } } @@ -144,9 +143,7 @@ public class TestLeafQueue { when(queue).createContainer( any(SchedulerApp.class), any(SchedulerNode.class), - any(Resource.class), - any(Priority.class) - ); + any(Resource.class)); // 2. Stub out LeafQueue.parent.completedContainer CSQueue parent = queue.getParent(); @@ -205,8 +202,6 @@ public class TestLeafQueue { assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(1, a.getMetrics().getAllocatedGB()); // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // you can get one container more than user-limit @@ -214,16 +209,12 @@ public class TestLeafQueue { assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(2, a.getMetrics().getAllocatedGB()); // Can't allocate 3rd due to user-limit a.assignContainers(clusterResource, node_0); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(2, a.getMetrics().getAllocatedGB()); // Bump up user-limit-factor, now allocate should work a.setUserLimitFactor(10); @@ -231,16 +222,12 @@ public class TestLeafQueue { assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(3, a.getMetrics().getAllocatedGB()); // One more should work, for app_1, due to user-limit-factor a.assignContainers(clusterResource, node_0); assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(4, a.getMetrics().getAllocatedGB()); // Test max-capacity // Now - no more allocs since we are at max-cap @@ -249,8 +236,6 @@ public class TestLeafQueue { assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(4, a.getMetrics().getAllocatedGB()); // Release each container from app_0 for (RMContainer rmContainer : app_0.getLiveContainers()) { @@ -260,8 +245,6 @@ public class TestLeafQueue { assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(1, a.getMetrics().getAllocatedGB()); // Release each container from app_1 for (RMContainer rmContainer : app_1.getLiveContainers()) { @@ -271,8 +254,6 @@ public class TestLeafQueue { assertEquals(0*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(0, a.getMetrics().getAllocatedGB()); } @Test @@ -492,8 +473,6 @@ public class TestLeafQueue { assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(1, a.getMetrics().getAllocatedGB()); // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // you can get one container more than user-limit @@ -501,8 +480,6 @@ public class TestLeafQueue { assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(2, a.getMetrics().getAllocatedGB()); // Now, reservation should kick in for app_1 a.assignContainers(clusterResource, node_0); @@ -511,8 +488,6 @@ public class TestLeafQueue { assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentReservation().getMemory()); assertEquals(2*GB, node_0.getUsedResource().getMemory()); - assertEquals(4, a.getMetrics().getReservedGB()); - assertEquals(2, a.getMetrics().getAllocatedGB()); // Now free 1 container from app_0 i.e. 1G a.completedContainer(clusterResource, app_0, node_0, @@ -523,8 +498,6 @@ public class TestLeafQueue { assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentReservation().getMemory()); assertEquals(1*GB, node_0.getUsedResource().getMemory()); - assertEquals(4, a.getMetrics().getReservedGB()); - assertEquals(1, a.getMetrics().getAllocatedGB()); // Now finish another container from app_0 and fulfill the reservation a.completedContainer(clusterResource, app_0, node_0, @@ -535,8 +508,6 @@ public class TestLeafQueue { assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentReservation().getMemory()); assertEquals(4*GB, node_0.getUsedResource().getMemory()); - assertEquals(0, a.getMetrics().getReservedGB()); - assertEquals(4, a.getMetrics().getAllocatedGB()); } @Test 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 8459e51d5c2..84dbbac8676 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 @@ -161,13 +161,11 @@ public class TestUtils { } public static Container getMockContainer( - ContainerId containerId, NodeId nodeId, - Resource resource, Priority priority) { + ContainerId containerId, NodeId nodeId, Resource resource) { Container container = mock(Container.class); when(container.getId()).thenReturn(containerId); when(container.getNodeId()).thenReturn(nodeId); when(container.getResource()).thenReturn(resource); - when(container.getPriority()).thenReturn(priority); return container; } } 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/TestNodesPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java deleted file mode 100644 index e0583a20075..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.server.resourcemanager.webapp; - -import java.io.PrintWriter; - -import org.apache.hadoop.yarn.server.resourcemanager.RMContext; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.NodesPage.NodesBlock; -import org.apache.hadoop.yarn.webapp.test.WebAppTests; -import org.junit.Test; -import org.mockito.Mockito; - -/** - * This tests the NodesPage block table that it should contain the table body - * data for all the columns in the table as specified in the header. - */ -public class TestNodesPage { - - @Test - public void testNodesBlockRender() throws Exception { - int numberOfRacks = 2; - int numberOfNodesPerRack = 2; - // Number of Actual Table Headers for NodesPage.NodesBlock might change in - // future. In that case this value should be adjusted to the new value. - int numberOfActualTableHeaders = 7; - - PrintWriter writer = WebAppTests.testBlock( - NodesBlock.class, - RMContext.class, - TestRMWebApp.mockRMContext(3, numberOfRacks, numberOfNodesPerRack, - 8 * TestRMWebApp.GiB)).getInstance(PrintWriter.class); - - Mockito.verify(writer, Mockito.times(numberOfActualTableHeaders)).print( - " release = new ArrayList(); - AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest( - appAttempt.getAppAttemptId(), 0, 0F, ask, release); + AllocateRequest allocateRequest = + recordFactory.newRecordInstance(AllocateRequest.class); + allocateRequest.setApplicationAttemptId(appAttempt.getAppAttemptId()); + allocateRequest.setResponseId(0); + allocateRequest.addAllAsks(ask); + allocateRequest.addAllReleases(release); List allocatedContainers = scheduler.allocate(allocateRequest) .getAMResponse().getAllocatedContainers(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/src/site/apt/SingleCluster.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/src/site/apt/SingleCluster.apt.vm deleted file mode 100644 index affb277b7ff..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/src/site/apt/SingleCluster.apt.vm +++ /dev/null @@ -1,180 +0,0 @@ -~~ Licensed 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. See accompanying LICENSE file. - - --- - Hadoop MapReduce Next Generation ${project.version} - Setting up a Single Node Cluster. - --- - --- - ${maven.build.timestamp} - -Hadoop MapReduce Next Generation - Setting up a Single Node Cluster. - - \[ {{{./index.html}Go Back}} \] - -* Mapreduce Tarball - - You should be able to obtain the MapReduce tarball from the release. - If not, you should be able to create a tarball from the source. - -+---+ -$ mvn clean install -DskipTests -$ cd hadoop-mapreduce-project -$ mvn clean install assembly:assembly -+---+ - <> You will need protoc installed of version 2.4.1 or greater. - - To ignore the native builds in mapreduce you can use <<<-P-cbuild>>> argument - for maven. The tarball should be available in <<>> directory. - - -* Setting up the environment. - - Assuming you have installed hadoop-common/hadoop-hdfs and exported - <<$HADOOP_COMMON_HOME>>/<<$HADOOP_COMMON_HOME>>, untar hadoop mapreduce - tarball and set environment variable <<$HADOOP_MAPRED_HOME>> to the - untarred directory. Set <<$YARN_HOME>> the same as <<$HADOOP_MAPRED_HOME>>. - - <> The following instructions assume you have hdfs running. - -* Setting up Configuration. - - To start the ResourceManager and NodeManager, you will have to update the configs. - Assuming your $HADOOP_CONF_DIR is the configuration directory and has the installed - configs for HDFS and <<>>. There are 2 config files you will have to setup - <<>> and <<>>. - -** Setting up <<>> - - Add the following configs to your <<>>. - -+---+ - - mapreduce.cluster.temp.dir - - No description - true - - - - mapreduce.cluster.local.dir - - No description - true - -+---+ - -** Setting up <<>> - -Add the following configs to your <<>> - -+---+ - - yarn.resourcemanager.resource-tracker.address - host:port - host is the hostname of the resource manager and - port is the port on which the NodeManagers contact the Resource Manager. - - - - - yarn.resourcemanager.scheduler.address - host:port - host is the hostname of the resourcemanager and port is the port - on which the Applications in the cluster talk to the Resource Manager. - - - - - yarn.resourcemanager.scheduler.class - org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler - In case you do not want to use the default scheduler - - - - yarn.resourcemanager.address - host:port - the host is the hostname of the ResourceManager and the port is the port on - which the clients can talk to the Resource Manager. - - - - yarn.nodemanager.local-dirs - - the local directories used by the nodemanager - - - - yarn.nodemanager.address - 0.0.0.0:port - the nodemanagers bind to this port - - - - yarn.nodemanager.resource.memory-gb - 10 - the amount of memory on the NodeManager in GB - - - - yarn.nodemanager.remote-app-log-dir - /app-logs - directory on hdfs where the application logs are moved to - - - - yarn.nodemanager.log-dirs - - the directories used by Nodemanagers as log directories - - - - yarn.nodemanager.aux-services - mapreduce.shuffle - shuffle service that needs to be set for Map Reduce to run - -+---+ - -* Create Symlinks. - - You will have to create the following symlinks: - -+---+ -$ cd $HADOOP_COMMON_HOME/share/hadoop/common/lib/ -$ ln -s $HADOOP_MAPRED_HOME/modules/hadoop-mapreduce-client-app-*-SNAPSHOT.jar . -$ ln -s $HADOOP_MAPRED_HOME/modules/hadoop-mapreduce-client-jobclient-*-SNAPSHOT.jar . -$ ln -s $HADOOP_MAPRED_HOME/modules/hadoop-mapreduce-client-common-*-SNAPSHOT.jar . -$ ln -s $HADOOP_MAPRED_HOME/modules/hadoop-mapreduce-client-shuffle-*-SNAPSHOT.jar . -$ ln -s $HADOOP_MAPRED_HOME/modules/hadoop-mapreduce-client-core-*-SNAPSHOT.jar . -$ ln -s $HADOOP_MAPRED_HOME/modules/hadoop-yarn-common-*-SNAPSHOT.jar . -$ ln -s $HADOOP_MAPRED_HOME/modules/hadoop-yarn-api-*-SNAPSHOT.jar . -+---+ -* Running daemons. - - Assuming that the environment variables <<$HADOOP_COMMON_HOME>>, <<$HADOOP_HDFS_HOME>>, <<$HADOO_MAPRED_HOME>>, - <<$YARN_HOME>>, <<$JAVA_HOME>> and <<$HADOOP_CONF_DIR>> have been set appropriately. - Set $<<$YARN_CONF_DIR>> the same as $<> - - Run ResourceManager and NodeManager as: - -+---+ -$ cd $HADOOP_MAPRED_HOME -$ bin/yarn-daemon.sh start resourcemanager -$ bin/yarn-daemon.sh start nodemanager -+---+ - - You should be up and running. You can run randomwriter as: - -+---+ -$ $HADOOP_COMMON_HOME/bin/hadoop jar hadoop-examples.jar randomwriter out -+---+ - -Good luck. diff --git a/hadoop-mapreduce-project/hadoop-yarn/src/site/apt/index.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/src/site/apt/index.apt.vm deleted file mode 100644 index db9fe870349..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/src/site/apt/index.apt.vm +++ /dev/null @@ -1,39 +0,0 @@ -~~ Licensed 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. See accompanying LICENSE file. - - --- - Hadoop MapReduce Next Generation ${project.version} - --- - --- - ${maven.build.timestamp} - -Hadoop MapReduce Next Generation - -* Architecture - - The new architecture introduced in 0.23, divides the two major functions - of the JobTracker, resource management and job scheduling/monitoring, into separate - components. - The new ResourceManager manages the global assignment of compute resources to applications - and the per-application ApplicationMaster manages the application’s scheduling and coordination. - An application is either a single job in the classic MapReduce jobs or a DAG of such jobs. - The ResourceManager and per-machine NodeManager server, which manages the user processes on that - machine, form the computation fabric. The per-application ApplicationMaster is, in effect, a - framework specific library and is tasked with negotiating resources from the ResourceManager - and working with the NodeManager(s) to execute and monitor the tasks. - -* User Documentation - - * {{{./SingleCluster.html}SingleCluster}} - - * {{{./apidocs/index.html}JavaDocs}} - diff --git a/hadoop-mapreduce-project/hadoop-yarn/src/site/site.xml b/hadoop-mapreduce-project/hadoop-yarn/src/site/site.xml deleted file mode 100644 index 35a75cb2e55..00000000000 --- a/hadoop-mapreduce-project/hadoop-yarn/src/site/site.xml +++ /dev/null @@ -1,34 +0,0 @@ - - - - - - -   - - - - org.apache.maven.skins - maven-stylus-skin - 1.1 - - - - - - - - - diff --git a/hadoop-mapreduce-project/src/contrib/fairscheduler/ivy.xml b/hadoop-mapreduce-project/src/contrib/fairscheduler/ivy.xml index 0b910158df5..e927032d7db 100644 --- a/hadoop-mapreduce-project/src/contrib/fairscheduler/ivy.xml +++ b/hadoop-mapreduce-project/src/contrib/fairscheduler/ivy.xml @@ -48,9 +48,9 @@ + rev="${hadoop-common.version}" conf="common->default"/> + rev="${hadoop-common.version}" conf="test->default"> 0) { LOG.warn("-file option is deprecated, please use generic option" + " -files instead."); - - String fileList = null; + StringBuilder unpackRegex = new StringBuilder( + config_.getPattern(MRJobConfig.JAR_UNPACK_PATTERN, + JobConf.UNPACK_JAR_PATTERN_DEFAULT).pattern()); for (String file : values) { packageFiles_.add(file); - try { - URI pathURI = new URI(file); - Path path = new Path(pathURI); - FileSystem localFs = FileSystem.getLocal(config_); - String finalPath = path.makeQualified(localFs).toString(); - fileList = fileList == null ? finalPath : fileList + "," + finalPath; - } catch (Exception e) { - throw new IllegalArgumentException(e); - } + String fname = new File(file).getName(); + unpackRegex.append("|(?:").append(Pattern.quote(fname)).append(")"); } - config_.set("tmpfiles", config_.get("tmpfiles", "") + - (fileList == null ? "" : fileList)); + config_.setPattern(MRJobConfig.JAR_UNPACK_PATTERN, + Pattern.compile(unpackRegex.toString())); validate(packageFiles_); } - + String fsName = cmdLine.getOptionValue("dfs"); if (null != fsName){ LOG.warn("-dfs option is deprecated, please use -fs instead."); diff --git a/hadoop-mapreduce-project/src/contrib/vaidya/src/java/org/apache/hadoop/vaidya/vaidya.sh b/hadoop-mapreduce-project/src/contrib/vaidya/src/java/org/apache/hadoop/vaidya/vaidya.sh index 2a32cbd1c9d..8ac5b61a5d3 100644 --- a/hadoop-mapreduce-project/src/contrib/vaidya/src/java/org/apache/hadoop/vaidya/vaidya.sh +++ b/hadoop-mapreduce-project/src/contrib/vaidya/src/java/org/apache/hadoop/vaidya/vaidya.sh @@ -31,78 +31,17 @@ script=`basename "$this"` bin=`cd "$bin"; pwd` this="$bin/$script" -# Check if HADOOP_HOME AND JAVA_HOME is set. -if [ -z "$HADOOP_HOME" ] && [ -z "$HADOOP_PREFIX" ] ; then - echo "HADOOP_HOME or HADOOP_PREFIX environment variable should be defined" +# Check if HADOOP_PREFIX AND JAVA_HOME is set. +if [ -z $HADOOP_PREFIX ] ; then + echo "HADOOP_PREFIX environment variable not defined" exit -1; fi -if [ -z "$JAVA_HOME" ] ; then +if [ -z $JAVA_HOME ] ; then echo "JAVA_HOME environment variable not defined" exit -1; fi -if [ -z "$HADOOP_PREFIX" ]; then - hadoopVersion=`$HADOOP_HOME/bin/hadoop version | awk 'BEGIN { RS = "" ; FS = "\n" } ; { print $1 }' | awk '{print $2}'` -else - hadoopVersion=`$HADOOP_PREFIX/bin/hadoop version | awk 'BEGIN { RS = "" ; FS = "\n" } ; { print $1 }' | awk '{print $2}'` -fi +hadoopVersion=`$HADOOP_PREFIX/bin/hadoop version | grep Hadoop | awk '{print $2}'` -# so that filenames w/ spaces are handled correctly in loops below -IFS= - -# for releases, add core hadoop jar to CLASSPATH -if [ -e $HADOOP_PREFIX/share/hadoop/hadoop-core-* ]; then - for f in $HADOOP_PREFIX/share/hadoop/hadoop-core-*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done - - # add libs to CLASSPATH - for f in $HADOOP_PREFIX/share/hadoop/lib/*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done -else - # tarball layout - if [ -e $HADOOP_HOME/hadoop-core-* ]; then - for f in $HADOOP_HOME/hadoop-core-*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done - fi - if [ -e $HADOOP_HOME/build/hadoop-core-* ]; then - for f in $HADOOP_HOME/build/hadoop-core-*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done - fi - for f in $HADOOP_HOME/lib/*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done - - if [ -d "$HADOOP_HOME/build/ivy/lib/Hadoop/common" ]; then - for f in $HADOOP_HOME/build/ivy/lib/Hadoop/common/*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done - fi -fi - -# Set the Vaidya home -if [ -d "$HADOOP_PREFIX/share/hadoop/contrib/vaidya/" ]; then - VAIDYA_HOME=$HADOOP_PREFIX/share/hadoop/contrib/vaidya/ -fi -if [ -d "$HADOOP_HOME/contrib/vaidya" ]; then - VAIDYA_HOME=$HADOOP_HOME/contrib/vaidya/ -fi -if [ -d "$HADOOP_HOME/build/contrib/vaidya" ]; then - VAIDYA_HOME=$HADOOP_HOME/build/contrib/vaidya/ -fi - -# add user-specified CLASSPATH last -if [ "$HADOOP_USER_CLASSPATH_FIRST" = "" ] && [ "$HADOOP_CLASSPATH" != "" ]; then - CLASSPATH=${CLASSPATH}:${HADOOP_CLASSPATH} -fi - -# restore ordinary behaviour -unset IFS - -echo "$CLASSPATH" - -$JAVA_HOME/bin/java -Xmx1024m -classpath $VAIDYA_HOME/hadoop-vaidya-${hadoopVersion}.jar:${CLASSPATH} org.apache.hadoop.vaidya.postexdiagnosis.PostExPerformanceDiagnoser $@ +$JAVA_HOME/bin/java -Xmx1024m -classpath $HADOOP_PREFIX/hadoop-${hadoopVersion}-core.jar:$HADOOP_PREFIX/contrib/vaidya/hadoop-${hadoopVersion}-vaidya.jar:$HADOOP_PREFIX/lib/commons-logging-1.0.4.jar:${CLASSPATH} org.apache.hadoop.vaidya.postexdiagnosis.PostExPerformanceDiagnoser $@ diff --git a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTrackerClientProtocolProvider.java b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTrackerClientProtocolProvider.java index c695816e414..42c958d77c1 100644 --- a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTrackerClientProtocolProvider.java +++ b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTrackerClientProtocolProvider.java @@ -37,30 +37,26 @@ public class JobTrackerClientProtocolProvider extends ClientProtocolProvider { @Override public ClientProtocol create(Configuration conf) throws IOException { String framework = conf.get(MRConfig.FRAMEWORK_NAME); - if (!MRConfig.CLASSIC_FRAMEWORK_NAME.equals(framework)) { + if (framework != null && !framework.equals("classic")) { return null; } String tracker = conf.get(JTConfig.JT_IPC_ADDRESS, "local"); if (!"local".equals(tracker)) { return createRPCProxy(JobTracker.getAddress(conf), conf); - } else { - throw new IOException("Invalid \"" + JTConfig.JT_IPC_ADDRESS - + "\" configuration value for JobTracker: \"" - + tracker + "\""); } + return null; } @Override - public ClientProtocol create(InetSocketAddress addr, Configuration conf) - throws IOException { + public ClientProtocol create(InetSocketAddress addr, Configuration conf) throws IOException { return createRPCProxy(addr, conf); } - + private ClientProtocol createRPCProxy(InetSocketAddress addr, Configuration conf) throws IOException { return (ClientProtocol) RPC.getProxy(ClientProtocol.class, - ClientProtocol.versionID, addr, UserGroupInformation.getCurrentUser(), - conf, NetUtils.getSocketFactory(conf, ClientProtocol.class)); + ClientProtocol.versionID, addr, UserGroupInformation.getCurrentUser(), + conf, NetUtils.getSocketFactory(conf, ClientProtocol.class)); } @Override diff --git a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/LocalClientProtocolProvider.java b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/LocalClientProtocolProvider.java index d09b222ee9b..68d10bc4d00 100644 --- a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/LocalClientProtocolProvider.java +++ b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/LocalClientProtocolProvider.java @@ -37,16 +37,11 @@ public class LocalClientProtocolProvider extends ClientProtocolProvider { if (framework != null && !framework.equals("local")) { return null; } - String tracker = conf.get(JTConfig.JT_IPC_ADDRESS, "local"); - if ("local".equals(tracker)) { + if ("local".equals(conf.get(JTConfig.JT_IPC_ADDRESS, "local"))) { conf.setInt("mapreduce.job.maps", 1); return new LocalJobRunner(conf); - } else { - - throw new IOException("Invalid \"" + JTConfig.JT_IPC_ADDRESS - + "\" configuration value for LocalJobRunner : \"" - + tracker + "\""); } + return null; } @Override diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/MiniMRCluster.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/MiniMRCluster.java index 86980bb73d6..7581f8bc7be 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/MiniMRCluster.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/MiniMRCluster.java @@ -382,7 +382,6 @@ public class MiniMRCluster { UserGroupInformation ugi) { JobConf result = new JobConf(conf); FileSystem.setDefaultUri(result, namenode); - result.set(MRConfig.FRAMEWORK_NAME, MRConfig.CLASSIC_FRAMEWORK_NAME); result.set(JTConfig.JT_IPC_ADDRESS, "localhost:"+jobTrackerPort); result.set(JTConfig.JT_HTTP_ADDRESS, "127.0.0.1:" + jobTrackerInfoPort); diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/QueueManagerTestUtils.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/QueueManagerTestUtils.java index 4cb0fee616c..dee6f57b72f 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/QueueManagerTestUtils.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/QueueManagerTestUtils.java @@ -24,7 +24,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.Cluster; import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.QueueState; import org.apache.hadoop.mapreduce.SleepJob; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; @@ -315,7 +314,6 @@ public class QueueManagerTestUtils { final long reduceSleepTime, boolean shouldComplete, String userInfo, String queueName, Configuration clientConf) throws IOException, InterruptedException, ClassNotFoundException { - clientConf.set(MRConfig.FRAMEWORK_NAME, MRConfig.CLASSIC_FRAMEWORK_NAME); clientConf.set(JTConfig.JT_IPC_ADDRESS, "localhost:" + miniMRCluster.getJobTrackerPort()); UserGroupInformation ugi; diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRClasspath.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRClasspath.java index 2563902d4bc..911aa2cf7c2 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRClasspath.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestMiniMRClasspath.java @@ -55,7 +55,6 @@ public class TestMiniMRClasspath extends TestCase { file.close(); } FileSystem.setDefaultUri(conf, fs.getUri()); - conf.set(JTConfig.FRAMEWORK_NAME, JTConfig.CLASSIC_FRAMEWORK_NAME); conf.set(JTConfig.JT_IPC_ADDRESS, jobTracker); conf.setJobName("wordcount"); conf.setInputFormat(TextInputFormat.class); @@ -122,7 +121,6 @@ public class TestMiniMRClasspath extends TestCase { file.close(); } FileSystem.setDefaultUri(conf, uri); - conf.set(JTConfig.FRAMEWORK_NAME, JTConfig.CLASSIC_FRAMEWORK_NAME); conf.set(JTConfig.JT_IPC_ADDRESS, jobTracker); conf.setJobName("wordcount"); conf.setInputFormat(TextInputFormat.class); diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java index 5e510094ced..dc3355bb4b8 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java @@ -27,14 +27,13 @@ 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.hdfs.MiniDFSCluster; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.IdentityReducer; -import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; import org.apache.hadoop.util.Progressable; @@ -68,7 +67,6 @@ public class TestSpecialCharactersInOutputPath extends TestCase { // use WordCount example FileSystem.setDefaultUri(conf, fileSys); - conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.CLASSIC_FRAMEWORK_NAME); conf.set(JTConfig.JT_IPC_ADDRESS, jobTracker); conf.setJobName("foo"); diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java deleted file mode 100644 index a9044e24308..00000000000 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java +++ /dev/null @@ -1,99 +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; - -import java.io.IOException; - -import junit.framework.TestCase; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapred.LocalJobRunner; -import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; -import org.junit.Test; - -public class TestClientProtocolProviderImpls extends TestCase { - - @Test - public void testClusterWithLocalClientProvider() throws Exception { - - Configuration conf = new Configuration(); - - try { - conf.set(MRConfig.FRAMEWORK_NAME, "incorrect"); - new Cluster(conf); - fail("Cluster should not be initialized with incorrect framework name"); - } catch (IOException e) { - - } - - try { - conf.set(MRConfig.FRAMEWORK_NAME, "local"); - conf.set(JTConfig.JT_IPC_ADDRESS, "127.0.0.1:0"); - - new Cluster(conf); - fail("Cluster with Local Framework name should use local JT address"); - } catch (IOException e) { - - } - - try { - conf.set(JTConfig.JT_IPC_ADDRESS, "local"); - Cluster cluster = new Cluster(conf); - assertTrue(cluster.getClient() instanceof LocalJobRunner); - cluster.close(); - } catch (IOException e) { - - } - } - - @Test - public void testClusterWithJTClientProvider() throws Exception { - - Configuration conf = new Configuration(); - try { - conf.set(MRConfig.FRAMEWORK_NAME, "incorrect"); - new Cluster(conf); - fail("Cluster should not be initialized with incorrect framework name"); - - } catch (IOException e) { - - } - - try { - conf.set(MRConfig.FRAMEWORK_NAME, "classic"); - conf.set(JTConfig.JT_IPC_ADDRESS, "local"); - new Cluster(conf); - fail("Cluster with classic Framework name shouldnot use local JT address"); - - } catch (IOException e) { - - } - - try { - conf = new Configuration(); - conf.set(MRConfig.FRAMEWORK_NAME, "classic"); - conf.set(JTConfig.JT_IPC_ADDRESS, "127.0.0.1:0"); - Cluster cluster = new Cluster(conf); - cluster.close(); - } catch (IOException e) { - - } - } - -} diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index ec342060fe2..a1114a13664 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -76,9 +76,6 @@ https://repository.apache.org/content/repositories/snapshots 1.0.3 - - ${project.build.directory}/test-dir - ${test.build.dir} @@ -562,25 +559,6 @@ - - org.apache.maven.plugins - maven-antrun-plugin - - - create-testdirs - validate - - run - - - - - - - - - - org.apache.maven.plugins maven-compiler-plugin