Committing rest of merge from trunk (accidentally only committed the HDFS portion before)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1214546 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2011-12-14 23:34:04 +00:00
commit b77c107ccb
167 changed files with 8442 additions and 2261 deletions

View File

@ -3,6 +3,7 @@ Hadoop Change Log
Trunk (unreleased changes)
INCOMPATIBLE CHANGES
HADOOP-7920. Remove Avro Rpc. (suresh)
NEW FEATURES
HADOOP-7773. Add support for protocol buffer based RPC engine.
@ -136,6 +137,10 @@ Trunk (unreleased changes)
HADOOP-7913 Fix bug in ProtoBufRpcEngine (sanjay)
HADOOP-7810. move hadoop archive to core from tools. (tucu)
HADOOP-7892. IPC logs too verbose after "RpcKind" introduction (todd)
OPTIMIZATIONS
HADOOP-7761. Improve the performance of raw comparisons. (todd)

View File

@ -239,11 +239,6 @@
<artifactId>avro</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-ipc</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>net.sf.kosmosfs</groupId>
<artifactId>kfs</artifactId>
@ -282,7 +277,6 @@
<phase>generate-test-sources</phase>
<goals>
<goal>schema</goal>
<goal>protocol</goal>
</goals>
</execution>
</executions>

View File

@ -1,269 +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.ipc;
import java.io.Closeable;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import javax.net.SocketFactory;
import org.apache.avro.ipc.Responder;
import org.apache.avro.ipc.Transceiver;
import org.apache.avro.ipc.reflect.ReflectRequestor;
import org.apache.avro.ipc.reflect.ReflectResponder;
import org.apache.avro.ipc.specific.SpecificRequestor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
/** Tunnel Avro-format RPC requests over a Hadoop {@link RPC} connection. This
* does not give cross-language wire compatibility, since the Hadoop RPC wire
* format is non-standard, but it does permit use of Avro's protocol versioning
* features for inter-Java RPCs. */
@InterfaceStability.Evolving
public class AvroRpcEngine implements RpcEngine {
private static final Log LOG = LogFactory.getLog(RPC.class);
private static int VERSION = 1;
// the implementation we tunnel through
private static final RpcEngine ENGINE = new WritableRpcEngine();
/** Tunnel an Avro RPC request and response through Hadoop's RPC. */
private static interface TunnelProtocol extends VersionedProtocol {
//WritableRpcEngine expects a versionID in every protocol.
public static final long versionID = VERSION;
/** All Avro methods and responses go through this. */
BufferListWritable call(String protocol, BufferListWritable request)
throws IOException;
}
/** A Writable that holds a List<ByteBuffer>, The Avro RPC Transceiver's
* basic unit of data transfer.*/
private static class BufferListWritable implements Writable {
private List<ByteBuffer> buffers;
public BufferListWritable() {} // required for RPC Writables
public BufferListWritable(List<ByteBuffer> buffers) {
this.buffers = buffers;
}
public void readFields(DataInput in) throws IOException {
int size = in.readInt();
buffers = new ArrayList<ByteBuffer>(size);
for (int i = 0; i < size; i++) {
int length = in.readInt();
ByteBuffer buffer = ByteBuffer.allocate(length);
in.readFully(buffer.array(), 0, length);
buffers.add(buffer);
}
}
public void write(DataOutput out) throws IOException {
out.writeInt(buffers.size());
for (ByteBuffer buffer : buffers) {
out.writeInt(buffer.remaining());
out.write(buffer.array(), buffer.position(), buffer.remaining());
}
}
}
/** An Avro RPC Transceiver that tunnels client requests through Hadoop
* RPC. */
private static class ClientTransceiver extends Transceiver {
private TunnelProtocol tunnel;
private InetSocketAddress remote;
private String protocol;
public ClientTransceiver(InetSocketAddress addr,
UserGroupInformation ticket,
Configuration conf, SocketFactory factory,
int rpcTimeout, String protocol)
throws IOException {
this.tunnel = ENGINE.getProxy(TunnelProtocol.class, VERSION,
addr, ticket, conf, factory,
rpcTimeout).getProxy();
this.remote = addr;
this.protocol = protocol;
}
public String getRemoteName() { return remote.toString(); }
public List<ByteBuffer> transceive(List<ByteBuffer> request)
throws IOException {
return tunnel.call(protocol, new BufferListWritable(request)).buffers;
}
public List<ByteBuffer> readBuffers() throws IOException {
throw new UnsupportedOperationException();
}
public void writeBuffers(List<ByteBuffer> buffers) throws IOException {
throw new UnsupportedOperationException();
}
public void close() throws IOException {
RPC.stopProxy(tunnel);
}
}
/** Construct a client-side proxy object that implements the named protocol,
* talking to a server at the named address.
* @param <T>*/
@SuppressWarnings("unchecked")
public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
InetSocketAddress addr, UserGroupInformation ticket,
Configuration conf, SocketFactory factory,
int rpcTimeout)
throws IOException {
return new ProtocolProxy<T>(protocol,
(T)Proxy.newProxyInstance(
protocol.getClassLoader(),
new Class[] { protocol },
new Invoker(protocol, addr, ticket, conf, factory, rpcTimeout)),
false);
}
private class Invoker implements InvocationHandler, Closeable {
private final ClientTransceiver tx;
private final SpecificRequestor requestor;
public Invoker(Class<?> protocol, InetSocketAddress addr,
UserGroupInformation ticket, Configuration conf,
SocketFactory factory,
int rpcTimeout) throws IOException {
this.tx = new ClientTransceiver(addr, ticket, conf, factory, rpcTimeout,
protocol.getName());
this.requestor = createRequestor(protocol, tx);
}
@Override public Object invoke(Object proxy, Method method, Object[] args)
throws Throwable {
return requestor.invoke(proxy, method, args);
}
public void close() throws IOException {
tx.close();
}
}
protected SpecificRequestor createRequestor(Class<?> protocol,
Transceiver transeiver) throws IOException {
return new ReflectRequestor(protocol, transeiver);
}
protected Responder createResponder(Class<?> iface, Object impl) {
return new ReflectResponder(iface, impl);
}
/** An Avro RPC Responder that can process requests passed via Hadoop RPC. */
private class TunnelResponder implements TunnelProtocol {
private Map<String, Responder> responders =
new HashMap<String, Responder>();
public void addProtocol(Class<?> iface, Object impl) {
responders.put(iface.getName(), createResponder(iface, impl));
}
@Override
public long getProtocolVersion(String protocol, long version)
throws IOException {
return VERSION;
}
@Override
public ProtocolSignature getProtocolSignature(
String protocol, long version, int clientMethodsHashCode)
throws IOException {
return ProtocolSignature.getProtocolSignature
(clientMethodsHashCode, VERSION, TunnelProtocol.class);
}
public BufferListWritable call(String protocol, BufferListWritable request)
throws IOException {
Responder responder = responders.get(protocol);
if (responder == null)
throw new IOException("No responder for: "+protocol);
return new BufferListWritable(responder.respond(request.buffers));
}
}
public Object[] call(Method method, Object[][] params,
InetSocketAddress[] addrs, UserGroupInformation ticket,
Configuration conf) throws IOException {
throw new UnsupportedOperationException();
}
private class Server extends WritableRpcEngine.Server {
private TunnelResponder responder = new TunnelResponder();
public Server(Class<?> iface, Object impl, String bindAddress,
int port, int numHandlers, int numReaders,
int queueSizePerHandler, boolean verbose,
Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager
) throws IOException {
super((Class)null, new Object(), conf,
bindAddress, port, numHandlers, numReaders,
queueSizePerHandler, verbose, secretManager);
// RpcKind is WRITABLE since Avro is tunneled through WRITABLE
super.addProtocol(RpcKind.RPC_WRITABLE, TunnelProtocol.class, responder);
responder.addProtocol(iface, impl);
}
@Override
public Server
addProtocol(RpcKind rpcKind, Class<?> protocolClass, Object protocolImpl)
throws IOException {
responder.addProtocol(protocolClass, protocolImpl);
return this;
}
}
/** Construct a server for a protocol implementation instance listening on a
* port and address. */
public RPC.Server getServer(Class<?> iface, Object impl, String bindAddress,
int port, int numHandlers, int numReaders,
int queueSizePerHandler, boolean verbose,
Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager
) throws IOException {
return new Server
(iface, impl, bindAddress, port, numHandlers, numReaders,
queueSizePerHandler, verbose, conf, secretManager);
}
}

View File

@ -56,9 +56,8 @@ public class RpcPayloadHeader implements Writable {
public enum RpcKind {
RPC_BUILTIN ((short) 1), // Used for built in calls by tests
RPC_WRITABLE ((short) 2), // Use WritableRpcEngine
RPC_PROTOCOL_BUFFER ((short) 3), // Use ProtobufRpcEngine
RPC_AVRO ((short) 4); // Use AvroRpcEngine
static final short MAX_INDEX = RPC_AVRO.value; // used for array size
RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
private static final short FIRST_INDEX = RPC_BUILTIN.value;
private final short value;

View File

@ -170,7 +170,7 @@ public abstract class Server {
throw new IllegalArgumentException("ReRegistration of rpcKind: " +
rpcKind);
}
LOG.info("rpcKind=" + rpcKind +
LOG.debug("rpcKind=" + rpcKind +
", rpcRequestWrapperClass=" + rpcRequestWrapperClass +
", rpcInvoker=" + rpcInvoker);
}

View File

@ -1,42 +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.
{
"protocol" : "AvroSpecificTestProtocol",
"namespace" : "org.apache.hadoop.ipc",
"messages" : {
"echo" : {
"request" : [ {
"name" : "message",
"type" : "string"
} ],
"response" : "string"
},
"add" : {
"request" : [ {
"name" : "arg1",
"type" : "int"
}, {
"name" : "arg2",
"type" : "int",
"default" : 0
} ],
"response" : "int"
}
}
}

View File

@ -1,227 +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.ipc;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
import java.io.IOException;
import java.net.InetSocketAddress;
import javax.security.sasl.Sasl;
import junit.framework.Assert;
import junit.framework.TestCase;
import org.apache.avro.AvroRemoteException;
import org.apache.avro.util.Utf8;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.TestSaslRPC.CustomSecurityInfo;
import org.apache.hadoop.ipc.TestSaslRPC.TestTokenIdentifier;
import org.apache.hadoop.ipc.TestSaslRPC.TestTokenSecretManager;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SaslRpcServer;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.fs.CommonConfigurationKeys;
/** Unit tests for AvroRpc. */
public class TestAvroRpc extends TestCase {
private static final String ADDRESS = "0.0.0.0";
public static final Log LOG =
LogFactory.getLog(TestAvroRpc.class);
int datasize = 1024*100;
int numThreads = 50;
public TestAvroRpc(String name) { super(name); }
public static interface EmptyProtocol {}
public static class EmptyImpl implements EmptyProtocol {}
public static class TestImpl implements AvroTestProtocol {
public void ping() {}
public String echo(String value) { return value; }
public int add(int v1, int v2) { return v1 + v2; }
public int error() throws Problem {
throw new Problem();
}
}
public void testReflect() throws Exception {
testReflect(false);
}
public void testSecureReflect() throws Exception {
testReflect(true);
}
public void testSpecific() throws Exception {
testSpecific(false);
}
public void testSecureSpecific() throws Exception {
testSpecific(true);
}
private void testReflect(boolean secure) throws Exception {
Configuration conf = new Configuration();
TestTokenSecretManager sm = null;
if (secure) {
makeSecure(conf);
sm = new TestTokenSecretManager();
}
UserGroupInformation.setConfiguration(conf);
RPC.setProtocolEngine(conf, EmptyProtocol.class, AvroRpcEngine.class);
RPC.setProtocolEngine(conf, AvroTestProtocol.class, AvroRpcEngine.class);
RPC.Server server = RPC.getServer(EmptyProtocol.class, new EmptyImpl(),
ADDRESS, 0, 5, true, conf, sm);
server.addProtocol(RpcKind.RPC_WRITABLE,
AvroTestProtocol.class, new TestImpl());
try {
server.start();
InetSocketAddress addr = NetUtils.getConnectAddress(server);
if (secure) {
addToken(sm, addr);
//QOP must be auth
Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
}
AvroTestProtocol proxy =
(AvroTestProtocol)RPC.getProxy(AvroTestProtocol.class, 0, addr, conf);
proxy.ping();
String echo = proxy.echo("hello world");
assertEquals("hello world", echo);
int intResult = proxy.add(1, 2);
assertEquals(3, intResult);
boolean caught = false;
try {
proxy.error();
} catch (AvroRemoteException e) {
if(LOG.isDebugEnabled()) {
LOG.debug("Caught " + e);
}
caught = true;
}
assertTrue(caught);
} finally {
resetSecurity();
server.stop();
}
}
private void makeSecure(Configuration conf) {
conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
conf.set("hadoop.rpc.socket.factory.class.default", "");
//Avro doesn't work with security annotations on protocol.
//Avro works ONLY with custom security context
SecurityUtil.setSecurityInfoProviders(new CustomSecurityInfo());
}
private void resetSecurity() {
SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]);
}
private void addToken(TestTokenSecretManager sm,
InetSocketAddress addr) throws IOException {
final UserGroupInformation current = UserGroupInformation.getCurrentUser();
TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
.getUserName()));
Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
sm);
Text host = new Text(addr.getAddress().getHostAddress() + ":"
+ addr.getPort());
token.setService(host);
LOG.info("Service IP address for token is " + host);
current.addToken(token);
}
private void testSpecific(boolean secure) throws Exception {
Configuration conf = new Configuration();
TestTokenSecretManager sm = null;
if (secure) {
makeSecure(conf);
sm = new TestTokenSecretManager();
}
UserGroupInformation.setConfiguration(conf);
RPC.setProtocolEngine(conf, AvroSpecificTestProtocol.class,
AvroSpecificRpcEngine.class);
Server server = RPC.getServer(AvroSpecificTestProtocol.class,
new AvroSpecificTestProtocolImpl(), ADDRESS, 0, 5, true,
conf, sm);
try {
server.start();
InetSocketAddress addr = NetUtils.getConnectAddress(server);
if (secure) {
addToken(sm, addr);
//QOP must be auth
Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
}
AvroSpecificTestProtocol proxy =
(AvroSpecificTestProtocol)RPC.getProxy(AvroSpecificTestProtocol.class,
0, addr, conf);
CharSequence echo = proxy.echo("hello world");
assertEquals("hello world", echo.toString());
int intResult = proxy.add(1, 2);
assertEquals(3, intResult);
} finally {
resetSecurity();
server.stop();
}
}
public static class AvroSpecificTestProtocolImpl implements
AvroSpecificTestProtocol {
@Override
public int add(int arg1, int arg2) throws AvroRemoteException {
return arg1 + arg2;
}
@Override
public CharSequence echo(CharSequence msg) throws AvroRemoteException {
return msg;
}
}
}

View File

@ -269,6 +269,13 @@
</resources>
<plugins>
<plugin>
<!-- workaround for filtered/unfiltered resources in same directory -->
<!-- remove when maven-eclipse-plugin 2.9 is available -->
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-eclipse-plugin</artifactId>
<version>2.6</version>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>

View File

@ -30,6 +30,7 @@
<modules>
<module>hadoop-hdfs</module>
<module>hadoop-hdfs-httpfs</module>
<module>hadoop-hdfs/src/contrib/bkjournal</module>
</modules>
<build>

View File

@ -3,6 +3,7 @@ Hadoop MapReduce Change Log
Trunk (unreleased changes)
INCOMPATIBLE CHANGES
MAPREDUCE-3545. Remove Avro RPC. (suresh)
NEW FEATURES
@ -77,6 +78,12 @@ Trunk (unreleased changes)
MAPREDUCE-3389. MRApps loads the 'mrapp-generated-classpath' file with
classpath from the build machine. (tucu)
MAPREDUCE-3544. gridmix build is broken, requires hadoop-archives to be added as
ivy dependency. (tucu)
MAPREDUCE-3557. MR1 test fail to compile because of missing hadoop-archives dependency.
(tucu)
Release 0.23.1 - Unreleased
INCOMPATIBLE CHANGES
@ -85,6 +92,9 @@ Release 0.23.1 - Unreleased
MAPREDUCE-3121. NodeManager should handle disk-failures (Ravi Gummadi via mahadev)
MAPREDUCE-2863. Support web services for YARN and MR components. (Thomas
Graves via vinodkv)
IMPROVEMENTS
MAPREDUCE-3297. Moved log related components into yarn-common so that
@ -276,6 +286,22 @@ Release 0.23.1 - Unreleased
MAPREDUCE-3537. Fix race condition in DefaultContainerExecutor which led
to container localization occuring in wrong directories. (acmurthy)
MAPREDUCE-3542. Support "FileSystemCounter" legacy counter group name for
compatibility. (tomwhite)
MAPREDUCE-3426. Fixed MR AM in uber mode to write map intermediate outputs
in the correct directory to work properly in secure mode. (Hitesh Shah via
vinodkv)
MAPREDUCE-3541. Fix broken TestJobQueueClient test. (Ravi Prakash via
mahadev)
MAPREDUCE-3398. Fixed log aggregation to work correctly in secure mode.
(Siddharth Seth via vinodkv)
MAPREDUCE-3530. Fixed an NPE occuring during scheduling in the
ResourceManager. (Arun C Murthy via vinodkv)
Release 0.23.0 - 2011-11-01
INCOMPATIBLE CHANGES

View File

@ -22,20 +22,19 @@ import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
import java.net.URI;
import java.util.HashSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSError;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.TypeConverter;
@ -47,13 +46,12 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunched
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.service.AbstractService;
/**
@ -80,7 +78,10 @@ public class LocalContainerLauncher extends AbstractService implements
super(LocalContainerLauncher.class.getName());
this.context = context;
this.umbilical = umbilical;
// umbilical: MRAppMaster creates (taskAttemptListener), passes to us (TODO/FIXME: pointless to use RPC to talk to self; should create LocalTaskAttemptListener or similar: implement umbilical protocol but skip RPC stuff)
// umbilical: MRAppMaster creates (taskAttemptListener), passes to us
// (TODO/FIXME: pointless to use RPC to talk to self; should create
// LocalTaskAttemptListener or similar: implement umbilical protocol
// but skip RPC stuff)
try {
curFC = FileContext.getFileContext(curDir.toURI());
@ -152,7 +153,6 @@ public class LocalContainerLauncher extends AbstractService implements
* ]]
* - runs Task (runSubMap() or runSubReduce())
* - TA can safely send TA_UPDATE since in RUNNING state
* [modulo possible TA-state-machine race noted below: CHECK (TODO)]
*/
private class SubtaskRunner implements Runnable {
@ -162,6 +162,7 @@ public class LocalContainerLauncher extends AbstractService implements
SubtaskRunner() {
}
@SuppressWarnings("unchecked")
@Override
public void run() {
ContainerLauncherEvent event = null;
@ -183,7 +184,7 @@ public class LocalContainerLauncher extends AbstractService implements
ContainerRemoteLaunchEvent launchEv =
(ContainerRemoteLaunchEvent)event;
TaskAttemptId attemptID = launchEv.getTaskAttemptID(); //FIXME: can attemptID ever be null? (only if retrieved over umbilical?)
TaskAttemptId attemptID = launchEv.getTaskAttemptID();
Job job = context.getAllJobs().get(attemptID.getTaskId().getJobId());
int numMapTasks = job.getTotalMaps();
@ -204,7 +205,6 @@ public class LocalContainerLauncher extends AbstractService implements
// port number is set to -1 in this case.
context.getEventHandler().handle(
new TaskAttemptContainerLaunchedEvent(attemptID, -1));
//FIXME: race condition here? or do we have same kind of lock on TA handler => MapTask can't send TA_UPDATE before TA_CONTAINER_LAUNCHED moves TA to RUNNING state? (probably latter)
if (numMapTasks == 0) {
doneWithMaps = true;
@ -259,6 +259,7 @@ public class LocalContainerLauncher extends AbstractService implements
}
}
@SuppressWarnings("deprecation")
private void runSubtask(org.apache.hadoop.mapred.Task task,
final TaskType taskType,
TaskAttemptId attemptID,
@ -270,6 +271,19 @@ public class LocalContainerLauncher extends AbstractService implements
try {
JobConf conf = new JobConf(getConfig());
conf.set(JobContext.TASK_ID, task.getTaskID().toString());
conf.set(JobContext.TASK_ATTEMPT_ID, classicAttemptID.toString());
conf.setBoolean(JobContext.TASK_ISMAP, (taskType == TaskType.MAP));
conf.setInt(JobContext.TASK_PARTITION, task.getPartition());
conf.set(JobContext.ID, task.getJobID().toString());
// Use the AM's local dir env to generate the intermediate step
// output files
String[] localSysDirs = StringUtils.getTrimmedStrings(
System.getenv(ApplicationConstants.LOCAL_DIR_ENV));
conf.setStrings(MRConfig.LOCAL_DIR, localSysDirs);
LOG.info(MRConfig.LOCAL_DIR + " for uber task: "
+ conf.get(MRConfig.LOCAL_DIR));
// mark this as an uberized subtask so it can set task counter
// (longer-term/FIXME: could redefine as job counter and send
@ -285,12 +299,12 @@ public class LocalContainerLauncher extends AbstractService implements
if (doneWithMaps) {
LOG.error("CONTAINER_REMOTE_LAUNCH contains a map task ("
+ attemptID + "), but should be finished with maps");
// throw new RuntimeException() (FIXME: what's appropriate here?)
throw new RuntimeException();
}
MapTask map = (MapTask)task;
map.setConf(conf);
//CODE-REVIEWER QUESTION: why not task.getConf() or map.getConf() instead of conf? do we need Task's localizeConfiguration() run on this first?
map.run(conf, umbilical);
if (renameOutputs) {
@ -305,19 +319,23 @@ public class LocalContainerLauncher extends AbstractService implements
} else /* TaskType.REDUCE */ {
if (!doneWithMaps) {
//check if event-queue empty? whole idea of counting maps vs. checking event queue is a tad wacky...but could enforce ordering (assuming no "lost events") at LocalMRAppMaster [CURRENT BUG(?): doesn't send reduce event until maps all done]
// check if event-queue empty? whole idea of counting maps vs.
// checking event queue is a tad wacky...but could enforce ordering
// (assuming no "lost events") at LocalMRAppMaster [CURRENT BUG(?):
// doesn't send reduce event until maps all done]
LOG.error("CONTAINER_REMOTE_LAUNCH contains a reduce task ("
+ attemptID + "), but not yet finished with maps");
// throw new RuntimeException() (FIXME) // or push reduce event back onto end of queue? (probably former)
throw new RuntimeException();
}
ReduceTask reduce = (ReduceTask)task;
// a.k.a. "mapreduce.jobtracker.address" in LocalJobRunner:
// set framework name to local to make task local
conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.LOCAL_FRAMEWORK_NAME);
conf.set(MRConfig.MASTER_ADDRESS, "local"); // bypass shuffle
ReduceTask reduce = (ReduceTask)task;
reduce.setConf(conf);
reduce.run(conf, umbilical);
//relocalize(); // needed only if more than one reducer supported (is MAPREDUCE-434 fixed yet?)
}
@ -334,18 +352,7 @@ public class LocalContainerLauncher extends AbstractService implements
try {
if (task != null) {
// do cleanup for the task
// if (childUGI == null) { // no need to job into doAs block
task.taskCleanup(umbilical);
// } else {
// final Task taskFinal = task;
// childUGI.doAs(new PrivilegedExceptionAction<Object>() {
// @Override
// public Object run() throws Exception {
// taskFinal.taskCleanup(umbilical);
// return null;
// }
// });
// }
task.taskCleanup(umbilical);
}
} catch (Exception e) {
LOG.info("Exception cleaning up: "
@ -354,51 +361,21 @@ public class LocalContainerLauncher extends AbstractService implements
// Report back any failures, for diagnostic purposes
ByteArrayOutputStream baos = new ByteArrayOutputStream();
exception.printStackTrace(new PrintStream(baos));
// if (classicAttemptID != null) {
umbilical.reportDiagnosticInfo(classicAttemptID, baos.toString());
// }
umbilical.reportDiagnosticInfo(classicAttemptID, baos.toString());
throw new RuntimeException();
} catch (Throwable throwable) {
LOG.fatal("Error running local (uberized) 'child' : "
+ StringUtils.stringifyException(throwable));
// if (classicAttemptID != null) {
Throwable tCause = throwable.getCause();
String cause = (tCause == null)
? throwable.getMessage()
: StringUtils.stringifyException(tCause);
umbilical.fatalError(classicAttemptID, cause);
// }
Throwable tCause = throwable.getCause();
String cause = (tCause == null)
? throwable.getMessage()
: StringUtils.stringifyException(tCause);
umbilical.fatalError(classicAttemptID, cause);
throw new RuntimeException();
} finally {
/*
FIXME: do we need to do any of this stuff? (guessing not since not in own JVM)
RPC.stopProxy(umbilical);
DefaultMetricsSystem.shutdown();
// Shutting down log4j of the child-vm...
// This assumes that on return from Task.run()
// there is no more logging done.
LogManager.shutdown();
*/
}
}
/* FIXME: may not need renameMapOutputForReduce() anymore? TEST!
${local.dir}/usercache/$user/appcache/$appId/$contId/ == $cwd for containers;
contains launch_container.sh script, which, when executed, creates symlinks and
sets up env
"$local.dir"/usercache/$user/appcache/$appId/$contId/file.out
"$local.dir"/usercache/$user/appcache/$appId/$contId/file.out.idx (?)
"$local.dir"/usercache/$user/appcache/$appId/output/$taskId/ is where file.out* is moved after MapTask done
OHO! no further need for this at all? $taskId is unique per subtask
now => should work fine to leave alone. TODO: test with teragen or
similar
*/
/**
* Within the _local_ filesystem (not HDFS), all activity takes place within
* a single subdir (${local.dir}/usercache/$user/appcache/$appId/$contId/),
@ -409,14 +386,21 @@ sets up env
* filenames instead of "file.out". (All of this is entirely internal,
* so there are no particular compatibility issues.)
*/
@SuppressWarnings("deprecation")
private void renameMapOutputForReduce(JobConf conf, TaskAttemptId mapId,
MapOutputFile subMapOutputFile)
throws IOException {
FileSystem localFs = FileSystem.getLocal(conf);
// move map output to reduce input
Path mapOut = subMapOutputFile.getOutputFile();
FileStatus mStatus = localFs.getFileStatus(mapOut);
Path reduceIn = subMapOutputFile.getInputFileForWrite(
TypeConverter.fromYarn(mapId).getTaskID(), localFs.getLength(mapOut));
TypeConverter.fromYarn(mapId).getTaskID(), mStatus.getLen());
if (LOG.isDebugEnabled()) {
LOG.debug("Renaming map output file for task attempt "
+ mapId.toString() + " from original location " + mapOut.toString()
+ " to destination " + reduceIn.toString());
}
if (!localFs.mkdirs(reduceIn.getParent())) {
throw new IOException("Mkdirs failed to create "
+ reduceIn.getParent().toString());
@ -429,8 +413,7 @@ sets up env
* Also within the local filesystem, we need to restore the initial state
* of the directory as much as possible. Compare current contents against
* the saved original state and nuke everything that doesn't belong, with
* the exception of the renamed map outputs (see above).
FIXME: do we really need to worry about renamed map outputs, or already moved to output dir on commit? if latter, fix comment
* the exception of the renamed map outputs.
*
* Any jobs that go out of their way to rename or delete things from the
* local directory are considered broken and deserve what they get...

View File

@ -236,6 +236,13 @@ public class MapReduceChildJVM {
getTaskLogFile(TaskLog.LogName.PROFILE)
)
);
if (task.isMapTask()) {
vargs.add(conf.get(MRJobConfig.TASK_MAP_PROFILE_PARAMS, ""));
}
else {
vargs.add(conf.get(MRJobConfig.TASK_REDUCE_PROFILE_PARAMS, ""));
}
}
}

View File

@ -156,6 +156,7 @@ public class MRAppMaster extends CompositeService {
private OutputCommitter committer;
private JobEventDispatcher jobEventDispatcher;
private boolean inRecovery = false;
private SpeculatorEventDispatcher speculatorEventDispatcher;
private Job job;
private Credentials fsTokens = new Credentials(); // Filled during init
@ -265,8 +266,9 @@ public class MRAppMaster extends CompositeService {
addIfService(speculator);
}
speculatorEventDispatcher = new SpeculatorEventDispatcher(conf);
dispatcher.register(Speculator.EventType.class,
new SpeculatorEventDispatcher(conf));
speculatorEventDispatcher);
// service to allocate containers from RM (if non-uber) or to fake it (uber)
containerAllocator = createContainerAllocator(clientService, context);
@ -386,7 +388,7 @@ public class MRAppMaster extends CompositeService {
// This will also send the final report to the ResourceManager
LOG.info("Calling stop for all the services");
stop();
// Send job-end notification
try {
LOG.info("Job end notification started for jobID : "
@ -401,14 +403,14 @@ public class MRAppMaster extends CompositeService {
} catch (Throwable t) {
LOG.warn("Graceful stop failed ", t);
}
// Cleanup staging directory
try {
cleanupStagingDir();
} catch(IOException io) {
LOG.warn("Failed to delete staging dir");
}
//Bring the process down by force.
//Not needed after HADOOP-7140
LOG.info("Exiting MR AppMaster..GoodBye!");
@ -790,10 +792,6 @@ public class MRAppMaster extends CompositeService {
// job-init to be done completely here.
jobEventDispatcher.handle(initJobEvent);
// send init to speculator. This won't yest start as dispatcher isn't
// started yet.
dispatcher.getEventHandler().handle(
new SpeculatorEvent(job.getID(), clock.getTime()));
// JobImpl's InitTransition is done (call above is synchronous), so the
// "uber-decision" (MR-1220) has been made. Query job and switch to
@ -801,9 +799,15 @@ public class MRAppMaster extends CompositeService {
// and container-launcher services/event-handlers).
if (job.isUber()) {
speculatorEventDispatcher.disableSpeculation();
LOG.info("MRAppMaster uberizing job " + job.getID()
+ " in local container (\"uber-AM\").");
+ " in local container (\"uber-AM\") on node "
+ nmHost + ":" + nmPort + ".");
} else {
// send init to speculator only for non-uber jobs.
// This won't yet start as dispatcher isn't started yet.
dispatcher.getEventHandler().handle(
new SpeculatorEvent(job.getID(), clock.getTime()));
LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
+ "job " + job.getID() + ".");
}
@ -865,17 +869,24 @@ public class MRAppMaster extends CompositeService {
private class SpeculatorEventDispatcher implements
EventHandler<SpeculatorEvent> {
private final Configuration conf;
private volatile boolean disabled;
public SpeculatorEventDispatcher(Configuration config) {
this.conf = config;
}
@Override
public void handle(SpeculatorEvent event) {
if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|| conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
if (!disabled &&
(conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|| conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false))) {
// Speculator IS enabled, direct the event to there.
speculator.handle(event);
}
}
public void disableSpeculation() {
disabled = true;
}
}
private static void validateInputParam(String value, String param)

View File

@ -151,7 +151,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("mapreduce", AppContext.class, appContext, "ws").with(conf).
start(new AMWebApp());
} catch (Exception e) {
LOG.error("Webapps failed to start. Ignoring for now:", e);

View File

@ -54,6 +54,7 @@ import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
import org.apache.hadoop.mapreduce.jobhistory.JobUnsuccessfulCompletionEvent;
import org.apache.hadoop.mapreduce.lib.chain.ChainMapper;
import org.apache.hadoop.mapreduce.lib.chain.ChainReducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.security.TokenCache;
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
@ -583,13 +584,13 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
if (getState() == JobState.NEW) {
return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
appSubmitTime, startTime, finishTime, setupProgress, 0.0f, 0.0f,
cleanupProgress, remoteJobConfFile.toString(), amInfos);
cleanupProgress, remoteJobConfFile.toString(), amInfos, isUber);
}
return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
appSubmitTime, startTime, finishTime, setupProgress,
computeProgress(mapTasks), computeProgress(reduceTasks),
cleanupProgress, remoteJobConfFile.toString(), amInfos);
cleanupProgress, remoteJobConfFile.toString(), amInfos, isUber);
} finally {
readLock.unlock();
}
@ -812,6 +813,129 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
return amInfos;
}
/**
* Decide whether job can be run in uber mode based on various criteria.
* @param dataInputLength Total length for all splits
*/
private void makeUberDecision(long dataInputLength) {
//FIXME: need new memory criterion for uber-decision (oops, too late here;
// until AM-resizing supported,
// must depend on job client to pass fat-slot needs)
// these are no longer "system" settings, necessarily; user may override
int sysMaxMaps = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
//FIXME: handling multiple reduces within a single AM does not seem to
//work.
// int sysMaxReduces =
// job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
int sysMaxReduces = 1;
long sysMaxBytes = conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
conf.getLong("dfs.block.size", 64*1024*1024)); //FIXME: this is
// wrong; get FS from [File?]InputFormat and default block size from that
long sysMemSizeForUberSlot =
conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
MRJobConfig.DEFAULT_MR_AM_VMEM_MB);
boolean uberEnabled =
conf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
boolean smallNumMapTasks = (numMapTasks <= sysMaxMaps);
boolean smallNumReduceTasks = (numReduceTasks <= sysMaxReduces);
boolean smallInput = (dataInputLength <= sysMaxBytes);
// ignoring overhead due to UberAM and statics as negligible here:
boolean smallMemory =
( (Math.max(conf.getLong(MRJobConfig.MAP_MEMORY_MB, 0),
conf.getLong(MRJobConfig.REDUCE_MEMORY_MB, 0))
<= sysMemSizeForUberSlot)
|| (sysMemSizeForUberSlot == JobConf.DISABLED_MEMORY_LIMIT));
boolean notChainJob = !isChainJob(conf);
// User has overall veto power over uberization, or user can modify
// limits (overriding system settings and potentially shooting
// themselves in the head). Note that ChainMapper/Reducer are
// fundamentally incompatible with MR-1220; they employ a blocking
// queue between the maps/reduces and thus require parallel execution,
// while "uber-AM" (MR AM + LocalContainerLauncher) loops over tasks
// and thus requires sequential execution.
isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
&& smallInput && smallMemory && notChainJob;
if (isUber) {
LOG.info("Uberizing job " + jobId + ": " + numMapTasks + "m+"
+ numReduceTasks + "r tasks (" + dataInputLength
+ " input bytes) will run sequentially on single node.");
// make sure reduces are scheduled only after all map are completed
conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
1.0f);
// uber-subtask attempts all get launched on same node; if one fails,
// probably should retry elsewhere, i.e., move entire uber-AM: ergo,
// limit attempts to 1 (or at most 2? probably not...)
conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
conf.setInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 1);
// disable speculation
conf.setBoolean(MRJobConfig.MAP_SPECULATIVE, false);
conf.setBoolean(MRJobConfig.REDUCE_SPECULATIVE, false);
} else {
StringBuilder msg = new StringBuilder();
msg.append("Not uberizing ").append(jobId).append(" because:");
if (!uberEnabled)
msg.append(" not enabled;");
if (!smallNumMapTasks)
msg.append(" too many maps;");
if (!smallNumReduceTasks)
msg.append(" too many reduces;");
if (!smallInput)
msg.append(" too much input;");
if (!smallMemory)
msg.append(" too much RAM;");
if (!notChainJob)
msg.append(" chainjob");
LOG.info(msg.toString());
}
}
/**
* ChainMapper and ChainReducer must execute in parallel, so they're not
* compatible with uberization/LocalContainerLauncher (100% sequential).
*/
private boolean isChainJob(Configuration conf) {
boolean isChainJob = false;
try {
String mapClassName = conf.get(MRJobConfig.MAP_CLASS_ATTR);
if (mapClassName != null) {
Class<?> mapClass = Class.forName(mapClassName);
if (ChainMapper.class.isAssignableFrom(mapClass))
isChainJob = true;
}
} catch (ClassNotFoundException cnfe) {
// don't care; assume it's not derived from ChainMapper
}
try {
String reduceClassName = conf.get(MRJobConfig.REDUCE_CLASS_ATTR);
if (reduceClassName != null) {
Class<?> reduceClass = Class.forName(reduceClassName);
if (ChainReducer.class.isAssignableFrom(reduceClass))
isChainJob = true;
}
} catch (ClassNotFoundException cnfe) {
// don't care; assume it's not derived from ChainReducer
}
return isChainJob;
}
/*
private int getBlockSize() {
String inputClassName = conf.get(MRJobConfig.INPUT_FORMAT_CLASS_ATTR);
if (inputClassName != null) {
Class<?> inputClass - Class.forName(inputClassName);
if (FileInputFormat<K, V>)
}
}
*/
public static class InitTransition
implements MultipleArcTransition<JobImpl, JobEvent, JobState> {
@ -863,81 +987,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
inputLength += taskSplitMetaInfo[i].getInputDataLength();
}
//FIXME: need new memory criterion for uber-decision (oops, too late here;
// until AM-resizing supported, must depend on job client to pass fat-slot needs)
// these are no longer "system" settings, necessarily; user may override
int sysMaxMaps = job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
int sysMaxReduces =
job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
long sysMaxBytes = job.conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
job.conf.getLong("dfs.block.size", 64*1024*1024)); //FIXME: this is
// wrong; get FS from [File?]InputFormat and default block size from that
//long sysMemSizeForUberSlot = JobTracker.getMemSizeForReduceSlot();
// FIXME [could use default AM-container memory size...]
boolean uberEnabled =
job.conf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
boolean smallNumMapTasks = (job.numMapTasks <= sysMaxMaps);
boolean smallNumReduceTasks = (job.numReduceTasks <= sysMaxReduces);
boolean smallInput = (inputLength <= sysMaxBytes);
boolean smallMemory = true; //FIXME (see above)
// ignoring overhead due to UberTask and statics as negligible here:
// FIXME && (Math.max(memoryPerMap, memoryPerReduce) <= sysMemSizeForUberSlot
// || sysMemSizeForUberSlot == JobConf.DISABLED_MEMORY_LIMIT)
boolean notChainJob = !isChainJob(job.conf);
// User has overall veto power over uberization, or user can modify
// limits (overriding system settings and potentially shooting
// themselves in the head). Note that ChainMapper/Reducer are
// fundamentally incompatible with MR-1220; they employ a blocking
// User has overall veto power over uberization, or user can modify
// limits (overriding system settings and potentially shooting
// themselves in the head). Note that ChainMapper/Reducer are
// fundamentally incompatible with MR-1220; they employ a blocking
// queue between the maps/reduces and thus require parallel execution,
// while "uber-AM" (MR AM + LocalContainerLauncher) loops over tasks
// and thus requires sequential execution.
job.isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
&& smallInput && smallMemory && notChainJob;
if (job.isUber) {
LOG.info("Uberizing job " + job.jobId + ": " + job.numMapTasks + "m+"
+ job.numReduceTasks + "r tasks (" + inputLength
+ " input bytes) will run sequentially on single node.");
//TODO: also note which node?
// make sure reduces are scheduled only after all map are completed
job.conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
1.0f);
// uber-subtask attempts all get launched on same node; if one fails,
// probably should retry elsewhere, i.e., move entire uber-AM: ergo,
// limit attempts to 1 (or at most 2? probably not...)
job.conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
job.conf.setInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 1);
// disable speculation: makes no sense to speculate an entire job
//canSpeculateMaps = canSpeculateReduces = false; // [TODO: in old
//version, ultimately was from conf.getMapSpeculativeExecution(),
//conf.getReduceSpeculativeExecution()]
} else {
StringBuilder msg = new StringBuilder();
msg.append("Not uberizing ").append(job.jobId).append(" because:");
if (!uberEnabled)
msg.append(" not enabled;");
if (!smallNumMapTasks)
msg.append(" too many maps;");
if (!smallNumReduceTasks)
msg.append(" too many reduces;");
if (!smallInput)
msg.append(" too much input;");
if (!smallMemory)
msg.append(" too much RAM;");
if (!notChainJob)
msg.append(" chainjob");
LOG.info(msg.toString());
}
job.makeUberDecision(inputLength);
job.taskAttemptCompletionEvents =
new ArrayList<TaskAttemptCompletionEvent>(
job.numMapTasks + job.numReduceTasks + 10);
@ -1008,35 +1059,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
}
}
/**
* ChainMapper and ChainReducer must execute in parallel, so they're not
* compatible with uberization/LocalContainerLauncher (100% sequential).
*/
boolean isChainJob(Configuration conf) {
boolean isChainJob = false;
try {
String mapClassName = conf.get(MRJobConfig.MAP_CLASS_ATTR);
if (mapClassName != null) {
Class<?> mapClass = Class.forName(mapClassName);
if (ChainMapper.class.isAssignableFrom(mapClass))
isChainJob = true;
}
} catch (ClassNotFoundException cnfe) {
// don't care; assume it's not derived from ChainMapper
}
try {
String reduceClassName = conf.get(MRJobConfig.REDUCE_CLASS_ATTR);
if (reduceClassName != null) {
Class<?> reduceClass = Class.forName(reduceClassName);
if (ChainReducer.class.isAssignableFrom(reduceClass))
isChainJob = true;
}
} catch (ClassNotFoundException cnfe) {
// don't care; assume it's not derived from ChainReducer
}
return isChainJob;
}
private void createMapTasks(JobImpl job, long inputLength,
TaskSplitMetaInfo[] splits) {
for (int i=0; i < job.numMapTasks; ++i) {

View File

@ -60,8 +60,8 @@ public class LocalContainerAllocator extends RMCommunicator
private static final Log LOG =
LogFactory.getLog(LocalContainerAllocator.class);
@SuppressWarnings("rawtypes")
private final EventHandler eventHandler;
// private final ApplicationId appID;
private AtomicInteger containerCount = new AtomicInteger();
private long retryInterval;
private long retrystartTime;
@ -73,8 +73,6 @@ public class LocalContainerAllocator extends RMCommunicator
AppContext context) {
super(clientService, context);
this.eventHandler = context.getEventHandler();
// this.appID = context.getApplicationID();
}
@Override
@ -88,6 +86,7 @@ public class LocalContainerAllocator extends RMCommunicator
retrystartTime = System.currentTimeMillis();
}
@SuppressWarnings("unchecked")
@Override
protected synchronized void heartbeat() throws Exception {
AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(
@ -124,6 +123,7 @@ public class LocalContainerAllocator extends RMCommunicator
}
}
@SuppressWarnings("unchecked")
@Override
public void handle(ContainerAllocatorEvent event) {
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
import java.io.IOException;
import java.security.PrivilegedAction;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -65,7 +66,7 @@ public abstract class RMCommunicator extends AbstractService {
private int rmPollInterval;//millis
protected ApplicationId applicationId;
protected ApplicationAttemptId applicationAttemptId;
private volatile boolean stopped;
private AtomicBoolean stopped;
protected Thread allocatorThread;
protected EventHandler eventHandler;
protected AMRMProtocol scheduler;
@ -88,6 +89,7 @@ public abstract class RMCommunicator extends AbstractService {
this.eventHandler = context.getEventHandler();
this.applicationId = context.getApplicationID();
this.applicationAttemptId = context.getApplicationAttemptId();
this.stopped = new AtomicBoolean(false);
}
@Override
@ -213,7 +215,10 @@ public abstract class RMCommunicator extends AbstractService {
@Override
public void stop() {
stopped = true;
if (stopped.getAndSet(true)) {
// return if already stopped
return;
}
allocatorThread.interrupt();
try {
allocatorThread.join();
@ -228,7 +233,7 @@ public abstract class RMCommunicator extends AbstractService {
allocatorThread = new Thread(new Runnable() {
@Override
public void run() {
while (!stopped && !Thread.currentThread().isInterrupted()) {
while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
try {
Thread.sleep(rmPollInterval);
try {

View File

@ -18,8 +18,9 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import static org.apache.hadoop.yarn.util.StringHelper.*;
import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
import org.apache.hadoop.yarn.webapp.WebApp;
/**
@ -29,6 +30,9 @@ public class AMWebApp extends WebApp implements AMParams {
@Override
public void setup() {
bind(JAXBContextResolver.class);
bind(GenericExceptionHandler.class);
bind(AMWebServices.class);
route("/", AppController.class);
route("/app", AppController.class);
route(pajoin("/job", JOB_ID), AppController.class, "job");

View File

@ -0,0 +1,362 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp;
import java.io.IOException;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response.Status;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobACL;
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.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AppInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobsInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.webapp.BadRequestException;
import org.apache.hadoop.yarn.webapp.NotFoundException;
import com.google.inject.Inject;
@Path("/ws/v1/mapreduce")
public class AMWebServices {
private final AppContext appCtx;
private final App app;
private final Configuration conf;
@Inject
public AMWebServices(final App app, final AppContext context,
final Configuration conf) {
this.appCtx = context;
this.app = app;
this.conf = conf;
}
Boolean hasAccess(Job job, HttpServletRequest request) {
UserGroupInformation callerUgi = UserGroupInformation
.createRemoteUser(request.getRemoteUser());
if (!job.checkAccess(callerUgi, JobACL.VIEW_JOB)) {
return false;
}
return true;
}
/**
* check for job access.
*
* @param job
* the job that is being accessed
*/
void checkAccess(Job job, HttpServletRequest request) {
if (!hasAccess(job, request)) {
throw new WebApplicationException(Status.UNAUTHORIZED);
}
}
@GET
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public AppInfo get() {
return getAppInfo();
}
@GET
@Path("/info")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public AppInfo getAppInfo() {
return new AppInfo(this.app, this.app.context);
}
@GET
@Path("/jobs")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobsInfo getJobs(@Context HttpServletRequest hsr) {
JobsInfo allJobs = new JobsInfo();
for (Job job : appCtx.getAllJobs().values()) {
// getAllJobs only gives you a partial we want a full
Job fullJob = appCtx.getJob(job.getID());
if (fullJob == null) {
continue;
}
allJobs.add(new JobInfo(fullJob, hasAccess(fullJob, hsr)));
}
return allJobs;
}
@GET
@Path("/jobs/{jobid}")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobInfo getJob(@Context HttpServletRequest hsr,
@PathParam("jobid") String jid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
return new JobInfo(job, hasAccess(job, hsr));
}
@GET
@Path("/jobs/{jobid}/counters")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobCounterInfo getJobCounters(@Context HttpServletRequest hsr,
@PathParam("jobid") String jid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
checkAccess(job, hsr);
return new JobCounterInfo(this.appCtx, job);
}
@GET
@Path("/jobs/{jobid}/tasks/{taskid}/counters")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobTaskCounterInfo getSingleTaskCounters(
@Context HttpServletRequest hsr, @PathParam("jobid") String jid,
@PathParam("taskid") String tid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = this.appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
checkAccess(job, hsr);
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
return new JobTaskCounterInfo(task);
}
@GET
@Path("/jobs/{jobid}/conf")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public ConfInfo getJobConf(@Context HttpServletRequest hsr,
@PathParam("jobid") String jid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
checkAccess(job, hsr);
ConfInfo info;
try {
info = new ConfInfo(job, this.conf);
} catch (IOException e) {
throw new NotFoundException("unable to load configuration for job: " + jid);
}
return info;
}
@GET
@Path("/jobs/{jobid}/tasks")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public TasksInfo getJobTasks(@Context HttpServletRequest hsr,
@PathParam("jobid") String jid, @QueryParam("type") String type) {
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
checkAccess(job, hsr);
TasksInfo allTasks = new TasksInfo();
for (Task task : job.getTasks().values()) {
TaskType ttype = null;
if (type != null && !type.isEmpty()) {
try {
ttype = MRApps.taskType(type);
} catch (YarnException e) {
throw new BadRequestException("tasktype must be either m or r"); }
}
if (ttype != null && task.getType() != ttype) {
continue;
}
allTasks.add(new TaskInfo(task));
}
return allTasks;
}
@GET
@Path("/jobs/{jobid}/tasks/{taskid}")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public TaskInfo getJobTask(@Context HttpServletRequest hsr,
@PathParam("jobid") String jid, @PathParam("taskid") String tid) {
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
checkAccess(job, hsr);
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
return new TaskInfo(task);
}
@GET
@Path("/jobs/{jobid}/tasks/{taskid}/attempts")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public TaskAttemptsInfo getJobTaskAttempts(@Context HttpServletRequest hsr,
@PathParam("jobid") String jid, @PathParam("taskid") String tid) {
TaskAttemptsInfo attempts = new TaskAttemptsInfo();
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
checkAccess(job, hsr);
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
for (TaskAttempt ta : task.getAttempts().values()) {
if (ta != null) {
if (task.getType() == TaskType.REDUCE) {
attempts.add(new ReduceTaskAttemptInfo(ta, task.getType()));
} else {
attempts.add(new TaskAttemptInfo(ta, task.getType(), true));
}
}
}
return attempts;
}
@GET
@Path("/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public TaskAttemptInfo getJobTaskAttemptId(@Context HttpServletRequest hsr,
@PathParam("jobid") String jid, @PathParam("taskid") String tid,
@PathParam("attemptid") String attId) {
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
checkAccess(job, hsr);
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
if (attemptId == null) {
throw new NotFoundException("task attempt id " + attId
+ " not found or invalid");
}
TaskAttempt ta = task.getAttempt(attemptId);
if (ta == null) {
throw new NotFoundException("Error getting info on task attempt id "
+ attId);
}
if (task.getType() == TaskType.REDUCE) {
return new ReduceTaskAttemptInfo(ta, task.getType());
} else {
return new TaskAttemptInfo(ta, task.getType(), true);
}
}
@GET
@Path("/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/counters")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobTaskAttemptCounterInfo getJobTaskAttemptIdCounters(
@Context HttpServletRequest hsr, @PathParam("jobid") String jid,
@PathParam("taskid") String tid, @PathParam("attemptid") String attId) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = this.appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
checkAccess(job, hsr);
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
if (attemptId == null) {
throw new NotFoundException("task attempt id " + attId
+ " not found or invalid");
}
TaskAttempt ta = task.getAttempt(attemptId);
if (ta == null) {
throw new NotFoundException("Error getting info on task attempt id "
+ attId);
}
return new JobTaskAttemptCounterInfo(ta);
}
}

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.mapreduce.JobACL;
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.job.Job;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AppInfo;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@ -72,13 +73,14 @@ public class AppController extends Controller implements AMParams {
* Render the /info page with an overview of current application.
*/
public void info() {
AppInfo info = new AppInfo(app, app.context);
info("Application Master Overview").
_("Application ID:", $(APP_ID)).
_("Application Name:", app.context.getApplicationName()).
_("User:", app.context.getUser()).
_("Started on:", Times.format(app.context.getStartTime())).
_("Application ID:", info.getId()).
_("Application Name:", info.getName()).
_("User:", info.getUser()).
_("Started on:", Times.format(info.getStartTime())).
_("Elasped: ", org.apache.hadoop.util.StringUtils.formatTime(
Times.elapsed(app.context.getStartTime(), 0)));
info.getElapsedTime() ));
render(InfoPage.class);
}

View File

@ -22,14 +22,14 @@ import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.JOB_ID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._TH;
import java.io.IOException;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
@ -71,11 +71,8 @@ public class ConfBlock extends HtmlBlock {
}
Path confPath = job.getConfFile();
try {
//Read in the configuration file and put it in a key/value table.
FileContext fc = FileContext.getFileContext(confPath.toUri(), conf);
Configuration jobConf = new Configuration(false);
jobConf.addResource(fc.open(confPath));
ConfInfo info = new ConfInfo(job, this.conf);
html.div().h3(confPath.toString())._();
TBODY<TABLE<Hamlet>> tbody = html.
// Tasks table
@ -87,10 +84,10 @@ public class ConfBlock extends HtmlBlock {
_().
_().
tbody();
for(Map.Entry<String, String> entry : jobConf) {
for (ConfEntryInfo entry : info.getProperties()) {
tbody.
tr().
td(entry.getKey()).
td(entry.getName()).
td(entry.getValue()).
_();
}

View File

@ -0,0 +1,77 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp;
import java.util.Set;
import java.util.HashSet;
import java.util.Arrays;
import com.sun.jersey.api.json.JSONConfiguration;
import com.sun.jersey.api.json.JSONJAXBContext;
import com.google.inject.Singleton;
import javax.ws.rs.ext.ContextResolver;
import javax.ws.rs.ext.Provider;
import javax.xml.bind.JAXBContext;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AppInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.CounterGroupInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.CounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobsInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterGroupInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
@Singleton
@Provider
public class JAXBContextResolver implements ContextResolver<JAXBContext> {
private JAXBContext context;
private final Set<Class> types;
// you have to specify all the dao classes here
private final Class[] cTypes = {AppInfo.class, CounterInfo.class,
JobTaskAttemptCounterInfo.class, JobTaskCounterInfo.class,
TaskCounterGroupInfo.class, ConfInfo.class, JobCounterInfo.class,
TaskCounterInfo.class, CounterGroupInfo.class, JobInfo.class,
JobsInfo.class, ReduceTaskAttemptInfo.class, TaskAttemptInfo.class,
TaskInfo.class, TasksInfo.class, TaskAttemptsInfo.class,
ConfEntryInfo.class};
public JAXBContextResolver() throws Exception {
this.types = new HashSet<Class>(Arrays.asList(cTypes));
this.context = new JSONJAXBContext(JSONConfiguration.natural().
rootUnwrapping(false).build(), cTypes);
}
@Override
public JAXBContext getContext(Class<?> objectType) {
return (types.contains(objectType)) ? context : null;
}
}

View File

@ -18,47 +18,32 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import com.google.inject.Inject;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.JOB_ID;
import static org.apache.hadoop.yarn.util.StringHelper.join;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._EVEN;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._INFO_WRAP;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._ODD;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._TH;
import java.util.Date;
import java.util.Map;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import org.apache.hadoop.yarn.webapp.view.InfoBlock;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*;
import static org.apache.hadoop.yarn.util.StringHelper.*;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
import com.google.inject.Inject;
public class JobBlock extends HtmlBlock {
final AppContext appContext;
int runningMapTasks = 0;
int pendingMapTasks = 0;
int runningReduceTasks = 0;
int pendingReduceTasks = 0;
int newMapAttempts = 0;
int runningMapAttempts = 0;
int killedMapAttempts = 0;
int failedMapAttempts = 0;
int successfulMapAttempts = 0;
int newReduceAttempts = 0;
int runningReduceAttempts = 0;
int killedReduceAttempts = 0;
int failedReduceAttempts = 0;
int successfulReduceAttempts = 0;
@Inject JobBlock(AppContext appctx) {
appContext = appctx;
}
@ -77,23 +62,13 @@ public class JobBlock extends HtmlBlock {
p()._("Sorry, ", jid, " not found.")._();
return;
}
JobReport jobReport = job.getReport();
String mapPct = percent(jobReport.getMapProgress());
String reducePct = percent(jobReport.getReduceProgress());
int mapTasks = job.getTotalMaps();
int mapTasksComplete = job.getCompletedMaps();
int reduceTasks = job.getTotalReduces();
int reducesTasksComplete = job.getCompletedReduces();
long startTime = jobReport.getStartTime();
long finishTime = jobReport.getFinishTime();
countTasksAndAttempts(job);
JobInfo jinfo = new JobInfo(job, true);
info("Job Overview").
_("Job Name:", job.getName()).
_("State:", job.getState()).
_("Uberized:", job.isUber()).
_("Started:", new Date(startTime)).
_("Elapsed:", StringUtils.formatTime(
Times.elapsed(startTime, finishTime)));
_("Job Name:", jinfo.getName()).
_("State:", jinfo.getState()).
_("Uberized:", jinfo.isUberized()).
_("Started:", new Date(jinfo.getStartTime())).
_("Elapsed:", StringUtils.formatTime(jinfo.getElapsedTime()));
html.
_(InfoBlock.class).
div(_INFO_WRAP).
@ -112,25 +87,25 @@ public class JobBlock extends HtmlBlock {
a(url("tasks", jid, "m"), "Map")._().
td().
div(_PROGRESSBAR).
$title(join(mapPct, '%')). // tooltip
$title(join(jinfo.getMapProgressPercent(), '%')). // tooltip
div(_PROGRESSBAR_VALUE).
$style(join("width:", mapPct, '%'))._()._()._().
td(String.valueOf(mapTasks)).
td(String.valueOf(pendingMapTasks)).
td(String.valueOf(runningMapTasks)).
td(String.valueOf(mapTasksComplete))._().
$style(join("width:", jinfo.getMapProgressPercent(), '%'))._()._()._().
td(String.valueOf(jinfo.getMapsTotal())).
td(String.valueOf(jinfo.getMapsPending())).
td(String.valueOf(jinfo.getMapsRunning())).
td(String.valueOf(jinfo.getMapsCompleted()))._().
tr(_EVEN).
th().
a(url("tasks", jid, "r"), "Reduce")._().
td().
div(_PROGRESSBAR).
$title(join(reducePct, '%')). // tooltip
$title(join(jinfo.getReduceProgressPercent(), '%')). // tooltip
div(_PROGRESSBAR_VALUE).
$style(join("width:", reducePct, '%'))._()._()._().
td(String.valueOf(reduceTasks)).
td(String.valueOf(pendingReduceTasks)).
td(String.valueOf(runningReduceTasks)).
td(String.valueOf(reducesTasksComplete))._()
$style(join("width:", jinfo.getReduceProgressPercent(), '%'))._()._()._().
td(String.valueOf(jinfo.getReducesTotal())).
td(String.valueOf(jinfo.getReducesPending())).
td(String.valueOf(jinfo.getReducesRunning())).
td(String.valueOf(jinfo.getReducesCompleted()))._()
._().
// Attempts table
@ -145,110 +120,41 @@ public class JobBlock extends HtmlBlock {
tr(_ODD).
th("Maps").
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.NEW.toString()),
String.valueOf(newMapAttempts))._().
TaskAttemptStateUI.NEW.toString()),
String.valueOf(jinfo.getNewMapAttempts()))._().
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.RUNNING.toString()),
String.valueOf(runningMapAttempts))._().
TaskAttemptStateUI.RUNNING.toString()),
String.valueOf(jinfo.getRunningMapAttempts()))._().
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.FAILED.toString()),
String.valueOf(failedMapAttempts))._().
TaskAttemptStateUI.FAILED.toString()),
String.valueOf(jinfo.getFailedMapAttempts()))._().
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.KILLED.toString()),
String.valueOf(killedMapAttempts))._().
TaskAttemptStateUI.KILLED.toString()),
String.valueOf(jinfo.getKilledMapAttempts()))._().
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.SUCCESSFUL.toString()),
String.valueOf(successfulMapAttempts))._().
TaskAttemptStateUI.SUCCESSFUL.toString()),
String.valueOf(jinfo.getSuccessfulMapAttempts()))._().
_().
tr(_EVEN).
th("Reduces").
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.NEW.toString()),
String.valueOf(newReduceAttempts))._().
TaskAttemptStateUI.NEW.toString()),
String.valueOf(jinfo.getNewReduceAttempts()))._().
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.RUNNING.toString()),
String.valueOf(runningReduceAttempts))._().
TaskAttemptStateUI.RUNNING.toString()),
String.valueOf(jinfo.getRunningReduceAttempts()))._().
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.FAILED.toString()),
String.valueOf(failedReduceAttempts))._().
TaskAttemptStateUI.FAILED.toString()),
String.valueOf(jinfo.getFailedReduceAttempts()))._().
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.KILLED.toString()),
String.valueOf(killedReduceAttempts))._().
TaskAttemptStateUI.KILLED.toString()),
String.valueOf(jinfo.getKilledReduceAttempts()))._().
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.SUCCESSFUL.toString()),
String.valueOf(successfulReduceAttempts))._().
TaskAttemptStateUI.SUCCESSFUL.toString()),
String.valueOf(jinfo.getSuccessfulReduceAttempts()))._().
_().
_().
_();
}
private void countTasksAndAttempts(Job job) {
Map<TaskId, Task> tasks = job.getTasks();
for (Task task : tasks.values()) {
switch (task.getType()) {
case MAP:
// Task counts
switch (task.getState()) {
case RUNNING:
++runningMapTasks;
break;
case SCHEDULED:
++pendingMapTasks;
break;
}
break;
case REDUCE:
// Task counts
switch (task.getState()) {
case RUNNING:
++runningReduceTasks;
break;
case SCHEDULED:
++pendingReduceTasks;
break;
}
break;
}
// Attempts counts
Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
for (TaskAttempt attempt : attempts.values()) {
int newAttempts = 0, running = 0, successful = 0, failed = 0, killed =0;
if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
++newAttempts;
} else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
.getState())) {
++running;
} else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
.getState())) {
++successful;
} else if (TaskAttemptStateUI.FAILED
.correspondsTo(attempt.getState())) {
++failed;
} else if (TaskAttemptStateUI.KILLED
.correspondsTo(attempt.getState())) {
++killed;
}
switch (task.getType()) {
case MAP:
newMapAttempts += newAttempts;
runningMapAttempts += running;
successfulMapAttempts += successful;
failedMapAttempts += failed;
killedMapAttempts += killed;
break;
case REDUCE:
newReduceAttempts += newAttempts;
runningReduceAttempts += running;
successfulReduceAttempts += successful;
failedReduceAttempts += failed;
killedReduceAttempts += killed;
break;
}
}
}
}
}

View File

@ -18,18 +18,19 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import com.google.inject.Inject;
import static org.apache.hadoop.yarn.util.StringHelper.join;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.*;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import static org.apache.hadoop.yarn.util.StringHelper.*;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
import com.google.inject.Inject;
public class JobsBlock extends HtmlBlock {
final AppContext appContext;
@ -54,38 +55,31 @@ public class JobsBlock extends HtmlBlock {
th("Reduces Total").
th("Reduces Completed")._()._().
tbody();
for (Job job : appContext.getAllJobs().values()) {
String jobID = MRApps.toString(job.getID());
JobReport report = job.getReport();
String mapPct = percent(report.getMapProgress());
String mapsTotal = String.valueOf(job.getTotalMaps());
String mapsCompleted = String.valueOf(job.getCompletedMaps());
String reducePct = percent(report.getReduceProgress());
String reduceTotal = String.valueOf(job.getTotalReduces());
String reduceCompleted = String.valueOf(job.getCompletedReduces());
for (Job j : appContext.getAllJobs().values()) {
JobInfo job = new JobInfo(j, false);
tbody.
tr().
td().
span().$title(String.valueOf(job.getID().getId()))._(). // for sorting
a(url("job", jobID), jobID)._().
td(job.getName().toString()).
td(job.getState().toString()).
span().$title(String.valueOf(job.getId()))._(). // for sorting
a(url("job", job.getId()), job.getId())._().
td(job.getName()).
td(job.getState()).
td().
span().$title(mapPct)._(). // for sorting
span().$title(job.getMapProgressPercent())._(). // for sorting
div(_PROGRESSBAR).
$title(join(mapPct, '%')). // tooltip
$title(join(job.getMapProgressPercent(), '%')). // tooltip
div(_PROGRESSBAR_VALUE).
$style(join("width:", mapPct, '%'))._()._()._().
td(mapsTotal).
td(mapsCompleted).
$style(join("width:", job.getMapProgressPercent(), '%'))._()._()._().
td(String.valueOf(job.getMapsTotal())).
td(String.valueOf(job.getMapsCompleted())).
td().
span().$title(reducePct)._(). // for sorting
span().$title(job.getReduceProgressPercent())._(). // for sorting
div(_PROGRESSBAR).
$title(join(reducePct, '%')). // tooltip
$title(join(job.getReduceProgressPercent(), '%')). // tooltip
div(_PROGRESSBAR_VALUE).
$style(join("width:", reducePct, '%'))._()._()._().
td(reduceTotal).
td(reduceCompleted)._();
$style(join("width:", job.getReduceProgressPercent(), '%'))._()._()._().
td(String.valueOf(job.getReducesTotal())).
td(String.valueOf(job.getReducesCompleted()))._();
}
tbody._()._();
}

View File

@ -18,23 +18,29 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import static org.apache.hadoop.yarn.util.StringHelper.percent;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
import java.util.Collection;
import com.google.common.base.Joiner;
import com.google.inject.Inject;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.webapp.SubView;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.*;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TD;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import static org.apache.hadoop.yarn.util.StringHelper.*;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
import com.google.inject.Inject;
public class TaskPage extends AppView {
@ -66,24 +72,26 @@ public class TaskPage extends AppView {
th(".tsh", "Elapsed").
th(".note", "Note")._()._().
tbody();
for (TaskAttempt ta : getTaskAttempts()) {
String taid = MRApps.toString(ta.getID());
String progress = percent(ta.getProgress());
ContainerId containerId = ta.getAssignedContainerID();
for (TaskAttempt attempt : getTaskAttempts()) {
TaskAttemptInfo ta = new TaskAttemptInfo(attempt, true);
String taid = ta.getId();
String progress = percent(ta.getProgress() / 100);
ContainerId containerId = ta.getAssignedContainerId();
String nodeHttpAddr = ta.getNodeHttpAddress();
long startTime = ta.getLaunchTime();
String nodeHttpAddr = ta.getNode();
long startTime = ta.getStartTime();
long finishTime = ta.getFinishTime();
long elapsed = Times.elapsed(startTime, finishTime);
long elapsed = ta.getElapsedTime();
String diag = ta.getNote() == null ? "" : ta.getNote();
TD<TR<TBODY<TABLE<Hamlet>>>> nodeTd = tbody.
tr().
td(".id", taid).
td(".progress", progress).
td(".state", ta.getState().toString()).
td(".state", ta.getState()).
td().
a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr);
if (containerId != null) {
String containerIdStr = ConverterUtils.toString(containerId);
String containerIdStr = ta.getAssignedContainerIdStr();
nodeTd._(" ").
a(".logslink", url("http://", nodeHttpAddr, "node", "containerlogs",
containerIdStr, app.getJob().getUserName()), "logs");
@ -92,7 +100,7 @@ public class TaskPage extends AppView {
td(".ts", Times.format(startTime)).
td(".ts", Times.format(finishTime)).
td(".dt", StringUtils.formatTime(elapsed)).
td(".note", Joiner.on('\n').join(ta.getDiagnostics()))._();
td(".note", diag)._();
}
tbody._()._();
}

View File

@ -18,21 +18,24 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import com.google.inject.Inject;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
import static org.apache.hadoop.yarn.util.StringHelper.join;
import static org.apache.hadoop.yarn.util.StringHelper.percent;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.*;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*;
import static org.apache.hadoop.yarn.util.StringHelper.*;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
import com.google.inject.Inject;
public class TasksBlock extends HtmlBlock {
final App app;
@ -67,16 +70,16 @@ public class TasksBlock extends HtmlBlock {
if (type != null && task.getType() != type) {
continue;
}
String tid = MRApps.toString(task.getID());
TaskReport report = task.getReport();
String pct = percent(report.getProgress());
long startTime = report.getStartTime();
long finishTime = report.getFinishTime();
long elapsed = Times.elapsed(startTime, finishTime);
TaskInfo info = new TaskInfo(task);
String tid = info.getId();
String pct = percent(info.getProgress() / 100);
long startTime = info.getStartTime();
long finishTime = info.getFinishTime();
long elapsed = info.getElapsedTime();
tbody.
tr().
td().
br().$title(String.valueOf(task.getID().getId()))._(). // sorting
br().$title(String.valueOf(info.getTaskNum()))._(). // sorting
a(url("task", tid), tid)._().
td().
br().$title(pct)._().
@ -84,7 +87,7 @@ public class TasksBlock extends HtmlBlock {
$title(join(pct, '%')). // tooltip
div(_PROGRESSBAR_VALUE).
$style(join("width:", pct, '%'))._()._()._().
td(report.getTaskState().toString()).
td(info.getState()).
td().
br().$title(String.valueOf(startTime))._().
_(Times.format(startTime))._().

View File

@ -0,0 +1,70 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.webapp.App;
import org.apache.hadoop.yarn.util.Times;
@XmlRootElement(name = "info")
@XmlAccessorType(XmlAccessType.FIELD)
public class AppInfo {
protected String appId;
protected String name;
protected String user;
protected String hostname;
protected long startedOn;
protected long elapsedTime;
public AppInfo() {
}
public AppInfo(App app, AppContext context) {
this.appId = context.getApplicationID().toString();
this.name = context.getApplicationName().toString();
this.user = context.getUser().toString();
this.startedOn = context.getStartTime();
this.elapsedTime = Times.elapsed(context.getStartTime(), 0);
}
public String getId() {
return this.appId;
}
public String getName() {
return this.name;
}
public String getUser() {
return this.user;
}
public long getStartTime() {
return this.startedOn;
}
public long getElapsedTime() {
return this.elapsedTime;
}
}

View File

@ -0,0 +1,46 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement
@XmlAccessorType(XmlAccessType.FIELD)
public class ConfEntryInfo {
protected String name;
protected String value;
public ConfEntryInfo() {
}
public ConfEntryInfo(String key, String value) {
this.name = key;
this.value = value;
}
public String getName() {
return this.name;
}
public String getValue() {
return this.value;
}
}

View File

@ -0,0 +1,66 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Map;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
@XmlRootElement
@XmlAccessorType(XmlAccessType.FIELD)
public class ConfInfo {
protected String path;
protected ArrayList<ConfEntryInfo> property;
public ConfInfo() {
}
public ConfInfo(Job job, Configuration conf) throws IOException {
Path confPath = job.getConfFile();
this.property = new ArrayList<ConfEntryInfo>();
// Read in the configuration file and put it in a key/value table.
FileContext fc = FileContext.getFileContext(confPath.toUri(), conf);
Configuration jobConf = new Configuration(false);
jobConf.addResource(fc.open(confPath));
this.path = confPath.toString();
for (Map.Entry<String, String> entry : jobConf) {
this.property.add(new ConfEntryInfo(entry.getKey(), entry.getValue()));
}
}
public ArrayList<ConfEntryInfo> getProperties() {
return this.property;
}
public String getPath() {
return this.path;
}
}

View File

@ -0,0 +1,54 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.mapreduce.v2.api.records.Counter;
import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
@XmlRootElement(name = "counterGroup")
@XmlAccessorType(XmlAccessType.FIELD)
public class CounterGroupInfo {
protected String counterGroupName;
@XmlElement(name = "counter")
protected ArrayList<CounterInfo> counter;
public CounterGroupInfo() {
}
public CounterGroupInfo(String name, CounterGroup g, CounterGroup mg,
CounterGroup rg) {
this.counterGroupName = name;
this.counter = new ArrayList<CounterInfo>();
for (Counter c : g.getAllCounters().values()) {
Counter mc = mg == null ? null : mg.getCounter(c.getName());
Counter rc = rg == null ? null : rg.getCounter(c.getName());
CounterInfo cinfo = new CounterInfo(c, mc, rc);
this.counter.add(cinfo);
}
}
}

View File

@ -0,0 +1,44 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.mapreduce.v2.api.records.Counter;
@XmlRootElement
@XmlAccessorType(XmlAccessType.FIELD)
public class CounterInfo {
protected String counterName;
protected long totalCounterValue;
protected long mapCounterValue;
protected long reduceCounterValue;
public CounterInfo() {
}
public CounterInfo(Counter counter, Counter mc, Counter rc) {
this.counterName = counter.getName();
this.totalCounterValue = counter.getValue();
this.mapCounterValue = mc == null ? 0 : mc.getValue();
this.reduceCounterValue = rc == null ? 0 : rc.getValue();
}
}

View File

@ -0,0 +1,100 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.ArrayList;
import java.util.Map;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
@XmlRootElement(name = "jobCounters")
@XmlAccessorType(XmlAccessType.FIELD)
public class JobCounterInfo {
@XmlTransient
protected Counters total = null;
@XmlTransient
protected Counters map = null;
@XmlTransient
protected Counters reduce = null;
protected String id;
protected ArrayList<CounterGroupInfo> counterGroups;
public JobCounterInfo() {
}
public JobCounterInfo(AppContext ctx, Job job) {
getCounters(ctx, job);
counterGroups = new ArrayList<CounterGroupInfo>();
this.id = MRApps.toString(job.getID());
int numGroups = 0;
if (total != null) {
for (CounterGroup g : total.getAllCounterGroups().values()) {
if (g != null) {
CounterGroup mg = map == null ? null : map.getCounterGroup(g
.getName());
CounterGroup rg = reduce == null ? null : reduce.getCounterGroup(g
.getName());
++numGroups;
CounterGroupInfo cginfo = new CounterGroupInfo(g.getName(), g, mg, rg);
counterGroups.add(cginfo);
}
}
}
}
private void getCounters(AppContext ctx, Job job) {
total = JobImpl.newCounters();
if (job == null) {
return;
}
map = JobImpl.newCounters();
reduce = JobImpl.newCounters();
// Get all types of counters
Map<TaskId, Task> tasks = job.getTasks();
for (Task t : tasks.values()) {
Counters counters = t.getCounters();
JobImpl.incrAllCounters(total, counters);
switch (t.getType()) {
case MAP:
JobImpl.incrAllCounters(map, counters);
break;
case REDUCE:
JobImpl.incrAllCounters(reduce, counters);
break;
}
}
}
}

View File

@ -0,0 +1,349 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import static org.apache.hadoop.yarn.util.StringHelper.percent;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.util.Times;
@XmlRootElement(name = "job")
@XmlAccessorType(XmlAccessType.FIELD)
public class JobInfo {
// ok for any user to see
protected long startTime;
protected long finishTime;
protected long elapsedTime;
protected String id;
protected String name;
protected String user;
protected String state;
protected int mapsTotal;
protected int mapsCompleted;
protected float mapProgress;
protected int reducesTotal;
protected int reducesCompleted;
protected float reduceProgress;
@XmlTransient
protected String mapProgressPercent;
@XmlTransient
protected String reduceProgressPercent;
// these should only be seen if acls allow
protected int mapsPending;
protected int mapsRunning;
protected int reducesPending;
protected int reducesRunning;
protected boolean uberized;
protected String diagnostics;
protected int newReduceAttempts = 0;
protected int runningReduceAttempts = 0;
protected int failedReduceAttempts = 0;
protected int killedReduceAttempts = 0;
protected int successfulReduceAttempts = 0;
protected int newMapAttempts = 0;
protected int runningMapAttempts = 0;
protected int failedMapAttempts = 0;
protected int killedMapAttempts = 0;
protected int successfulMapAttempts = 0;
protected ArrayList<ConfEntryInfo> acls;
@XmlTransient
protected int numMaps;
@XmlTransient
protected int numReduces;
public JobInfo() {
}
public JobInfo(Job job, Boolean hasAccess) {
this.id = MRApps.toString(job.getID());
JobReport report = job.getReport();
countTasksAndAttempts(job);
this.startTime = report.getStartTime();
this.finishTime = report.getFinishTime();
this.elapsedTime = Times.elapsed(this.startTime, this.finishTime);
if (this.elapsedTime == -1) {
this.elapsedTime = 0;
}
this.name = job.getName().toString();
this.user = job.getUserName();
this.state = job.getState().toString();
this.mapsTotal = job.getTotalMaps();
this.mapsCompleted = job.getCompletedMaps();
this.mapProgress = report.getMapProgress() * 100;
this.mapProgressPercent = percent(report.getMapProgress());
this.reducesTotal = job.getTotalReduces();
this.reducesCompleted = job.getCompletedReduces();
this.reduceProgress = report.getReduceProgress() * 100;
this.reduceProgressPercent = percent(report.getReduceProgress());
this.acls = new ArrayList<ConfEntryInfo>();
if (hasAccess) {
this.uberized = job.isUber();
List<String> diagnostics = job.getDiagnostics();
if (diagnostics != null && !diagnostics.isEmpty()) {
StringBuffer b = new StringBuffer();
for (String diag : diagnostics) {
b.append(diag);
}
this.diagnostics = b.toString();
}
Map<JobACL, AccessControlList> allacls = job.getJobACLs();
if (allacls != null) {
for (Map.Entry<JobACL, AccessControlList> entry : allacls.entrySet()) {
this.acls.add(new ConfEntryInfo(entry.getKey().getAclName(), entry
.getValue().getAclString()));
}
}
}
}
public int getNewReduceAttempts() {
return this.newReduceAttempts;
}
public int getKilledReduceAttempts() {
return this.killedReduceAttempts;
}
public int getFailedReduceAttempts() {
return this.failedReduceAttempts;
}
public int getRunningReduceAttempts() {
return this.runningReduceAttempts;
}
public int getSuccessfulReduceAttempts() {
return this.successfulReduceAttempts;
}
public int getNewMapAttempts() {
return this.newMapAttempts;
}
public int getKilledMapAttempts() {
return this.killedMapAttempts;
}
public ArrayList<ConfEntryInfo> getAcls() {
return acls;
}
public int getFailedMapAttempts() {
return this.failedMapAttempts;
}
public int getRunningMapAttempts() {
return this.runningMapAttempts;
}
public int getSuccessfulMapAttempts() {
return this.successfulMapAttempts;
}
public int getReducesCompleted() {
return this.reducesCompleted;
}
public int getReducesTotal() {
return this.reducesTotal;
}
public int getReducesPending() {
return this.reducesPending;
}
public int getReducesRunning() {
return this.reducesRunning;
}
public int getMapsCompleted() {
return this.mapsCompleted;
}
public int getMapsTotal() {
return this.mapsTotal;
}
public int getMapsPending() {
return this.mapsPending;
}
public int getMapsRunning() {
return this.mapsRunning;
}
public String getState() {
return this.state;
}
public String getUser() {
return this.user;
}
public String getName() {
return this.name;
}
public String getId() {
return this.id;
}
public long getStartTime() {
return this.startTime;
}
public long getElapsedTime() {
return this.elapsedTime;
}
public long getFinishTime() {
return this.finishTime;
}
public boolean isUberized() {
return this.uberized;
}
public String getdiagnostics() {
return this.diagnostics;
}
public float getMapProgress() {
return this.mapProgress;
}
public String getMapProgressPercent() {
return this.mapProgressPercent;
}
public float getReduceProgress() {
return this.reduceProgress;
}
public String getReduceProgressPercent() {
return this.reduceProgressPercent;
}
/**
* Go through a job and update the member variables with counts for
* information to output in the page.
*
* @param job
* the job to get counts for.
*/
private void countTasksAndAttempts(Job job) {
numReduces = 0;
numMaps = 0;
final Map<TaskId, Task> tasks = job.getTasks();
if (tasks == null) {
return;
}
for (Task task : tasks.values()) {
switch (task.getType()) {
case MAP:
// Task counts
switch (task.getState()) {
case RUNNING:
++this.mapsRunning;
break;
case SCHEDULED:
++this.mapsPending;
break;
}
break;
case REDUCE:
// Task counts
switch (task.getState()) {
case RUNNING:
++this.reducesRunning;
break;
case SCHEDULED:
++this.reducesPending;
break;
}
break;
}
// Attempts counts
Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
int newAttempts, running, successful, failed, killed;
for (TaskAttempt attempt : attempts.values()) {
newAttempts = 0;
running = 0;
successful = 0;
failed = 0;
killed = 0;
if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
++newAttempts;
} else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt.getState())) {
++running;
} else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
.getState())) {
++successful;
} else if (TaskAttemptStateUI.FAILED.correspondsTo(attempt.getState())) {
++failed;
} else if (TaskAttemptStateUI.KILLED.correspondsTo(attempt.getState())) {
++killed;
}
switch (task.getType()) {
case MAP:
this.newMapAttempts += newAttempts;
this.runningMapAttempts += running;
this.successfulMapAttempts += successful;
this.failedMapAttempts += failed;
this.killedMapAttempts += killed;
break;
case REDUCE:
this.newReduceAttempts += newAttempts;
this.runningReduceAttempts += running;
this.successfulReduceAttempts += successful;
this.failedReduceAttempts += failed;
this.killedReduceAttempts += killed;
break;
}
}
}
}
}

View File

@ -0,0 +1,63 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
@XmlRootElement(name = "JobTaskAttemptCounters")
@XmlAccessorType(XmlAccessType.FIELD)
public class JobTaskAttemptCounterInfo {
@XmlTransient
protected Counters total = null;
protected String id;
protected ArrayList<TaskCounterGroupInfo> taskCounterGroups;
public JobTaskAttemptCounterInfo() {
}
public JobTaskAttemptCounterInfo(TaskAttempt taskattempt) {
long value = 0;
this.id = MRApps.toString(taskattempt.getID());
total = taskattempt.getCounters();
taskCounterGroups = new ArrayList<TaskCounterGroupInfo>();
if (total != null) {
for (CounterGroup g : total.getAllCounterGroups().values()) {
if (g != null) {
TaskCounterGroupInfo cginfo = new TaskCounterGroupInfo(g.getName(), g);
if (cginfo != null) {
taskCounterGroups.add(cginfo);
}
}
}
}
}
}

View File

@ -0,0 +1,59 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
@XmlRootElement(name = "jobTaskCounters")
@XmlAccessorType(XmlAccessType.FIELD)
public class JobTaskCounterInfo {
@XmlTransient
protected Counters total = null;
protected String id;
protected ArrayList<TaskCounterGroupInfo> taskCounterGroups;
public JobTaskCounterInfo() {
}
public JobTaskCounterInfo(Task task) {
total = task.getCounters();
this.id = MRApps.toString(task.getID());
taskCounterGroups = new ArrayList<TaskCounterGroupInfo>();
if (total != null) {
for (CounterGroup g : total.getAllCounterGroups().values()) {
if (g != null) {
TaskCounterGroupInfo cginfo = new TaskCounterGroupInfo(g.getName(), g);
taskCounterGroups.add(cginfo);
}
}
}
}
}

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by joblicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "jobs")
@XmlAccessorType(XmlAccessType.FIELD)
public class JobsInfo {
protected ArrayList<JobInfo> job = new ArrayList<JobInfo>();
public JobsInfo() {
} // JAXB needs this
public void add(JobInfo jobInfo) {
job.add(jobInfo);
}
public ArrayList<JobInfo> getJobs() {
return job;
}
}

View File

@ -0,0 +1,83 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.yarn.util.Times;
@XmlRootElement(name = "taskAttempt")
@XmlAccessorType(XmlAccessType.FIELD)
public class ReduceTaskAttemptInfo extends TaskAttemptInfo {
protected long shuffleFinishTime;
protected long mergeFinishTime;
protected long elapsedShuffleTime;
protected long elapsedMergeTime;
protected long elapsedReduceTime;
public ReduceTaskAttemptInfo() {
}
public ReduceTaskAttemptInfo(TaskAttempt ta, TaskType type) {
super(ta, type, false);
this.shuffleFinishTime = ta.getShuffleFinishTime();
this.mergeFinishTime = ta.getSortFinishTime();
this.elapsedShuffleTime = Times.elapsed(this.startTime,
this.shuffleFinishTime, false);
if (this.elapsedShuffleTime == -1) {
this.elapsedShuffleTime = 0;
}
this.elapsedMergeTime = Times.elapsed(this.shuffleFinishTime,
this.mergeFinishTime, false);
if (this.elapsedMergeTime == -1) {
this.elapsedMergeTime = 0;
}
this.elapsedReduceTime = Times.elapsed(this.mergeFinishTime,
this.finishTime, false);
if (this.elapsedReduceTime == -1) {
this.elapsedReduceTime = 0;
}
}
public long getShuffleFinishTime() {
return this.shuffleFinishTime;
}
public long getMergeFinishTime() {
return this.mergeFinishTime;
}
public long getElapsedShuffleTime() {
return this.elapsedShuffleTime;
}
public long getElapsedMergeTime() {
return this.elapsedMergeTime;
}
public long getElapsedReduceTime() {
return this.elapsedReduceTime;
}
}

View File

@ -0,0 +1,133 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.List;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlSeeAlso;
import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Times;
@XmlRootElement(name = "taskAttempt")
@XmlSeeAlso({ ReduceTaskAttemptInfo.class })
@XmlAccessorType(XmlAccessType.FIELD)
public class TaskAttemptInfo {
protected long startTime;
protected long finishTime;
protected long elapsedTime;
protected float progress;
protected String id;
protected String rack;
protected String state;
protected String nodeHttpAddress;
protected String diagnostics;
protected String type;
protected String assignedContainerId;
@XmlTransient
protected ContainerId assignedContainer;
public TaskAttemptInfo() {
}
public TaskAttemptInfo(TaskAttempt ta, Boolean isRunning) {
this(ta, TaskType.MAP, isRunning);
}
public TaskAttemptInfo(TaskAttempt ta, TaskType type, Boolean isRunning) {
this.type = type.toString();
this.id = MRApps.toString(ta.getID());
this.nodeHttpAddress = ta.getNodeHttpAddress();
this.startTime = ta.getLaunchTime();
this.finishTime = ta.getFinishTime();
this.assignedContainerId = ConverterUtils.toString(ta
.getAssignedContainerID());
this.assignedContainer = ta.getAssignedContainerID();
this.progress = ta.getProgress() * 100;
this.state = ta.getState().toString();
this.elapsedTime = Times
.elapsed(this.startTime, this.finishTime, isRunning);
if (this.elapsedTime == -1) {
this.elapsedTime = 0;
}
List<String> diagnostics = ta.getDiagnostics();
if (diagnostics != null && !diagnostics.isEmpty()) {
StringBuffer b = new StringBuffer();
for (String diag : diagnostics) {
b.append(diag);
}
this.diagnostics = b.toString();
}
this.rack = ta.getNodeRackName();
}
public String getAssignedContainerIdStr() {
return this.assignedContainerId;
}
public ContainerId getAssignedContainerId() {
return this.assignedContainer;
}
public String getState() {
return this.state;
}
public String getId() {
return this.id;
}
public long getStartTime() {
return this.startTime;
}
public long getFinishTime() {
return this.finishTime;
}
public float getProgress() {
return this.progress;
}
public long getElapsedTime() {
return this.elapsedTime;
}
public String getNode() {
return this.nodeHttpAddress;
}
public String getRack() {
return this.rack;
}
public String getNote() {
return this.diagnostics;
}
}

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by taskattemptlicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "taskattempts")
@XmlAccessorType(XmlAccessType.FIELD)
public class TaskAttemptsInfo {
protected ArrayList<TaskAttemptInfo> taskattempt = new ArrayList<TaskAttemptInfo>();
public TaskAttemptsInfo() {
} // JAXB needs this
public void add(TaskAttemptInfo taskattemptInfo) {
taskattempt.add(taskattemptInfo);
}
public ArrayList<TaskAttemptInfo> getTaskAttempts() {
return taskattempt;
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.mapreduce.v2.api.records.Counter;
import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
@XmlRootElement
@XmlAccessorType(XmlAccessType.FIELD)
public class TaskCounterGroupInfo {
protected String counterGroupName;
protected ArrayList<TaskCounterInfo> counter;
public TaskCounterGroupInfo() {
}
public TaskCounterGroupInfo(String name, CounterGroup g) {
this.counterGroupName = name;
this.counter = new ArrayList<TaskCounterInfo>();
for (Counter c : g.getAllCounters().values()) {
TaskCounterInfo cinfo = new TaskCounterInfo(c.getName(), c.getValue());
this.counter.add(cinfo);
}
}
}

View File

@ -0,0 +1,46 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "counter")
@XmlAccessorType(XmlAccessType.FIELD)
public class TaskCounterInfo {
protected String name;
protected long value;
public TaskCounterInfo() {
}
public TaskCounterInfo(String name, long value) {
this.name = name;
this.value = value;
}
public String getName() {
return name;
}
public long getValue() {
return value;
}
}

View File

@ -0,0 +1,122 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.util.Times;
@XmlRootElement(name = "task")
@XmlAccessorType(XmlAccessType.FIELD)
public class TaskInfo {
protected long startTime;
protected long finishTime;
protected long elapsedTime;
protected float progress;
protected String id;
protected String state;
protected String type;
protected String successfulAttempt;
@XmlTransient
int taskNum;
@XmlTransient
TaskAttempt successful;
public TaskInfo() {
}
public TaskInfo(Task task) {
TaskType ttype = task.getType();
this.type = ttype.toString();
TaskReport report = task.getReport();
this.startTime = report.getStartTime();
this.finishTime = report.getFinishTime();
this.elapsedTime = Times.elapsed(this.startTime, this.finishTime, false);
if (this.elapsedTime == -1) {
this.elapsedTime = 0;
}
this.state = report.getTaskState().toString();
this.progress = report.getProgress() * 100;
this.id = MRApps.toString(task.getID());
this.taskNum = task.getID().getId();
this.successful = getSuccessfulAttempt(task);
if (successful != null) {
this.successfulAttempt = MRApps.toString(successful.getID());
} else {
this.successfulAttempt = "";
}
}
public float getProgress() {
return this.progress;
}
public String getState() {
return this.state;
}
public String getId() {
return this.id;
}
public int getTaskNum() {
return this.taskNum;
}
public long getStartTime() {
return this.startTime;
}
public long getFinishTime() {
return this.finishTime;
}
public long getElapsedTime() {
return this.elapsedTime;
}
public String getSuccessfulAttempt() {
return this.successfulAttempt;
}
public TaskAttempt getSuccessful() {
return this.successful;
}
private TaskAttempt getSuccessfulAttempt(Task task) {
for (TaskAttempt attempt : task.getAttempts().values()) {
if (attempt.getState() == TaskAttemptState.SUCCEEDED) {
return attempt;
}
}
return null;
}
}

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by tasklicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "tasks")
@XmlAccessorType(XmlAccessType.FIELD)
public class TasksInfo {
protected ArrayList<TaskInfo> task = new ArrayList<TaskInfo>();
public TasksInfo() {
} // JAXB needs this
public void add(TaskInfo taskInfo) {
task.add(taskInfo);
}
public ArrayList<TaskInfo> getTasks() {
return task;
}
}

View File

@ -183,6 +183,7 @@ public class TestMRClientService {
Assert.assertEquals(1, amInfo.getContainerId().getApplicationAttemptId()
.getAttemptId());
Assert.assertTrue(amInfo.getStartTime() > 0);
Assert.assertEquals(false, jr.isUber());
}
private void verifyTaskAttemptReport(TaskAttemptReport tar) {

View File

@ -118,7 +118,7 @@ public class TestRMContainerAllocator {
Job mockJob = mock(Job.class);
when(mockJob.getReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
0, 0, 0, 0, 0, 0, "jobfile", null));
0, 0, 0, 0, 0, 0, "jobfile", null, false));
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
appAttemptId, mockJob);
@ -195,7 +195,7 @@ public class TestRMContainerAllocator {
Job mockJob = mock(Job.class);
when(mockJob.getReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
0, 0, 0, 0, 0, 0, "jobfile", null));
0, 0, 0, 0, 0, 0, "jobfile", null, false));
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
appAttemptId, mockJob);
@ -261,7 +261,7 @@ public class TestRMContainerAllocator {
Job mockJob = mock(Job.class);
when(mockJob.getReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
0, 0, 0, 0, 0, 0, "jobfile", null));
0, 0, 0, 0, 0, 0, "jobfile", null, false));
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
appAttemptId, mockJob);
@ -375,7 +375,7 @@ public class TestRMContainerAllocator {
public JobReport getReport() {
return MRBuilderUtils.newJobReport(this.jobId, "job", "user",
JobState.RUNNING, 0, 0, 0, this.setupProgress, this.mapProgress,
this.reduceProgress, this.cleanupProgress, "jobfile", null);
this.reduceProgress, this.cleanupProgress, "jobfile", null, false);
}
}
@ -511,7 +511,7 @@ public class TestRMContainerAllocator {
Job mockJob = mock(Job.class);
when(mockJob.getReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
0, 0, 0, 0, 0, 0, "jobfile", null));
0, 0, 0, 0, 0, 0, "jobfile", null, false));
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
appAttemptId, mockJob);
@ -610,7 +610,7 @@ public class TestRMContainerAllocator {
Job mockJob = mock(Job.class);
when(mockJob.getReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
0, 0, 0, 0, 0, 0, "jobfile", null));
0, 0, 0, 0, 0, 0, "jobfile", null, false));
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
appAttemptId, mockJob);

View File

@ -754,8 +754,6 @@ public class TestRuntimeEstimators {
}
class MyAppContext implements AppContext {
// I'll be making Avro objects by hand. Please don't do that very often.
private final ApplicationAttemptId myAppAttemptID;
private final ApplicationId myApplicationID;
private final JobId myJobID;

View File

@ -1,153 +0,0 @@
@namespace("org.apache.hadoop.mapreduce.v2.api")
protocol MRClientProtocol {
import idl "./yarn/yarn-api/src/main/avro/yarn-types.genavro";
enum TaskType {
MAP,
REDUCE
}
record JobID {
org.apache.hadoop.yarn.ApplicationID appID;
int id;
}
record TaskID {
JobID jobID;
TaskType taskType;
int id;
}
record TaskAttemptID {
TaskID taskID;
int id;
}
enum TaskState {
NEW,
SCHEDULED,
RUNNING,
SUCCEEDED,
FAILED,
KILL_WAIT,
KILLED
}
enum Phase {
STARTING,
MAP,
SHUFFLE,
SORT,
REDUCE,
CLEANUP
}
record Counter {
string name;
string displayName;
long value;
}
record CounterGroup {
string name;
string displayname;
map<Counter> counters;
}
record Counters {
map<CounterGroup> groups;
}
record TaskReport {
TaskID id;
TaskState state;
float progress;
long startTime;
long finishTime;
Counters counters;
array<TaskAttemptID> runningAttempts;
union{TaskAttemptID, null} successfulAttempt;
array<string> diagnostics;
}
enum TaskAttemptState {
NEW,
UNASSIGNED,
ASSIGNED,
RUNNING,
COMMIT_PENDING,
SUCCESS_CONTAINER_CLEANUP,
SUCCEEDED,
FAIL_CONTAINER_CLEANUP,
FAIL_TASK_CLEANUP,
FAILED,
KILL_CONTAINER_CLEANUP,
KILL_TASK_CLEANUP,
KILLED
}
record TaskAttemptReport {
TaskAttemptID id;
TaskAttemptState state;
float progress;
long startTime;
long finishTime;
Counters counters;
string diagnosticInfo;
string stateString;
Phase phase;
}
enum JobState {
NEW,
INITED,
RUNNING,
SUCCEEDED,
FAILED,
KILL_WAIT,
KILLED,
ERROR
}
record JobReport {
JobID id;
JobState state;
float mapProgress;
float reduceProgress;
float cleanupProgress;
float setupProgress;
long startTime;
long finishTime;
}
enum TaskAttemptCompletionEventStatus {
FAILED,
KILLED,
SUCCEEDED,
OBSOLETE,
TIPFAILED
}
record TaskAttemptCompletionEvent {
TaskAttemptID attemptId;
TaskAttemptCompletionEventStatus status;
string mapOutputServerAddress;
int attemptRunTime;
int eventId;
}
JobReport getJobReport(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
TaskReport getTaskReport(TaskID taskID) throws org.apache.hadoop.yarn.YarnRemoteException;
TaskAttemptReport getTaskAttemptReport(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
Counters getCounters(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
array<TaskAttemptCompletionEvent> getTaskAttemptCompletionEvents(JobID jobID, int fromEventId, int maxEvents) throws org.apache.hadoop.yarn.YarnRemoteException;
array<TaskReport> getTaskReports(JobID jobID, TaskType taskType) throws org.apache.hadoop.yarn.YarnRemoteException;
array<string> getDiagnostics(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
void killJob(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
void killTask(TaskID taskID) throws org.apache.hadoop.yarn.YarnRemoteException;
void killTaskAttempt(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
void failTaskAttempt(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
}

View File

@ -288,7 +288,7 @@ public class TypeConverter {
.getMapProgress(), jobreport.getReduceProgress(), jobreport
.getCleanupProgress(), fromYarn(jobreport.getJobState()),
jobPriority, jobreport.getUser(), jobreport.getJobName(), jobreport
.getJobFile(), trackingUrl);
.getJobFile(), trackingUrl, jobreport.isUber());
jobStatus.setFailureInfo(jobreport.getDiagnostics());
return jobStatus;
}
@ -421,7 +421,7 @@ public class TypeConverter {
TypeConverter.fromYarn(application.getYarnApplicationState()),
org.apache.hadoop.mapreduce.JobPriority.NORMAL,
application.getUser(), application.getName(),
application.getQueue(), jobFile, trackingUrl
application.getQueue(), jobFile, trackingUrl, false
);
jobStatus.setSchedulingInfo(trackingUrl); // Set AM tracking url
jobStatus.setStartTime(application.getStartTime());

View File

@ -36,6 +36,7 @@ public interface JobReport {
public abstract String getDiagnostics();
public abstract String getJobFile();
public abstract List<AMInfo> getAMInfos();
public abstract boolean isUber();
public abstract void setJobId(JobId jobId);
public abstract void setJobState(JobState jobState);
@ -52,4 +53,5 @@ public interface JobReport {
public abstract void setDiagnostics(String diagnostics);
public abstract void setJobFile(String jobFile);
public abstract void setAMInfos(List<AMInfo> amInfos);
public abstract void setIsUber(boolean isUber);
}

View File

@ -332,4 +332,16 @@ public class JobReportPBImpl extends ProtoBase<JobReportProto> implements
private JobState convertFromProtoFormat(JobStateProto e) {
return MRProtoUtils.convertFromProtoFormat(e);
}
@Override
public synchronized boolean isUber() {
JobReportProtoOrBuilder p = viaProto ? proto : builder;
return p.getIsUber();
}
@Override
public synchronized void setIsUber(boolean isUber) {
maybeInitBuilder();
builder.setIsUber(isUber);
}
}

View File

@ -60,7 +60,8 @@ public class MRBuilderUtils {
public static JobReport newJobReport(JobId jobId, String jobName,
String userName, JobState state, long submitTime, long startTime, long finishTime,
float setupProgress, float mapProgress, float reduceProgress,
float cleanupProgress, String jobFile, List<AMInfo> amInfos) {
float cleanupProgress, String jobFile, List<AMInfo> amInfos,
boolean isUber) {
JobReport report = Records.newRecord(JobReport.class);
report.setJobId(jobId);
report.setJobName(jobName);
@ -75,6 +76,7 @@ public class MRBuilderUtils {
report.setReduceProgress(reduceProgress);
report.setJobFile(jobFile);
report.setAMInfos(amInfos);
report.setIsUber(isUber);
return report;
}

View File

@ -152,6 +152,7 @@ message JobReportProto {
optional string jobFile = 13;
repeated AMInfoProto am_infos = 14;
optional int64 submit_time = 15;
optional bool is_uber = 16 [default = false];
}
message AMInfoProto {

View File

@ -109,7 +109,14 @@ class JobQueueClient extends Configured implements Tool {
return exitcode;
}
// format and print information about the passed in job queue.
void printJobQueueInfo(JobQueueInfo jobQueueInfo, Writer writer)
throws IOException {
printJobQueueInfo(jobQueueInfo, writer, "");
}
// format and print information about the passed in job queue.
@SuppressWarnings("deprecation")
void printJobQueueInfo(JobQueueInfo jobQueueInfo, Writer writer,
String prefix) throws IOException {
if (jobQueueInfo == null) {
@ -136,7 +143,7 @@ class JobQueueClient extends Configured implements Tool {
private void displayQueueList() throws IOException {
JobQueueInfo[] rootQueues = jc.getRootQueues();
for (JobQueueInfo queue : rootQueues) {
printJobQueueInfo(queue, new PrintWriter(System.out), "");
printJobQueueInfo(queue, new PrintWriter(System.out));
}
}
@ -174,7 +181,7 @@ class JobQueueClient extends Configured implements Tool {
System.out.println("Queue \"" + queue + "\" does not exist.");
return;
}
printJobQueueInfo(jobQueueInfo, new PrintWriter(System.out), "");
printJobQueueInfo(jobQueueInfo, new PrintWriter(System.out));
if (showJobs && (jobQueueInfo.getChildren() == null ||
jobQueueInfo.getChildren().size() == 0)) {
JobStatus[] jobs = jc.getJobsFromQueue(queue);

View File

@ -97,7 +97,7 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
String user, String jobName,
String jobFile, String trackingUrl) {
this(jobid, mapProgress, reduceProgress, cleanupProgress, runState,
JobPriority.NORMAL, user, jobName, jobFile, trackingUrl);
JobPriority.NORMAL, user, jobName, jobFile, trackingUrl);
}
/**
@ -135,7 +135,8 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
String user, String jobName, String jobFile,
String trackingUrl) {
this(jobid, 0.0f, mapProgress, reduceProgress,
cleanupProgress, runState, jp, user, jobName, jobFile, trackingUrl);
cleanupProgress, runState, jp, user, jobName, jobFile,
trackingUrl);
}
/**
@ -157,10 +158,57 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
int runState, JobPriority jp, String user, String jobName,
String jobFile, String trackingUrl) {
this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
runState, jp,
user, jobName, "default", jobFile, trackingUrl);
runState, jp, user, jobName, "default", jobFile, trackingUrl);
}
/**
* Create a job status object for a given jobid.
* @param jobid The jobid of the job
* @param setupProgress The progress made on the setup
* @param mapProgress The progress made on the maps
* @param reduceProgress The progress made on the reduces
* @param cleanupProgress The progress made on the cleanup
* @param runState The current state of the job
* @param jp Priority of the job.
* @param user userid of the person who submitted the job.
* @param jobName user-specified job name.
* @param jobFile job configuration file.
* @param trackingUrl link to the web-ui for details of the job.
* @param isUber Whether job running in uber mode
*/
public JobStatus(JobID jobid, float setupProgress, float mapProgress,
float reduceProgress, float cleanupProgress,
int runState, JobPriority jp, String user, String jobName,
String jobFile, String trackingUrl, boolean isUber) {
this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
runState, jp, user, jobName, "default", jobFile, trackingUrl, isUber);
}
/**
* Create a job status object for a given jobid.
* @param jobid The jobid of the job
* @param setupProgress The progress made on the setup
* @param mapProgress The progress made on the maps
* @param reduceProgress The progress made on the reduces
* @param cleanupProgress The progress made on the cleanup
* @param runState The current state of the job
* @param jp Priority of the job.
* @param user userid of the person who submitted the job.
* @param jobName user-specified job name.
* @param queue job queue name.
* @param jobFile job configuration file.
* @param trackingUrl link to the web-ui for details of the job.
*/
public JobStatus(JobID jobid, float setupProgress, float mapProgress,
float reduceProgress, float cleanupProgress,
int runState, JobPriority jp,
String user, String jobName, String queue,
String jobFile, String trackingUrl) {
this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
runState, jp,
user, jobName, queue, jobFile, trackingUrl, false);
}
/**
* Create a job status object for a given jobid.
* @param jobid The jobid of the job
@ -175,25 +223,25 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
* @param queue job queue name.
* @param jobFile job configuration file.
* @param trackingUrl link to the web-ui for details of the job.
* @param isUber Whether job running in uber mode
*/
public JobStatus(JobID jobid, float setupProgress, float mapProgress,
float reduceProgress, float cleanupProgress,
int runState, JobPriority jp,
String user, String jobName, String queue,
String jobFile, String trackingUrl) {
String jobFile, String trackingUrl, boolean isUber) {
super(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
getEnum(runState), org.apache.hadoop.mapreduce.JobPriority.valueOf(jp.name()),
user, jobName, queue, jobFile, trackingUrl);
user, jobName, queue, jobFile, trackingUrl, isUber);
}
public static JobStatus downgrade(org.apache.hadoop.mapreduce.JobStatus stat){
JobStatus old = new JobStatus(JobID.downgrade(stat.getJobID()),
stat.getSetupProgress(), stat.getMapProgress(), stat.getReduceProgress(),
stat.getCleanupProgress(), stat.getState().getValue(),
JobPriority.valueOf(stat.getPriority().name()),
stat.getUsername(), stat.getJobName(), stat.getJobFile(),
stat.getTrackingUrl());
stat.getTrackingUrl(), stat.isUber());
old.setStartTime(stat.getStartTime());
old.setFinishTime(stat.getFinishTime());
old.setSchedulingInfo(stat.getSchedulingInfo());

View File

@ -467,6 +467,7 @@ public class Job extends JobContextImpl implements JobContext {
sb.append("Job File: ").append(status.getJobFile()).append("\n");
sb.append("Job Tracking URL : ").append(status.getTrackingUrl());
sb.append("\n");
sb.append("Uber job : ").append(status.isUber()).append("\n");
sb.append("map() completion: ");
sb.append(status.getMapProgress()).append("\n");
sb.append("reduce() completion: ");
@ -1268,12 +1269,20 @@ public class Job extends JobContextImpl implements JobContext {
Job.getProgressPollInterval(clientConf);
/* make sure to report full progress after the job is done */
boolean reportedAfterCompletion = false;
boolean reportedUberMode = false;
while (!isComplete() || !reportedAfterCompletion) {
if (isComplete()) {
reportedAfterCompletion = true;
} else {
Thread.sleep(progMonitorPollIntervalMillis);
}
if (status.getState() == JobStatus.State.PREP) {
continue;
}
if (!reportedUberMode) {
reportedUberMode = true;
LOG.info("Job " + jobId + " running in uber mode : " + isUber());
}
String report =
(" map " + StringUtils.formatPercent(mapProgress(), 0)+
" reduce " +
@ -1497,4 +1506,10 @@ public class Job extends JobContextImpl implements JobContext {
conf.set(Job.OUTPUT_FILTER, newValue.toString());
}
public boolean isUber() throws IOException, InterruptedException {
ensureState(JobState.RUNNING);
updateStatus();
return status.isUber();
}
}

View File

@ -97,7 +97,7 @@ public class JobStatus implements Writable, Cloneable {
private int usedMem;
private int reservedMem;
private int neededMem;
private boolean isUber;
/**
*/
@ -115,17 +115,17 @@ public class JobStatus implements Writable, Cloneable {
* @param jp Priority of the job.
* @param user userid of the person who submitted the job.
* @param jobName user-specified job name.
* @param jobFile job configuration file.
* @param jobFile job configuration file.
* @param trackingUrl link to the web-ui for details of the job.
*/
public JobStatus(JobID jobid, float setupProgress, float mapProgress,
float reduceProgress, float cleanupProgress,
float reduceProgress, float cleanupProgress,
State runState, JobPriority jp, String user, String jobName,
String jobFile, String trackingUrl) {
this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
runState, jp, user, jobName, "default", jobFile, trackingUrl);
runState, jp, user, jobName, "default", jobFile, trackingUrl, false);
}
/**
* Create a job status object for a given jobid.
* @param jobid The jobid of the job
@ -138,14 +138,39 @@ public class JobStatus implements Writable, Cloneable {
* @param user userid of the person who submitted the job.
* @param jobName user-specified job name.
* @param queue queue name
* @param jobFile job configuration file.
* @param jobFile job configuration file.
* @param trackingUrl link to the web-ui for details of the job.
*/
public JobStatus(JobID jobid, float setupProgress, float mapProgress,
float reduceProgress, float cleanupProgress,
State runState, JobPriority jp,
String user, String jobName, String queue,
float reduceProgress, float cleanupProgress,
State runState, JobPriority jp,
String user, String jobName, String queue,
String jobFile, String trackingUrl) {
this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
runState, jp, user, jobName, queue, jobFile, trackingUrl, false);
}
/**
* Create a job status object for a given jobid.
* @param jobid The jobid of the job
* @param setupProgress The progress made on the setup
* @param mapProgress The progress made on the maps
* @param reduceProgress The progress made on the reduces
* @param cleanupProgress The progress made on the cleanup
* @param runState The current state of the job
* @param jp Priority of the job.
* @param user userid of the person who submitted the job.
* @param jobName user-specified job name.
* @param queue queue name
* @param jobFile job configuration file.
* @param trackingUrl link to the web-ui for details of the job.
* @param isUber Whether job running in uber mode
*/
public JobStatus(JobID jobid, float setupProgress, float mapProgress,
float reduceProgress, float cleanupProgress,
State runState, JobPriority jp,
String user, String jobName, String queue,
String jobFile, String trackingUrl, boolean isUber) {
this.jobid = jobid;
this.setupProgress = setupProgress;
this.mapProgress = mapProgress;
@ -161,8 +186,9 @@ public class JobStatus implements Writable, Cloneable {
this.jobName = jobName;
this.jobFile = jobFile;
this.trackingUrl = trackingUrl;
this.isUber = isUber;
}
/**
* Sets the map progress of this job
@ -411,6 +437,7 @@ public class JobStatus implements Writable, Cloneable {
Text.writeString(out, jobName);
Text.writeString(out, trackingUrl);
Text.writeString(out, jobFile);
out.writeBoolean(isUber);
// Serialize the job's ACLs
out.writeInt(jobACLs.size());
@ -438,6 +465,7 @@ public class JobStatus implements Writable, Cloneable {
this.jobName = Text.readString(in);
this.trackingUrl = Text.readString(in);
this.jobFile = Text.readString(in);
this.isUber = in.readBoolean();
// De-serialize the job's ACLs
int numACLs = in.readInt();
@ -562,9 +590,26 @@ public class JobStatus implements Writable, Cloneable {
this.neededMem = n;
}
/**
* Whether job running in uber mode
* @return job in uber-mode
*/
public synchronized boolean isUber() {
return isUber;
}
/**
* Set uber-mode flag
* @param isUber Whether job running in uber-mode
*/
public synchronized void setUber(boolean isUber) {
this.isUber = isUber;
}
public String toString() {
StringBuffer buffer = new StringBuffer();
buffer.append("job-id : " + jobid);
buffer.append("uber-mode : " + isUber);
buffer.append("map-progress : " + mapProgress);
buffer.append("reduce-progress : " + reduceProgress);
buffer.append("cleanup-progress : " + cleanupProgress);

View File

@ -150,6 +150,10 @@ public interface MRJobConfig {
public static final String NUM_REDUCE_PROFILES = "mapreduce.task.profile.reduces";
public static final String TASK_MAP_PROFILE_PARAMS = "mapreduce.task.profile.map.params";
public static final String TASK_REDUCE_PROFILE_PARAMS = "mapreduce.task.profile.reduce.params";
public static final String TASK_TIMEOUT = "mapreduce.task.timeout";
public static final String TASK_ID = "mapreduce.task.id";
@ -298,12 +302,6 @@ public interface MRJobConfig {
"mapreduce.job.ubertask.maxreduces";
public static final String JOB_UBERTASK_MAXBYTES =
"mapreduce.job.ubertask.maxbytes";
public static final String UBERTASK_JAVA_OPTS =
"mapreduce.ubertask.child.java.opts"; // or mapreduce.uber.java.opts?
public static final String UBERTASK_ULIMIT =
"mapreduce.ubertask.child.ulimit"; // or mapreduce.uber.ulimit?
public static final String UBERTASK_ENV =
"mapreduce.ubertask.child.env"; // or mapreduce.uber.env?
public static final String MR_PREFIX = "yarn.app.mapreduce.";

View File

@ -76,6 +76,7 @@ public abstract class AbstractCounters<C extends Counter,
TaskCounter.class.getName());
legacyMap.put("org.apache.hadoop.mapred.JobInProgress$Counter",
JobCounter.class.getName());
legacyMap.put("FileSystemCounter", FileSystemCounter.class.getName());
}
private final Limits limits = new Limits();

View File

@ -353,7 +353,7 @@ public class JobHistoryParser {
* The class where job information is aggregated into after parsing
*/
public static class JobInfo {
String errorInfo = "None";
String errorInfo = "";
long submitTime;
long finishTime;
JobID jobid;

View File

@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.text.ParseException;
import org.apache.hadoop.mapred.Counters.Counter;
import org.apache.hadoop.mapreduce.FileSystemCounter;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.junit.Test;
@ -102,6 +104,7 @@ public class TestCounters {
Counters counters = new Counters();
counters.incrCounter(TaskCounter.MAP_INPUT_RECORDS, 1);
counters.incrCounter(JobCounter.DATA_LOCAL_MAPS, 1);
counters.findCounter("file", FileSystemCounter.BYTES_READ).increment(1);
assertEquals("New name", 1, counters.findCounter(
TaskCounter.class.getName(), "MAP_INPUT_RECORDS").getValue());
@ -114,6 +117,14 @@ public class TestCounters {
assertEquals("Legacy name", 1, counters.findCounter(
"org.apache.hadoop.mapred.JobInProgress$Counter",
"DATA_LOCAL_MAPS").getValue());
assertEquals("New name", 1, counters.findCounter(
FileSystemCounter.class.getName(), "FILE_BYTES_READ").getValue());
assertEquals("New name and method", 1, counters.findCounter("file",
FileSystemCounter.BYTES_READ).getValue());
assertEquals("Legacy name", 1, counters.findCounter(
"FileSystemCounter",
"FILE_BYTES_READ").getValue());
}
public static void main(String[] args) throws IOException {

View File

@ -45,7 +45,7 @@ public class TestJobQueueClient {
ByteArrayOutputStream bbos = new ByteArrayOutputStream();
PrintWriter writer = new PrintWriter(bbos);
queueClient.printJobQueueInfo(parent, writer, "");
queueClient.printJobQueueInfo(parent, writer);
Assert.assertTrue("printJobQueueInfo did not print grandchild's name",
bbos.toString().contains("GrandChildQueue"));

View File

@ -63,17 +63,20 @@ public class TestJobMonitorAndPrint extends TestCase {
when(cluster.getConf()).thenReturn(conf);
when(cluster.getClient()).thenReturn(clientProtocol);
JobStatus jobStatus = new JobStatus(new JobID("job_000", 1), 0f, 0f, 0f, 0f,
State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname", "tmp-jobfile", "tmp-url");
State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname",
"tmp-jobfile", "tmp-url");
job = Job.getInstance(cluster, jobStatus, conf);
job = spy(job);
}
@Test
public void testJobMonitorAndPrint() throws Exception {
JobStatus jobStatus_1 = new JobStatus(new JobID("job_000", 1), 1f, 0.1f, 0.1f, 0f,
State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname", "tmp-jobfile", "tmp-url");
JobStatus jobStatus_2 = new JobStatus(new JobID("job_000", 1), 1f, 1f, 1f, 1f,
State.SUCCEEDED, JobPriority.HIGH, "tmp-user", "tmp-jobname", "tmp-jobfile", "tmp-url");
JobStatus jobStatus_1 = new JobStatus(new JobID("job_000", 1), 1f, 0.1f,
0.1f, 0f, State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname",
"tmp-queue", "tmp-jobfile", "tmp-url", true);
JobStatus jobStatus_2 = new JobStatus(new JobID("job_000", 1), 1f, 1f,
1f, 1f, State.SUCCEEDED, JobPriority.HIGH, "tmp-user", "tmp-jobname",
"tmp-queue", "tmp-jobfile", "tmp-url", true);
doAnswer(
new Answer<TaskCompletionEvent[]>() {
@ -102,15 +105,21 @@ public class TestJobMonitorAndPrint extends TestCase {
String line;
boolean foundHundred = false;
boolean foundComplete = false;
String match_1 = "map 100% reduce 100%";
String match_2 = "completed successfully";
boolean foundUber = false;
String match_1 = "uber mode : true";
String match_2 = "map 100% reduce 100%";
String match_3 = "completed successfully";
while ((line = r.readLine()) != null) {
foundHundred = line.contains(match_1);
if (line.contains(match_1)) {
foundUber = true;
}
foundHundred = line.contains(match_2);
if (foundHundred)
break;
}
line = r.readLine();
foundComplete = line.contains(match_2);
foundComplete = line.contains(match_3);
assertTrue(foundUber);
assertTrue(foundHundred);
assertTrue(foundComplete);
}

View File

@ -107,6 +107,7 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
report.setTrackingUrl(JobHistoryUtils.getHistoryUrl(conf, TypeConverter
.toYarn(TypeConverter.fromYarn(jobId)).getAppId()));
report.setAMInfos(getAMInfos());
report.setIsUber(isUber());
}
@Override

View File

@ -27,12 +27,11 @@ import java.security.PrivilegedExceptionAction;
import java.util.Arrays;
import java.util.Collection;
import org.apache.hadoop.ipc.Server;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
@ -79,14 +78,14 @@ import org.apache.hadoop.yarn.webapp.WebApp;
import org.apache.hadoop.yarn.webapp.WebApps;
/**
* This module is responsible for talking to the
* This module is responsible for talking to the
* JobClient (user facing).
*
*/
public class HistoryClientService extends AbstractService {
private static final Log LOG = LogFactory.getLog(HistoryClientService.class);
private MRClientProtocol protocolHandler;
private Server server;
private WebApp webApp;
@ -118,22 +117,22 @@ public class HistoryClientService extends AbstractService {
server =
rpc.getServer(MRClientProtocol.class, protocolHandler, address,
conf, null,
conf.getInt(JHAdminConfig.MR_HISTORY_CLIENT_THREAD_COUNT,
conf.getInt(JHAdminConfig.MR_HISTORY_CLIENT_THREAD_COUNT,
JHAdminConfig.DEFAULT_MR_HISTORY_CLIENT_THREAD_COUNT));
// Enable service authorization?
if (conf.getBoolean(
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
false)) {
server.refreshServiceAcl(conf, new MRAMPolicyProvider());
}
server.start();
this.bindAddress =
NetUtils.createSocketAddr(hostNameResolved.getHostAddress()
+ ":" + server.getPort());
LOG.info("Instantiated MRClientService at " + this.bindAddress);
super.start();
}
@ -141,7 +140,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).with(conf).at(bindAddress).start(webApp);
WebApps.$for("jobhistory", HistoryClientService.class, this, "ws").with(conf).at(bindAddress).start(webApp);
}
@Override
@ -158,7 +157,7 @@ public class HistoryClientService extends AbstractService {
private class MRClientProtocolHandler implements MRClientProtocol {
private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private Job verifyAndGetJob(final JobId jobID) throws YarnRemoteException {
UserGroupInformation loginUgi = null;
Job job = null;
@ -194,7 +193,7 @@ public class HistoryClientService extends AbstractService {
response.setCounters(job.getCounters());
return response;
}
@Override
public GetJobReportResponse getJobReport(GetJobReportRequest request) throws YarnRemoteException {
JobId jobId = request.getJobId();
@ -227,23 +226,23 @@ public class HistoryClientService extends AbstractService {
JobId jobId = request.getJobId();
int fromEventId = request.getFromEventId();
int maxEvents = request.getMaxEvents();
Job job = verifyAndGetJob(jobId);
GetTaskAttemptCompletionEventsResponse response = recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
response.addAllCompletionEvents(Arrays.asList(job.getTaskAttemptCompletionEvents(fromEventId, maxEvents)));
return response;
}
@Override
public KillJobResponse killJob(KillJobRequest request) throws YarnRemoteException {
throw RPCUtil.getRemoteException("Invalid operation on completed job");
}
@Override
public KillTaskResponse killTask(KillTaskRequest request) throws YarnRemoteException {
throw RPCUtil.getRemoteException("Invalid operation on completed job");
}
@Override
public KillTaskAttemptResponse killTaskAttempt(KillTaskAttemptRequest request) throws YarnRemoteException {
throw RPCUtil.getRemoteException("Invalid operation on completed job");
@ -252,15 +251,15 @@ public class HistoryClientService extends AbstractService {
@Override
public GetDiagnosticsResponse getDiagnostics(GetDiagnosticsRequest request) throws YarnRemoteException {
TaskAttemptId taskAttemptId = request.getTaskAttemptId();
Job job = verifyAndGetJob(taskAttemptId.getTaskId().getJobId());
GetDiagnosticsResponse response = recordFactory.newRecordInstance(GetDiagnosticsResponse.class);
response.addAllDiagnostics(job.getTask(taskAttemptId.getTaskId()).getAttempt(taskAttemptId).getDiagnostics());
return response;
}
@Override
@Override
public FailTaskAttemptResponse failTaskAttempt(FailTaskAttemptRequest request) throws YarnRemoteException {
throw RPCUtil.getRemoteException("Invalid operation on completed job");
}
@ -269,7 +268,7 @@ public class HistoryClientService extends AbstractService {
public GetTaskReportsResponse getTaskReports(GetTaskReportsRequest request) throws YarnRemoteException {
JobId jobId = request.getJobId();
TaskType taskType = request.getTaskType();
GetTaskReportsResponse response = recordFactory.newRecordInstance(GetTaskReportsResponse.class);
Job job = verifyAndGetJob(jobId);
Collection<Task> tasks = job.getTasks(taskType).values();

View File

@ -21,7 +21,7 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
import org.apache.hadoop.yarn.webapp.SubView;
import org.apache.hadoop.yarn.webapp.view.InfoBlock;
@ -45,8 +45,9 @@ public class HsAboutPage extends HsView {
* @return AttemptsBlock.class
*/
@Override protected Class<? extends SubView> content() {
HistoryInfo info = new HistoryInfo();
info("History Server").
_("BuildVersion", VersionInfo.getBuildVersion());
_("BuildVersion", info.getHadoopBuildVersion() + " on " + info.getHadoopVersionBuiltOn());
return InfoBlock.class;
}
}

View File

@ -34,6 +34,9 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
import org.apache.hadoop.security.authorize.AccessControlList;
@ -56,19 +59,6 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
public class HsJobBlock extends HtmlBlock {
final AppContext appContext;
int killedMapAttempts = 0;
int failedMapAttempts = 0;
int successfulMapAttempts = 0;
int killedReduceAttempts = 0;
int failedReduceAttempts = 0;
int successfulReduceAttempts = 0;
long avgMapTime = 0;
long avgReduceTime = 0;
long avgShuffleTime = 0;
long avgSortTime = 0;
int numMaps;
int numReduces;
@Inject HsJobBlock(AppContext appctx) {
appContext = appctx;
}
@ -85,37 +75,30 @@ public class HsJobBlock extends HtmlBlock {
return;
}
JobId jobID = MRApps.toJobID(jid);
Job job = appContext.getJob(jobID);
if (job == null) {
Job j = appContext.getJob(jobID);
if (j == null) {
html.
p()._("Sorry, ", jid, " not found.")._();
return;
}
Map<JobACL, AccessControlList> acls = job.getJobACLs();
List<AMInfo> amInfos = job.getAMInfos();
JobReport jobReport = job.getReport();
int mapTasks = job.getTotalMaps();
int mapTasksComplete = job.getCompletedMaps();
int reduceTasks = job.getTotalReduces();
int reducesTasksComplete = job.getCompletedReduces();
long startTime = jobReport.getStartTime();
long finishTime = jobReport.getFinishTime();
countTasksAndAttempts(job);
List<AMInfo> amInfos = j.getAMInfos();
JobInfo job = new JobInfo(j);
ResponseInfo infoBlock = info("Job Overview").
_("Job Name:", job.getName()).
_("User Name:", job.getUserName()).
_("Queue:", job.getQueueName()).
_("State:", job.getState()).
_("Uberized:", job.isUber()).
_("Started:", new Date(startTime)).
_("Finished:", new Date(finishTime)).
_("Started:", new Date(job.getStartTime())).
_("Finished:", new Date(job.getFinishTime())).
_("Elapsed:", StringUtils.formatTime(
Times.elapsed(startTime, finishTime, false)));
Times.elapsed(job.getStartTime(), job.getFinishTime(), false)));
String amString =
amInfos.size() == 1 ? "ApplicationMaster" : "ApplicationMasters";
List<String> diagnostics = job.getDiagnostics();
// todo - switch to use JobInfo
List<String> diagnostics = j.getDiagnostics();
if(diagnostics != null && !diagnostics.isEmpty()) {
StringBuffer b = new StringBuffer();
for(String diag: diagnostics) {
@ -124,18 +107,17 @@ public class HsJobBlock extends HtmlBlock {
infoBlock._("Diagnostics:", b.toString());
}
if(numMaps > 0) {
infoBlock._("Average Map Time", StringUtils.formatTime(avgMapTime));
if(job.getNumMaps() > 0) {
infoBlock._("Average Map Time", StringUtils.formatTime(job.getAvgMapTime()));
}
if(numReduces > 0) {
infoBlock._("Average Reduce Time", StringUtils.formatTime(avgReduceTime));
infoBlock._("Average Shuffle Time", StringUtils.formatTime(avgShuffleTime));
infoBlock._("Average Merge Time", StringUtils.formatTime(avgSortTime));
if(job.getNumReduces() > 0) {
infoBlock._("Average Reduce Time", StringUtils.formatTime(job.getAvgReduceTime()));
infoBlock._("Average Shuffle Time", StringUtils.formatTime(job.getAvgShuffleTime()));
infoBlock._("Average Merge Time", StringUtils.formatTime(job.getAvgMergeTime()));
}
for(Map.Entry<JobACL, AccessControlList> entry : acls.entrySet()) {
infoBlock._("ACL "+entry.getKey().getAclName()+":",
entry.getValue().getAclString());
for (ConfEntryInfo entry : job.getAcls()) {
infoBlock._("ACL "+entry.getName()+":", entry.getValue());
}
DIV<Hamlet> div = html.
_(InfoBlock.class).
@ -154,18 +136,14 @@ public class HsJobBlock extends HtmlBlock {
th(_TH, "Logs").
_();
for (AMInfo amInfo : amInfos) {
String nodeHttpAddress = amInfo.getNodeManagerHost() +
":" + amInfo.getNodeManagerHttpPort();
NodeId nodeId = BuilderUtils.newNodeId(
amInfo.getNodeManagerHost(), amInfo.getNodeManagerPort());
AMAttemptInfo attempt = new AMAttemptInfo(amInfo,
job.getId(), job.getUserName(), "", "");
table.tr().
td(String.valueOf(amInfo.getAppAttemptId().getAttemptId())).
td(new Date(amInfo.getStartTime()).toString()).
td().a(".nodelink", url("http://", nodeHttpAddress),
nodeHttpAddress)._().
td().a(".logslink", url("logs", nodeId.toString(),
amInfo.getContainerId().toString(), jid, job.getUserName()),
td(String.valueOf(attempt.getAttemptId())).
td(new Date(attempt.getStartTime()).toString()).
td().a(".nodelink", url("http://", attempt.getNodeHttpAddress()),
attempt.getNodeHttpAddress())._().
td().a(".logslink", url(attempt.getShortLogsLink()),
"logs")._().
_();
}
@ -184,13 +162,13 @@ public class HsJobBlock extends HtmlBlock {
tr(_ODD).
th().
a(url("tasks", jid, "m"), "Map")._().
td(String.valueOf(mapTasks)).
td(String.valueOf(mapTasksComplete))._().
td(String.valueOf(String.valueOf(job.getMapsTotal()))).
td(String.valueOf(String.valueOf(job.getMapsCompleted())))._().
tr(_EVEN).
th().
a(url("tasks", jid, "r"), "Reduce")._().
td(String.valueOf(reduceTasks)).
td(String.valueOf(reducesTasksComplete))._()
td(String.valueOf(String.valueOf(job.getReducesTotal()))).
td(String.valueOf(String.valueOf(job.getReducesCompleted())))._()
._().
// Attempts table
@ -204,99 +182,27 @@ public class HsJobBlock extends HtmlBlock {
th("Maps").
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.FAILED.toString()),
String.valueOf(failedMapAttempts))._().
String.valueOf(job.getFailedMapAttempts()))._().
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.KILLED.toString()),
String.valueOf(killedMapAttempts))._().
String.valueOf(job.getKilledMapAttempts()))._().
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.SUCCESSFUL.toString()),
String.valueOf(successfulMapAttempts))._().
String.valueOf(job.getSuccessfulMapAttempts()))._().
_().
tr(_EVEN).
th("Reduces").
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.FAILED.toString()),
String.valueOf(failedReduceAttempts))._().
String.valueOf(job.getFailedReduceAttempts()))._().
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.KILLED.toString()),
String.valueOf(killedReduceAttempts))._().
String.valueOf(job.getKilledReduceAttempts()))._().
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.SUCCESSFUL.toString()),
String.valueOf(successfulReduceAttempts))._().
String.valueOf(job.getSuccessfulReduceAttempts()))._().
_().
_().
_();
}
/**
* Go through a job and update the member variables with counts for
* information to output in the page.
* @param job the job to get counts for.
*/
private void countTasksAndAttempts(Job job) {
numReduces = 0;
numMaps = 0;
Map<TaskId, Task> tasks = job.getTasks();
for (Task task : tasks.values()) {
// Attempts counts
Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
for (TaskAttempt attempt : attempts.values()) {
int successful = 0, failed = 0, killed =0;
if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
//Do Nothing
} else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
.getState())) {
//Do Nothing
} else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
.getState())) {
++successful;
} else if (TaskAttemptStateUI.FAILED
.correspondsTo(attempt.getState())) {
++failed;
} else if (TaskAttemptStateUI.KILLED
.correspondsTo(attempt.getState())) {
++killed;
}
switch (task.getType()) {
case MAP:
successfulMapAttempts += successful;
failedMapAttempts += failed;
killedMapAttempts += killed;
if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
numMaps++;
avgMapTime += (attempt.getFinishTime() -
attempt.getLaunchTime());
}
break;
case REDUCE:
successfulReduceAttempts += successful;
failedReduceAttempts += failed;
killedReduceAttempts += killed;
if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
numReduces++;
avgShuffleTime += (attempt.getShuffleFinishTime() -
attempt.getLaunchTime());
avgSortTime += attempt.getSortFinishTime() -
attempt.getLaunchTime();
avgReduceTime += (attempt.getFinishTime() -
attempt.getShuffleFinishTime());
}
break;
}
}
}
if(numMaps > 0) {
avgMapTime = avgMapTime / numMaps;
}
if(numReduces > 0) {
avgReduceTime = avgReduceTime / numReduces;
avgShuffleTime = avgShuffleTime / numReduces;
avgSortTime = avgSortTime / numReduces;
}
}
}

View File

@ -21,10 +21,9 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
import java.text.SimpleDateFormat;
import java.util.Date;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
@ -38,8 +37,8 @@ import com.google.inject.Inject;
*/
public class HsJobsBlock extends HtmlBlock {
final AppContext appContext;
static final SimpleDateFormat dateFormat =
new SimpleDateFormat("yyyy.MM.dd HH:mm:ss z");
static final SimpleDateFormat dateFormat =
new SimpleDateFormat("yyyy.MM.dd HH:mm:ss z");
@Inject HsJobsBlock(AppContext appCtx) {
appContext = appCtx;
@ -68,28 +67,21 @@ public class HsJobsBlock extends HtmlBlock {
th("Reduces Completed")._()._().
tbody();
LOG.info("Getting list of all Jobs.");
for (Job job : appContext.getAllJobs().values()) {
String jobID = MRApps.toString(job.getID());
JobReport report = job.getReport();
String mapsTotal = String.valueOf(job.getTotalMaps());
String mapsCompleted = String.valueOf(job.getCompletedMaps());
String reduceTotal = String.valueOf(job.getTotalReduces());
String reduceCompleted = String.valueOf(job.getCompletedReduces());
long startTime = report.getStartTime();
long finishTime = report.getFinishTime();
for (Job j : appContext.getAllJobs().values()) {
JobInfo job = new JobInfo(j);
tbody.
tr().
td(dateFormat.format(new Date(startTime))).
td(dateFormat.format(new Date(finishTime))).
td().a(url("job", jobID), jobID)._().
td(job.getName().toString()).
td(dateFormat.format(new Date(job.getStartTime()))).
td(dateFormat.format(new Date(job.getFinishTime()))).
td().a(url("job", job.getId()), job.getId())._().
td(job.getName()).
td(job.getUserName()).
td(job.getQueueName()).
td(job.getState().toString()).
td(mapsTotal).
td(mapsCompleted).
td(reduceTotal).
td(reduceCompleted)._();
td(job.getState()).
td(String.valueOf(job.getMapsTotal())).
td(String.valueOf(job.getMapsCompleted())).
td(String.valueOf(job.getReducesTotal())).
td(String.valueOf(job.getReducesCompleted()))._();
}
tbody._().
tfoot().

View File

@ -20,12 +20,13 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.webapp.App;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.util.Times;
@ -65,7 +66,7 @@ public class HsTasksBlock extends HtmlBlock {
if (!symbol.isEmpty()) {
type = MRApps.taskType(symbol);
}
THEAD<TABLE<Hamlet>> thead = html.table("#tasks").thead();
//Create the spanning row
int attemptColSpan = type == TaskType.REDUCE ? 8 : 3;
@ -74,7 +75,7 @@ public class HsTasksBlock extends HtmlBlock {
th().$colspan(attemptColSpan).$class("ui-state-default").
_("Successful Attempt")._().
_();
TR<THEAD<TABLE<Hamlet>>> theadRow = thead.
tr().
th("Name").
@ -83,33 +84,33 @@ public class HsTasksBlock extends HtmlBlock {
th("Finish Time").
th("Elapsed Time").
th("Start Time"); //Attempt
if(type == TaskType.REDUCE) {
theadRow.th("Shuffle Finish Time"); //Attempt
theadRow.th("Merge Finish Time"); //Attempt
}
theadRow.th("Finish Time"); //Attempt
if(type == TaskType.REDUCE) {
theadRow.th("Elapsed Time Shuffle"); //Attempt
theadRow.th("Elapsed Time Merge"); //Attempt
theadRow.th("Elapsed Time Reduce"); //Attempt
}
theadRow.th("Elapsed Time"); //Attempt
TBODY<TABLE<Hamlet>> tbody = theadRow._()._().tbody();
for (Task task : app.getJob().getTasks().values()) {
if (type != null && task.getType() != type) {
continue;
}
String tid = MRApps.toString(task.getID());
TaskReport report = task.getReport();
long startTime = report.getStartTime();
long finishTime = report.getFinishTime();
long elapsed = Times.elapsed(startTime, finishTime, false);
TaskInfo info = new TaskInfo(task);
String tid = info.getId();
long startTime = info.getStartTime();
long finishTime = info.getFinishTime();
long elapsed = info.getElapsedTime();
long attemptStartTime = -1;
long shuffleFinishTime = -1;
long sortFinishTime = -1;
@ -118,30 +119,31 @@ public class HsTasksBlock extends HtmlBlock {
long elapsedSortTime = -1;;
long elapsedReduceTime = -1;
long attemptElapsed = -1;
TaskAttempt successful = getSuccessfulAttempt(task);
TaskAttempt successful = info.getSuccessful();
if(successful != null) {
attemptStartTime = successful.getLaunchTime();
attemptFinishTime = successful.getFinishTime();
TaskAttemptInfo ta;
if(type == TaskType.REDUCE) {
shuffleFinishTime = successful.getShuffleFinishTime();
sortFinishTime = successful.getSortFinishTime();
elapsedShuffleTime =
Times.elapsed(attemptStartTime, shuffleFinishTime, false);
elapsedSortTime =
Times.elapsed(shuffleFinishTime, sortFinishTime, false);
elapsedReduceTime =
Times.elapsed(sortFinishTime, attemptFinishTime, false);
ReduceTaskAttemptInfo rta = new ReduceTaskAttemptInfo(successful, type);
shuffleFinishTime = rta.getShuffleFinishTime();
sortFinishTime = rta.getMergeFinishTime();
elapsedShuffleTime = rta.getElapsedShuffleTime();
elapsedSortTime = rta.getElapsedMergeTime();
elapsedReduceTime = rta.getElapsedReduceTime();
ta = rta;
} else {
ta = new TaskAttemptInfo(successful, type, false);
}
attemptElapsed =
Times.elapsed(attemptStartTime, attemptFinishTime, false);
attemptStartTime = ta.getStartTime();
attemptFinishTime = ta.getFinishTime();
attemptElapsed = ta.getElapsedTime();
}
TR<TBODY<TABLE<Hamlet>>> row = tbody.tr();
row.
td().
br().$title(String.valueOf(task.getID().getId()))._(). // sorting
br().$title(String.valueOf(info.getTaskNum()))._(). // sorting
a(url("task", tid), tid)._().
td(report.getTaskState().toString()).
td(info.getState()).
td().
br().$title(String.valueOf(startTime))._().
_(Times.format(startTime))._().
@ -166,7 +168,7 @@ public class HsTasksBlock extends HtmlBlock {
td().
br().$title(String.valueOf(attemptFinishTime))._().
_(Times.format(attemptFinishTime))._();
if(type == TaskType.REDUCE) {
row.td().
br().$title(String.valueOf(elapsedShuffleTime))._().
@ -178,7 +180,7 @@ public class HsTasksBlock extends HtmlBlock {
br().$title(String.valueOf(elapsedReduceTime))._().
_(formatTime(elapsedReduceTime))._();
}
row.td().
br().$title(String.valueOf(attemptElapsed))._().
_(formatTime(attemptElapsed))._();
@ -194,7 +196,7 @@ public class HsTasksBlock extends HtmlBlock {
.$type(InputType.text).$name("elapsed_time").$value("Elapsed Time")._()
._().th().input("search_init").$type(InputType.text)
.$name("attempt_start_time").$value("Start Time")._()._();
if(type == TaskType.REDUCE) {
footRow.th().input("search_init").$type(InputType.text)
.$name("shuffle_time").$value("Shuffle Time")._()._();
@ -216,20 +218,12 @@ public class HsTasksBlock extends HtmlBlock {
footRow.th().input("search_init").$type(InputType.text)
.$name("attempt_elapsed").$value("Elapsed Time")._()._();
footRow._()._()._();
}
private String formatTime(long elapsed) {
return elapsed < 0 ? "N/A" : StringUtils.formatTime(elapsed);
}
private TaskAttempt getSuccessfulAttempt(Task task) {
for(TaskAttempt attempt: task.getAttempts().values()) {
if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
return attempt;
}
}
return null;
}
}

View File

@ -27,6 +27,7 @@ import static org.apache.hadoop.yarn.webapp.YarnWebParams.NM_NODENAME;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.webapp.AMParams;
import org.apache.hadoop.mapreduce.v2.hs.HistoryContext;
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
import org.apache.hadoop.yarn.webapp.WebApp;
public class HsWebApp extends WebApp implements AMParams {
@ -39,6 +40,9 @@ public class HsWebApp extends WebApp implements AMParams {
@Override
public void setup() {
bind(HsWebServices.class);
bind(JAXBContextResolver.class);
bind(GenericExceptionHandler.class);
bind(AppContext.class).toInstance(history);
route("/", HsController.class);
route("/app", HsController.class);

View File

@ -0,0 +1,469 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import java.io.IOException;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.UriInfo;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
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.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptsInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.webapp.BadRequestException;
import org.apache.hadoop.yarn.webapp.NotFoundException;
import org.apache.hadoop.yarn.webapp.WebApp;
import com.google.inject.Inject;
@Path("/ws/v1/history")
public class HsWebServices {
private final AppContext appCtx;
private WebApp webapp;
private final Configuration conf;
@Context
UriInfo uriInfo;
@Inject
public HsWebServices(final AppContext appCtx, final Configuration conf,
final WebApp webapp) {
this.appCtx = appCtx;
this.conf = conf;
this.webapp = webapp;
}
@GET
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public HistoryInfo get() {
return getHistoryInfo();
}
@GET
@Path("/info")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public HistoryInfo getHistoryInfo() {
return new HistoryInfo();
}
@GET
@Path("/mapreduce/jobs")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobsInfo getJobs(@QueryParam("user") String userQuery,
@QueryParam("limit") String count,
@QueryParam("queue") String queueQuery,
@QueryParam("startedTimeBegin") String startedBegin,
@QueryParam("startedTimeEnd") String startedEnd,
@QueryParam("finishedTimeBegin") String finishBegin,
@QueryParam("finishedTimeEnd") String finishEnd) {
JobsInfo allJobs = new JobsInfo();
long num = 0;
boolean checkCount = false;
boolean checkStart = false;
boolean checkEnd = false;
long countNum = 0;
// set values suitable in case both of begin/end not specified
long sBegin = 0;
long sEnd = Long.MAX_VALUE;
long fBegin = 0;
long fEnd = Long.MAX_VALUE;
if (count != null && !count.isEmpty()) {
checkCount = true;
try {
countNum = Long.parseLong(count);
} catch (NumberFormatException e) {
throw new BadRequestException(e.getMessage());
}
if (countNum <= 0) {
throw new BadRequestException("limit value must be greater then 0");
}
}
if (startedBegin != null && !startedBegin.isEmpty()) {
checkStart = true;
try {
sBegin = Long.parseLong(startedBegin);
} catch (NumberFormatException e) {
throw new BadRequestException(e.getMessage());
}
if (sBegin < 0) {
throw new BadRequestException("startedTimeBegin must be greater than 0");
}
}
if (startedEnd != null && !startedEnd.isEmpty()) {
checkStart = true;
try {
sEnd = Long.parseLong(startedEnd);
} catch (NumberFormatException e) {
throw new BadRequestException(e.getMessage());
}
if (sEnd < 0) {
throw new BadRequestException("startedTimeEnd must be greater than 0");
}
}
if (sBegin > sEnd) {
throw new BadRequestException(
"startedTimeEnd must be greater than startTimeBegin");
}
if (finishBegin != null && !finishBegin.isEmpty()) {
checkEnd = true;
try {
fBegin = Long.parseLong(finishBegin);
} catch (NumberFormatException e) {
throw new BadRequestException(e.getMessage());
}
if (fBegin < 0) {
throw new BadRequestException("finishTimeBegin must be greater than 0");
}
}
if (finishEnd != null && !finishEnd.isEmpty()) {
checkEnd = true;
try {
fEnd = Long.parseLong(finishEnd);
} catch (NumberFormatException e) {
throw new BadRequestException(e.getMessage());
}
if (fEnd < 0) {
throw new BadRequestException("finishTimeEnd must be greater than 0");
}
}
if (fBegin > fEnd) {
throw new BadRequestException(
"finishTimeEnd must be greater than finishTimeBegin");
}
for (Job job : appCtx.getAllJobs().values()) {
if (checkCount && num == countNum) {
break;
}
// getAllJobs only gives you a partial we want a full
Job fullJob = appCtx.getJob(job.getID());
if (fullJob == null) {
continue;
}
JobInfo jobInfo = new JobInfo(fullJob);
// can't really validate queue is a valid one since queues could change
if (queueQuery != null && !queueQuery.isEmpty()) {
if (!jobInfo.getQueueName().equals(queueQuery)) {
continue;
}
}
if (userQuery != null && !userQuery.isEmpty()) {
if (!jobInfo.getName().equals(userQuery)) {
continue;
}
}
if (checkStart
&& (jobInfo.getStartTime() < sBegin || jobInfo.getStartTime() > sEnd)) {
continue;
}
if (checkEnd
&& (jobInfo.getFinishTime() < fBegin || jobInfo.getFinishTime() > fEnd)) {
continue;
}
allJobs.add(jobInfo);
num++;
}
return allJobs;
}
@GET
@Path("/mapreduce/jobs/{jobid}")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobInfo getJob(@PathParam("jobid") String jid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
return new JobInfo(job);
}
@GET
@Path("/mapreduce/jobs/{jobid}/attempts")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public AMAttemptsInfo getJobAttempts(@PathParam("jobid") String jid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
AMAttemptsInfo amAttempts = new AMAttemptsInfo();
for (AMInfo amInfo : job.getAMInfos()) {
AMAttemptInfo attempt = new AMAttemptInfo(amInfo, MRApps.toString(job
.getID()), job.getUserName(), uriInfo.getBaseUri().toString(),
webapp.name());
amAttempts.add(attempt);
}
return amAttempts;
}
@GET
@Path("/mapreduce/jobs/{jobid}/counters")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobCounterInfo getJobCounters(@PathParam("jobid") String jid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
return new JobCounterInfo(this.appCtx, job);
}
@GET
@Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/counters")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobTaskCounterInfo getSingleTaskCounters(
@PathParam("jobid") String jid, @PathParam("taskid") String tid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = this.appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
return new JobTaskCounterInfo(task);
}
@GET
@Path("/mapreduce/jobs/{jobid}/conf")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public ConfInfo getJobConf(@PathParam("jobid") String jid) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
ConfInfo info;
try {
info = new ConfInfo(job, this.conf);
} catch (IOException e) {
throw new NotFoundException("unable to load configuration for job: "
+ jid);
}
return info;
}
@GET
@Path("/mapreduce/jobs/{jobid}/tasks")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public TasksInfo getJobTasks(@PathParam("jobid") String jid,
@QueryParam("type") String type) {
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
TasksInfo allTasks = new TasksInfo();
for (Task task : job.getTasks().values()) {
TaskType ttype = null;
if (type != null && !type.isEmpty()) {
try {
ttype = MRApps.taskType(type);
} catch (YarnException e) {
throw new BadRequestException("tasktype must be either m or r");
}
}
if (ttype != null && task.getType() != ttype) {
continue;
}
allTasks.add(new TaskInfo(task));
}
return allTasks;
}
@GET
@Path("/mapreduce/jobs/{jobid}/tasks/{taskid}")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public TaskInfo getJobTask(@PathParam("jobid") String jid,
@PathParam("taskid") String tid) {
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
return new TaskInfo(task);
}
@GET
@Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public TaskAttemptsInfo getJobTaskAttempts(@PathParam("jobid") String jid,
@PathParam("taskid") String tid) {
TaskAttemptsInfo attempts = new TaskAttemptsInfo();
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
for (TaskAttempt ta : task.getAttempts().values()) {
if (ta != null) {
if (task.getType() == TaskType.REDUCE) {
attempts.add(new ReduceTaskAttemptInfo(ta, task.getType()));
} else {
attempts.add(new TaskAttemptInfo(ta, task.getType(), false));
}
}
}
return attempts;
}
@GET
@Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public TaskAttemptInfo getJobTaskAttemptId(@PathParam("jobid") String jid,
@PathParam("taskid") String tid, @PathParam("attemptid") String attId) {
Job job = this.appCtx.getJob(MRApps.toJobID(jid));
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
if (attemptId == null) {
throw new NotFoundException("task attempt id " + attId
+ " not found or invalid");
}
TaskAttempt ta = task.getAttempt(attemptId);
if (ta == null) {
throw new NotFoundException("Error getting info on task attempt id "
+ attId);
}
if (task.getType() == TaskType.REDUCE) {
return new ReduceTaskAttemptInfo(ta, task.getType());
} else {
return new TaskAttemptInfo(ta, task.getType(), false);
}
}
@GET
@Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/counters")
@Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
public JobTaskAttemptCounterInfo getJobTaskAttemptIdCounters(
@PathParam("jobid") String jid, @PathParam("taskid") String tid,
@PathParam("attemptid") String attId) {
JobId jobId = MRApps.toJobID(jid);
if (jobId == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
Job job = this.appCtx.getJob(jobId);
if (job == null) {
throw new NotFoundException("job, " + jid + ", is not found");
}
TaskId taskID = MRApps.toTaskID(tid);
if (taskID == null) {
throw new NotFoundException("taskid " + tid + " not found or invalid");
}
Task task = job.getTask(taskID);
if (task == null) {
throw new NotFoundException("task not found with id " + tid);
}
TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
if (attemptId == null) {
throw new NotFoundException("task attempt id " + attId
+ " not found or invalid");
}
TaskAttempt ta = task.getAttempt(attemptId);
if (ta == null) {
throw new NotFoundException("Error getting info on task attempt id "
+ attId);
}
return new JobTaskAttemptCounterInfo(ta);
}
}

View File

@ -0,0 +1,78 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import com.google.inject.Singleton;
import com.sun.jersey.api.json.JSONConfiguration;
import com.sun.jersey.api.json.JSONJAXBContext;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import javax.ws.rs.ext.ContextResolver;
import javax.ws.rs.ext.Provider;
import javax.xml.bind.JAXBContext;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.CounterGroupInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.CounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterGroupInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterInfo;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptsInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
@Singleton
@Provider
public class JAXBContextResolver implements ContextResolver<JAXBContext> {
private JAXBContext context;
private final Set<Class> types;
// you have to specify all the dao classes here
private final Class[] cTypes = { HistoryInfo.class, JobInfo.class,
JobsInfo.class, TasksInfo.class, TaskAttemptsInfo.class, ConfInfo.class,
CounterInfo.class, JobTaskCounterInfo.class,
JobTaskAttemptCounterInfo.class,
TaskCounterInfo.class, JobCounterInfo.class, ReduceTaskAttemptInfo.class,
TaskAttemptInfo.class, TaskAttemptsInfo.class, CounterGroupInfo.class,
TaskCounterGroupInfo.class,
AMAttemptInfo.class, AMAttemptsInfo.class};
public JAXBContextResolver() throws Exception {
this.types = new HashSet<Class>(Arrays.asList(cTypes));
this.context = new JSONJAXBContext(JSONConfiguration.natural()
.rootUnwrapping(false).build(), cTypes);
}
@Override
public JAXBContext getContext(Class<?> objectType) {
return (types.contains(objectType)) ? context : null;
}
}

View File

@ -0,0 +1,96 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp.dao;
import static org.apache.hadoop.yarn.util.StringHelper.join;
import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.util.BuilderUtils;
@XmlRootElement(name = "amAttempt")
@XmlAccessorType(XmlAccessType.FIELD)
public class AMAttemptInfo {
protected String nodeHttpAddress;
protected String nodeId;
protected int id;
protected long startTime;
protected String containerId;
protected String logsLink;
@XmlTransient
protected String shortLogsLink;
public AMAttemptInfo() {
}
public AMAttemptInfo(AMInfo amInfo, String jobId, String user, String host,
String pathPrefix) {
this.nodeHttpAddress = amInfo.getNodeManagerHost() + ":"
+ amInfo.getNodeManagerHttpPort();
NodeId nodeId = BuilderUtils.newNodeId(amInfo.getNodeManagerHost(),
amInfo.getNodeManagerPort());
this.nodeId = nodeId.toString();
this.id = amInfo.getAppAttemptId().getAttemptId();
this.startTime = amInfo.getStartTime();
this.containerId = amInfo.getContainerId().toString();
this.logsLink = join(
host,
pathPrefix,
ujoin("logs", nodeId.toString(), amInfo.getContainerId().toString(),
jobId, user));
this.shortLogsLink = ujoin("logs", nodeId.toString(), amInfo
.getContainerId().toString(), jobId, user);
}
public String getNodeHttpAddress() {
return this.nodeHttpAddress;
}
public String getNodeId() {
return this.nodeId;
}
public int getAttemptId() {
return this.id;
}
public long getStartTime() {
return this.startTime;
}
public String getContainerId() {
return this.containerId;
}
public String getLogsLink() {
return this.logsLink;
}
public String getShortLogsLink() {
return this.shortLogsLink;
}
}

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by joblicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "attempts")
@XmlAccessorType(XmlAccessType.FIELD)
public class AMAttemptsInfo {
protected ArrayList<AMAttemptInfo> attempt = new ArrayList<AMAttemptInfo>();
public AMAttemptsInfo() {
} // JAXB needs this
public void add(AMAttemptInfo info) {
this.attempt.add(info);
}
public ArrayList<AMAttemptInfo> getAttempts() {
return this.attempt;
}
}

View File

@ -0,0 +1,53 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp.dao;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.util.VersionInfo;
@XmlRootElement
@XmlAccessorType(XmlAccessType.FIELD)
public class HistoryInfo {
protected String hadoopVersion;
protected String hadoopBuildVersion;
protected String hadoopVersionBuiltOn;
public HistoryInfo() {
this.hadoopVersion = VersionInfo.getVersion();
this.hadoopBuildVersion = VersionInfo.getBuildVersion();
this.hadoopVersionBuiltOn = VersionInfo.getDate();
}
public String getHadoopVersion() {
return this.hadoopVersion;
}
public String getHadoopBuildVersion() {
return this.hadoopBuildVersion;
}
public String getHadoopVersionBuiltOn() {
return this.hadoopVersionBuiltOn;
}
}

View File

@ -0,0 +1,295 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp.dao;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
import org.apache.hadoop.mapreduce.v2.hs.CompletedJob;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
import org.apache.hadoop.security.authorize.AccessControlList;
@XmlRootElement(name = "job")
@XmlAccessorType(XmlAccessType.FIELD)
public class JobInfo {
protected long startTime;
protected long finishTime;
protected String id;
protected String name;
protected String queue;
protected String user;
protected String state;
protected int mapsTotal;
protected int mapsCompleted;
protected int reducesTotal;
protected int reducesCompleted;
protected boolean uberized;
protected String diagnostics;
protected long avgMapTime = 0;
protected long avgReduceTime = 0;
protected long avgShuffleTime = 0;
protected long avgMergeTime = 0;
protected int failedReduceAttempts = 0;
protected int killedReduceAttempts = 0;
protected int successfulReduceAttempts = 0;
protected int failedMapAttempts = 0;
protected int killedMapAttempts = 0;
protected int successfulMapAttempts = 0;
protected ArrayList<ConfEntryInfo> acls;
@XmlTransient
protected int numMaps;
@XmlTransient
protected int numReduces;
public JobInfo() {
}
public JobInfo(Job job) {
this.id = MRApps.toString(job.getID());
JobReport report = job.getReport();
countTasksAndAttempts(job);
this.mapsTotal = job.getTotalMaps();
this.mapsCompleted = job.getCompletedMaps();
this.reducesTotal = job.getTotalReduces();
this.reducesCompleted = job.getCompletedReduces();
this.startTime = report.getStartTime();
this.finishTime = report.getFinishTime();
this.name = job.getName().toString();
this.queue = job.getQueueName();
this.user = job.getUserName();
this.state = job.getState().toString();
this.uberized = job.isUber();
List<String> diagnostics = job.getDiagnostics();
if (diagnostics != null && !diagnostics.isEmpty()) {
StringBuffer b = new StringBuffer();
for (String diag : diagnostics) {
b.append(diag);
}
this.diagnostics = b.toString();
}
this.acls = new ArrayList<ConfEntryInfo>();
if (job instanceof CompletedJob) {
Map<JobACL, AccessControlList> allacls = job.getJobACLs();
if (allacls != null) {
for (Map.Entry<JobACL, AccessControlList> entry : allacls.entrySet()) {
this.acls.add(new ConfEntryInfo(entry.getKey().getAclName(), entry
.getValue().getAclString()));
}
}
}
}
public long getNumMaps() {
return numMaps;
}
public long getNumReduces() {
return numReduces;
}
public long getAvgMapTime() {
return avgMapTime;
}
public long getAvgReduceTime() {
return avgReduceTime;
}
public long getAvgShuffleTime() {
return avgShuffleTime;
}
public long getAvgMergeTime() {
return avgMergeTime;
}
public long getFailedReduceAttempts() {
return failedReduceAttempts;
}
public long getKilledReduceAttempts() {
return killedReduceAttempts;
}
public long getSuccessfulReduceAttempts() {
return successfulReduceAttempts;
}
public long getFailedMapAttempts() {
return failedMapAttempts;
}
public long getKilledMapAttempts() {
return killedMapAttempts;
}
public long getSuccessfulMapAttempts() {
return successfulMapAttempts;
}
public ArrayList<ConfEntryInfo> getAcls() {
return acls;
}
public int getReducesCompleted() {
return this.reducesCompleted;
}
public int getReducesTotal() {
return this.reducesTotal;
}
public int getMapsCompleted() {
return this.mapsCompleted;
}
public int getMapsTotal() {
return this.mapsTotal;
}
public String getState() {
return this.state;
}
public String getUserName() {
return this.user;
}
public String getName() {
return this.name;
}
public String getQueueName() {
return this.queue;
}
public String getId() {
return this.id;
}
public long getStartTime() {
return this.startTime;
}
public long getFinishTime() {
return this.finishTime;
}
public boolean isUber() {
return this.uberized;
}
public String getDiagnostics() {
return this.diagnostics;
}
/**
* Go through a job and update the member variables with counts for
* information to output in the page.
*
* @param job
* the job to get counts for.
*/
private void countTasksAndAttempts(Job job) {
numReduces = 0;
numMaps = 0;
final Map<TaskId, Task> tasks = job.getTasks();
if (tasks == null) {
return;
}
for (Task task : tasks.values()) {
// Attempts counts
Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
int successful, failed, killed;
for (TaskAttempt attempt : attempts.values()) {
successful = 0;
failed = 0;
killed = 0;
if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
// Do Nothing
} else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt.getState())) {
// Do Nothing
} else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
.getState())) {
++successful;
} else if (TaskAttemptStateUI.FAILED.correspondsTo(attempt.getState())) {
++failed;
} else if (TaskAttemptStateUI.KILLED.correspondsTo(attempt.getState())) {
++killed;
}
switch (task.getType()) {
case MAP:
successfulMapAttempts += successful;
failedMapAttempts += failed;
killedMapAttempts += killed;
if (attempt.getState() == TaskAttemptState.SUCCEEDED) {
numMaps++;
avgMapTime += (attempt.getFinishTime() - attempt.getLaunchTime());
}
break;
case REDUCE:
successfulReduceAttempts += successful;
failedReduceAttempts += failed;
killedReduceAttempts += killed;
if (attempt.getState() == TaskAttemptState.SUCCEEDED) {
numReduces++;
avgShuffleTime += (attempt.getShuffleFinishTime() - attempt
.getLaunchTime());
avgMergeTime += attempt.getSortFinishTime()
- attempt.getLaunchTime();
avgReduceTime += (attempt.getFinishTime() - attempt
.getShuffleFinishTime());
}
break;
}
}
}
if (numMaps > 0) {
avgMapTime = avgMapTime / numMaps;
}
if (numReduces > 0) {
avgReduceTime = avgReduceTime / numReduces;
avgShuffleTime = avgShuffleTime / numReduces;
avgMergeTime = avgMergeTime / numReduces;
}
}
}

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by joblicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp.dao;
import java.util.ArrayList;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "jobs")
@XmlAccessorType(XmlAccessType.FIELD)
public class JobsInfo {
protected ArrayList<JobInfo> job = new ArrayList<JobInfo>();
public JobsInfo() {
} // JAXB needs this
public void add(JobInfo jobInfo) {
this.job.add(jobInfo);
}
public ArrayList<JobInfo> getJobs() {
return this.job;
}
}

View File

@ -168,7 +168,7 @@ public class TestClientServiceDelegate {
GetJobReportResponse jobReportResponse1 = mock(GetJobReportResponse.class);
when(jobReportResponse1.getJobReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "jobName-firstGen", "user",
JobState.RUNNING, 0, 0, 0, 0, 0, 0, 0, "anything", null));
JobState.RUNNING, 0, 0, 0, 0, 0, 0, 0, "anything", null, false));
// First AM returns a report with jobName firstGen and simulates AM shutdown
// on second invocation.
@ -180,7 +180,7 @@ public class TestClientServiceDelegate {
GetJobReportResponse jobReportResponse2 = mock(GetJobReportResponse.class);
when(jobReportResponse2.getJobReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "jobName-secondGen", "user",
JobState.RUNNING, 0, 0, 0, 0, 0, 0, 0, "anything", null));
JobState.RUNNING, 0, 0, 0, 0, 0, 0, 0, "anything", null, false));
// Second AM generation returns a report with jobName secondGen
MRClientProtocol secondGenAMProxy = mock(MRClientProtocol.class);

View File

@ -49,6 +49,7 @@ public class TestUberAM extends TestMRJobs {
}
@Override
@Test
public void testSleepJob()
throws IOException, InterruptedException, ClassNotFoundException {
if (mrCluster != null) {
@ -84,6 +85,7 @@ public class TestUberAM extends TestMRJobs {
}
@Override
@Test
public void testRandomWriter()
throws IOException, InterruptedException, ClassNotFoundException {
super.testRandomWriter();
@ -101,6 +103,7 @@ public class TestUberAM extends TestMRJobs {
}
@Override
@Test
public void testFailingMapper()
throws IOException, InterruptedException, ClassNotFoundException {
LOG.info("\n\n\nStarting uberized testFailingMapper().");

View File

@ -238,7 +238,7 @@
<dependency>
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-servlet</artifactId>
<version>2.0</version>
<version>3.0</version>
</dependency>
<dependency>
<groupId>junit</groupId>

View File

@ -1,27 +0,0 @@
@namespace("org.apache.hadoop.yarn")
protocol AMRMProtocol {
import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
// Scheduler
record Priority {
int priority;
}
record ResourceRequest {
Priority priority;
string hostName;
Resource capability;
int numContainers;
}
record AMResponse {
boolean reboot;
int responseId;
array<Container> containers;
}
void registerApplicationMaster(ApplicationMaster applicationMaster) throws YarnRemoteException;
void finishApplicationMaster(ApplicationMaster applicationMaster) throws YarnRemoteException;
AMResponse allocate(ApplicationStatus status, array<ResourceRequest> ask, array<Container> release) throws YarnRemoteException;
}

View File

@ -1,45 +0,0 @@
@namespace("org.apache.hadoop.yarn")
protocol ClientRMProtocol {
import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
record Priority {
int priority;
}
record ApplicationSubmissionContext {
ApplicationID applicationId;
union {null, string} applicationName;
Resource masterCapability; // TODO: Needs RM validation
//all the files required by the container to run the ApplicationMaster
//KEY-> destination dir name
//VALUE-> source path
map<URL> resources;
union {null, map<LocalResource>} resources_todo;
// TODO - Remove fsTokens (url encoded)
union {null, array<string>} fsTokens;
union {null, bytes} fsTokens_todo;
//env to be set before launching the command for ApplicationMaster
//KEY-> env variable name
//VALUE -> env variable value.
map<string> environment;
//command-line of the container that is going to launch the ApplicationMaster.
array<string> command;
union {null, string} queue;
union {null, Priority} priority;
string user; // TODO: Shouldn't pass it like this.
}
record YarnClusterMetrics {
int numNodeManagers;
}
ApplicationID getNewApplicationId() throws YarnRemoteException;
ApplicationMaster getApplicationMaster(ApplicationID applicationId) throws YarnRemoteException;
void submitApplication(ApplicationSubmissionContext context) throws YarnRemoteException;
void finishApplication(ApplicationID applicationId) throws YarnRemoteException;
YarnClusterMetrics getClusterMetrics() throws YarnRemoteException;
}

View File

@ -1,37 +0,0 @@
@namespace("org.apache.hadoop.yarn")
protocol ContainerManager {
import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
record ContainerLaunchContext {
ContainerID id;
string user; // TODO: Shouldn't pass it like this.
Resource resource; // TODO: Needs RM validation
union {null, map<LocalResource>} resources;
union {null, bytes} containerTokens; // FileSystem related and other application specific tokens.
union {null, map<bytes>} serviceData;
//env to be set before launching the command
//KEY-> env variable name
//VALUE -> env variable value.
map<string> env;
//commandline to launch the container. All resources are downloaded in the
//working directory of the command.
array<string> command;
}
record ContainerStatus {
ContainerID containerID;
ContainerState state;
int exitStatus;
}
void startContainer(ContainerLaunchContext container) throws YarnRemoteException;
void stopContainer(ContainerID containerID) throws YarnRemoteException;
void cleanupContainer(ContainerID containerID) throws YarnRemoteException;
ContainerStatus getContainerStatus(ContainerID containerID) throws YarnRemoteException;
}

View File

@ -1,109 +0,0 @@
@namespace("org.apache.hadoop.yarn")
protocol types {
record ApplicationID {
int id;
long clusterTimeStamp;
}
record ContainerID {
ApplicationID appID; // the application id to which this container belong.
int id;// unique string for this application
}
error YarnRemoteException {
union { null, string } message;
union { null, string } trace; //stackTrace
union { null, YarnRemoteException } cause;
}
record Resource {
int memory;
//int diskspace;
}
// State of the container on the ContainerManager.
enum ContainerState {
INTIALIZING,
RUNNING,
COMPLETE
}
record ContainerToken {
bytes identifier;
bytes password;
string kind;
string service;
}
record Container {
ContainerID id;
string hostName;
Resource resource;
ContainerState state;
union {ContainerToken, null} containerToken;
}
enum ApplicationState {
PENDING,
ALLOCATING,
ALLOCATED,
EXPIRED_PENDING,
LAUNCHING,
LAUNCHED,
RUNNING,
PAUSED,
CLEANUP,
COMPLETED,
KILLED,
FAILED
}
record ApplicationStatus {
int responseID; // TODO: This should be renamed as previousResponseID
ApplicationID applicationId;
float progress;
long lastSeen;
}
record ApplicationMaster {
ApplicationID applicationId;
union { null, string } host;
int rpcPort;
int httpPort;
ApplicationStatus status;
ApplicationState state;
union { null, string } clientToken;
}
record URL {
string scheme;
union { null, string } host;
int port;
string file;
}
enum LocalResourceVisibility {
// accessible to applications from all users
PUBLIC,
// accessible only to applications from the submitting user
PRIVATE,
// accessible only to this application
APPLICATION
}
enum LocalResourceType {
// an archive to be expanded
ARCHIVE,
// uninterpreted file
FILE
}
record LocalResource {
URL resource;
long size;
long timestamp;
LocalResourceType type;
LocalResourceVisibility state;
}
}

View File

@ -24,7 +24,6 @@ import org.apache.hadoop.security.UserGroupInformation;
* This is the API for the applications comprising of constants that YARN sets
* up for the applications and the containers.
*
* TODO: Should also be defined in avro/pb IDLs
* TODO: Investigate the semantics and security of each cross-boundary refs.
*/
public interface ApplicationConstants {

View File

@ -91,12 +91,7 @@ public class YarnConfiguration extends Configuration {
public static final String RM_CLIENT_THREAD_COUNT =
RM_PREFIX + "client.thread-count";
public static final int DEFAULT_RM_CLIENT_THREAD_COUNT = 10;
/** The expiry interval for application master reporting.*/
public static final String RM_AM_EXPIRY_INTERVAL_MS =
RM_PREFIX + "am.liveness-monitor.expiry-interval-ms";
public static final int DEFAULT_RM_AM_EXPIRY_INTERVAL_MS = 600000;
/** The Kerberos principal for the resource manager.*/
public static final String RM_PRINCIPAL =
RM_PREFIX + "principal";
@ -126,7 +121,17 @@ public class YarnConfiguration extends Configuration {
public static final int DEFAULT_RM_RESOURCE_TRACKER_PORT = 8025;
public static final String DEFAULT_RM_RESOURCE_TRACKER_ADDRESS =
"0.0.0.0:" + DEFAULT_RM_RESOURCE_TRACKER_PORT;
/** The expiry interval for application master reporting.*/
public static final String RM_AM_EXPIRY_INTERVAL_MS =
YARN_PREFIX + "am.liveness-monitor.expiry-interval-ms";
public static final int DEFAULT_RM_AM_EXPIRY_INTERVAL_MS = 600000;
/** How long to wait until a node manager is considered dead.*/
public static final String RM_NM_EXPIRY_INTERVAL_MS =
YARN_PREFIX + "nm.liveness-monitor.expiry-interval-ms";
public static final int DEFAULT_RM_NM_EXPIRY_INTERVAL_MS = 600000;
/** Are acls enabled.*/
public static final String YARN_ACL_ENABLE =
YARN_PREFIX + "acl.enable";
@ -160,12 +165,7 @@ public class YarnConfiguration extends Configuration {
/** The keytab for the resource manager.*/
public static final String RM_KEYTAB =
RM_PREFIX + "keytab";
/** How long to wait until a node manager is considered dead.*/
public static final String RM_NM_EXPIRY_INTERVAL_MS =
RM_PREFIX + "nm.liveness-monitor.expiry-interval-ms";
public static final int DEFAULT_RM_NM_EXPIRY_INTERVAL_MS = 600000;
/** How long to wait until a container is considered dead.*/
public static final String RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS =
RM_PREFIX + "rm.container-allocation.expiry-interval-ms";
@ -293,10 +293,16 @@ public class YarnConfiguration extends Configuration {
public static final String NM_LOG_DIRS = NM_PREFIX + "log-dirs";
public static final String DEFAULT_NM_LOG_DIRS = "/tmp/logs";
/** Interval at which the delayed token removal thread runs */
public static final String RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
RM_PREFIX + "delayed.delegation-token.removal-interval-ms";
public static final long DEFAULT_RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
30000l;
/** Whether to enable log aggregation */
public static final String NM_LOG_AGGREGATION_ENABLED = NM_PREFIX
public static final String LOG_AGGREGATION_ENABLED = YARN_PREFIX
+ "log-aggregation-enable";
public static final boolean DEFAULT_NM_LOG_AGGREGATION_ENABLED = false;
public static final boolean DEFAULT_LOG_AGGREGATION_ENABLED = false;
/**
* Number of seconds to retain logs on the NodeManager. Only applicable if Log

View File

@ -1,80 +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.ipc;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.AvroSpecificRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.YarnException;
/**
* This uses Hadoop RPC. Uses a tunnel AvroSpecificRpcEngine over
* Hadoop connection.
* This does not give cross-language wire compatibility, since the Hadoop
* RPC wire format is non-standard, but it does permit use of Avro's protocol
* versioning features for inter-Java RPCs.
*/
public class HadoopYarnRPC extends YarnRPC {
private static final Log LOG = LogFactory.getLog(HadoopYarnRPC.class);
@Override
public Object getProxy(Class protocol, InetSocketAddress addr,
Configuration conf) {
LOG.debug("Creating a HadoopYarnRpc proxy for protocol " + protocol);
RPC.setProtocolEngine(conf, protocol, AvroSpecificRpcEngine.class);
try {
return RPC.getProxy(protocol, 1, addr, conf);
} catch (IOException e) {
throw new YarnException(e);
}
}
@Override
public void stopProxy(Object proxy, Configuration conf) {
RPC.stopProxy(proxy);
}
@Override
public Server getServer(Class protocol, Object instance,
InetSocketAddress addr, Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager,
int numHandlers) {
LOG.debug("Creating a HadoopYarnRpc server for protocol " + protocol +
" with " + numHandlers + " handlers");
RPC.setProtocolEngine(conf, protocol, AvroSpecificRpcEngine.class);
final RPC.Server hadoopServer;
try {
hadoopServer = RPC.getServer(protocol, instance, addr.getHostName(),
addr.getPort(), numHandlers, false, conf, secretManager);
} catch (IOException e) {
throw new YarnException(e);
}
return hadoopServer;
}
}

View File

@ -0,0 +1,39 @@
/**
* 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.webapp;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Response.Status;
public class BadRequestException extends WebApplicationException {
private static final long serialVersionUID = 1L;
public BadRequestException() {
super(Status.BAD_REQUEST);
}
public BadRequestException(java.lang.Throwable cause) {
super(cause, Status.BAD_REQUEST);
}
public BadRequestException(String msg) {
super(new Exception(msg), Status.BAD_REQUEST);
}
}

View File

@ -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.yarn.webapp;
import java.io.IOException;
import javax.servlet.RequestDispatcher;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletRequestWrapper;
import javax.servlet.http.HttpServletResponse;
import com.google.inject.Singleton;
@Singleton
public class DefaultWrapperServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
public void doGet(HttpServletRequest req, HttpServletResponse resp)
throws ServletException, IOException {
RequestDispatcher rd = getServletContext().getNamedDispatcher("default");
HttpServletRequest wrapped = new HttpServletRequestWrapper(req) {
public String getServletPath() {
return "";
}
};
rd.forward(wrapped, resp);
}
}

View File

@ -0,0 +1,115 @@
/**
* 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.webapp;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Map;
import java.util.TreeMap;
import javax.servlet.http.HttpServletResponse;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.ext.ExceptionMapper;
import javax.ws.rs.ext.Provider;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.authorize.AuthorizationException;
import org.mortbay.util.ajax.JSON;
import com.google.inject.Singleton;
/**
* Handle webservices jersey exceptions and create json response in the format:
* { "RemoteException" :
* {
* "exception" : <exception type>,
* "javaClassName" : <classname of exception>,
* "message" : <error message from exception>
* }
* }
*/
@Singleton
@Provider
public class GenericExceptionHandler implements ExceptionMapper<Exception> {
public static final Log LOG = LogFactory
.getLog(GenericExceptionHandler.class);
private @Context
HttpServletResponse response;
@Override
public Response toResponse(Exception e) {
if (LOG.isTraceEnabled()) {
LOG.trace("GOT EXCEPITION", e);
}
// Don't catch this as filter forward on 404
// (ServletContainer.FEATURE_FILTER_FORWARD_ON_404)
// won't work and the web UI won't work!
if (e instanceof com.sun.jersey.api.NotFoundException) {
return ((com.sun.jersey.api.NotFoundException) e).getResponse();
}
// clear content type
response.setContentType(null);
// Convert exception
if (e instanceof RemoteException) {
e = ((RemoteException) e).unwrapRemoteException();
}
// Map response status
final Response.Status s;
if (e instanceof SecurityException) {
s = Response.Status.UNAUTHORIZED;
} else if (e instanceof AuthorizationException) {
s = Response.Status.UNAUTHORIZED;
} else if (e instanceof FileNotFoundException) {
s = Response.Status.NOT_FOUND;
} else if (e instanceof NotFoundException) {
s = Response.Status.NOT_FOUND;
} else if (e instanceof IOException) {
s = Response.Status.NOT_FOUND;
} else if (e instanceof UnsupportedOperationException) {
s = Response.Status.BAD_REQUEST;
} else if (e instanceof IllegalArgumentException) {
s = Response.Status.BAD_REQUEST;
} else if (e instanceof NumberFormatException) {
s = Response.Status.BAD_REQUEST;
} else if (e instanceof BadRequestException) {
s = Response.Status.BAD_REQUEST;
} else {
LOG.warn("INTERNAL_SERVER_ERROR", e);
s = Response.Status.INTERNAL_SERVER_ERROR;
}
// convert to json
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("exception", e.getClass().getSimpleName());
m.put("message", e.getMessage());
m.put("javaClassName", e.getClass().getName());
final Map<String, Object> m2 = new TreeMap<String, Object>();
m2.put(RemoteException.class.getSimpleName(), m);
final String js = JSON.toString(m2);
return Response.status(s).type(MediaType.APPLICATION_JSON).entity(js)
.build();
}
}

View File

@ -16,30 +16,29 @@
* limitations under the License.
*/
package org.apache.hadoop.ipc;
package org.apache.hadoop.yarn.webapp;
import java.io.IOException;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Response.Status;
import org.apache.avro.ipc.Responder;
import org.apache.avro.ipc.Transceiver;
import org.apache.avro.ipc.specific.SpecificRequestor;
import org.apache.avro.ipc.specific.SpecificResponder;
import org.apache.hadoop.classification.InterfaceStability;
/**
* AvroRpcEngine which uses Avro's "specific" APIs. The protocols generated
* via Avro IDL needs to use this Engine.
/*
* Created our own NotFoundException because com.sun.jersey.api.NotFoundException
* sets the Response and therefore won't be handled by the GenericExceptionhandler
* to fill in correct response.
*/
@InterfaceStability.Evolving
public class AvroSpecificRpcEngine extends AvroRpcEngine {
public class NotFoundException extends WebApplicationException {
protected SpecificRequestor createRequestor(Class<?> protocol,
Transceiver transeiver) throws IOException {
return new SpecificRequestor(protocol, transeiver);
private static final long serialVersionUID = 1L;
public NotFoundException() {
super(Status.NOT_FOUND);
}
protected Responder createResponder(Class<?> iface, Object impl) {
return new SpecificResponder(iface, impl);
public NotFoundException(java.lang.Throwable cause) {
super(cause, Status.NOT_FOUND);
}
public NotFoundException(String msg) {
super(new Exception(msg), Status.NOT_FOUND);
}
}

View File

@ -18,23 +18,28 @@
package org.apache.hadoop.yarn.webapp;
import static com.google.common.base.Preconditions.checkNotNull;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.http.HttpServer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.CharMatcher;
import static com.google.common.base.Preconditions.*;
import com.google.common.base.Splitter;
import com.google.common.collect.Lists;
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;
import org.apache.hadoop.http.HttpServer;
import org.apache.hadoop.yarn.util.StringHelper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.sun.jersey.api.core.ResourceConfig;
import com.sun.jersey.core.util.FeaturesAndProperties;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
import com.sun.jersey.spi.container.servlet.ServletContainer;
/**
* @see WebApps for a usage example
@ -45,9 +50,10 @@ public abstract class WebApp extends ServletModule {
public enum HTTP { GET, POST, HEAD, PUT, DELETE };
private volatile String name;
private volatile List<String> servePathSpecs = new ArrayList<String>();
private volatile List<String> servePathSpecs = new ArrayList<String>();
// path to redirect to if user goes to "/"
private volatile String redirectPath;
private volatile String wsName;
private volatile Configuration conf;
private volatile HttpServer httpServer;
private volatile GuiceFilter guiceFilter;
@ -104,18 +110,20 @@ public abstract class WebApp extends ServletModule {
void addServePathSpec(String path) { this.servePathSpecs.add(path); }
public String[] getServePathSpecs() {
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
* 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; }
void setWebServices (String name) { this.wsName = name; }
public String getRedirectPath() { return this.redirectPath; }
void setHostClass(Class<?> cls) {
@ -129,10 +137,32 @@ 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);
}
// Add in the web services filters/serves if app has them.
// Using Jersey/guice integration module. If user has web services
// they must have also bound a default one in their webapp code.
if (this.wsName != null) {
// There seems to be an issue with the guice/jersey integration
// where we have to list the stuff we don't want it to serve
// through the guicecontainer. In this case its everything except
// the the web services api prefix. We can't just change the filter
// from /* below - that doesn't work.
String regex = "(?!/" + this.wsName + ")";
serveRegex(regex).with(DefaultWrapperServlet.class);
Map<String, String> params = new HashMap<String, String>();
params.put(ResourceConfig.FEATURE_IMPLICIT_VIEWABLES, "true");
params.put(ServletContainer.FEATURE_FILTER_FORWARD_ON_404, "true");
params.put(FeaturesAndProperties.FEATURE_XMLROOTELEMENT_PROCESSING, "true");
filter("/*").through(GuiceContainer.class, params);
}
}
/**

View File

@ -72,6 +72,7 @@ public class WebApps {
}
final String name;
final String wsName;
final Class<T> api;
final T application;
String bindAddress = "0.0.0.0";
@ -82,10 +83,15 @@ public class WebApps {
private final HashSet<ServletStruct> servlets = new HashSet<ServletStruct>();
private final HashMap<String, Object> attributes = new HashMap<String, Object>();
Builder(String name, Class<T> api, T application) {
Builder(String name, Class<T> api, T application, String wsName) {
this.name = name;
this.api = api;
this.application = application;
this.wsName = wsName;
}
Builder(String name, Class<T> api, T application) {
this(name, api, application, null);
}
public Builder<T> at(String bindAddress) {
@ -142,6 +148,7 @@ public class WebApps {
};
}
webapp.setName(name);
webapp.setWebServices(wsName);
String basePath = "/" + name;
webapp.setRedirectPath(basePath);
if (basePath.equals("/")) {
@ -150,6 +157,14 @@ public class WebApps {
webapp.addServePathSpec(basePath);
webapp.addServePathSpec(basePath + "/*");
}
if (wsName != null && !wsName.equals(basePath)) {
if (wsName.equals("/")) {
webapp.addServePathSpec("/*");
} else {
webapp.addServePathSpec("/" + wsName);
webapp.addServePathSpec("/" + wsName + "/*");
}
}
if (conf == null) {
conf = new Configuration();
}
@ -231,6 +246,20 @@ public class WebApps {
}
}
/**
* Create a new webapp builder.
* @see WebApps for a complete example
* @param <T> application (holding the embedded webapp) type
* @param prefix of the webapp
* @param api the api class for the application
* @param app the application instance
* @param wsPrefix the prefix for the webservice api for this app
* @return a webapp builder
*/
public static <T> Builder<T> $for(String prefix, Class<T> api, T app, String wsPrefix) {
return new Builder<T>(prefix, api, app, wsPrefix);
}
/**
* Create a new webapp builder.
* @see WebApps for a complete example

View File

@ -53,8 +53,8 @@ public class AggregatedLogsBlock extends HtmlBlock {
logEntity = containerId.toString();
}
if (!conf.getBoolean(YarnConfiguration.NM_LOG_AGGREGATION_ENABLED,
YarnConfiguration.DEFAULT_NM_LOG_AGGREGATION_ENABLED)) {
if (!conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED)) {
html.h1()
._("Aggregation is not enabled. Try the nodemanager at " + nodeId)
._();

View File

@ -58,16 +58,6 @@ public class TestRPC {
private static final String EXCEPTION_CAUSE = "exception cause";
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
// @Test
// public void testAvroRPC() throws Exception {
// test(AvroYarnRPC.class.getName());
// }
//
// @Test
// public void testHadoopNativeRPC() throws Exception {
// test(HadoopYarnRPC.class.getName());
// }
@Test
public void testUnknownCall() {
Configuration conf = new Configuration();

View File

@ -1,40 +0,0 @@
@namespace("org.apache.hadoop.yarn")
protocol ResourceTracker {
import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
// ResourceTracker
record NodeID {
int id;
}
record NodeHealthStatus {
boolean isNodeHealthy;
union {string, null} healthReport;
long lastHealthReportTime;
}
record NodeStatus {
NodeID nodeId;
int responseId;
long lastSeen;
map<array<org.apache.hadoop.yarn.Container>> containers;
NodeHealthStatus nodeHealthStatus;
}
record RegistrationResponse {
NodeID nodeID;
union {bytes, null} secretKey;
}
record HeartbeatResponse {
int responseId;
boolean reboot;
array<org.apache.hadoop.yarn.Container> containersToCleanup;
array<org.apache.hadoop.yarn.ApplicationID> appplicationsToCleanup;
}
RegistrationResponse registerNodeManager(string node, org.apache.hadoop.yarn.Resource resource) throws YarnRemoteException;
HeartbeatResponse nodeHeartbeat(NodeStatus nodeStatus) throws YarnRemoteException;
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.api.records;
import java.util.List;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
@ -33,6 +34,9 @@ public interface NodeStatus {
public abstract void setContainersStatuses(
List<ContainerStatus> containersStatuses);
public abstract List<ApplicationId> getKeepAliveApplications();
public abstract void setKeepAliveApplications(List<ApplicationId> appIds);
NodeHealthStatus getNodeHealthStatus();
void setNodeHealthStatus(NodeHealthStatus healthStatus);

View File

@ -23,13 +23,16 @@ import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.ProtoBase;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.NodeHealthStatusPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeHealthStatusProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
@ -37,7 +40,9 @@ import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProtoOrBuilder;
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements NodeStatus {
public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements
NodeStatus {
NodeStatusProto proto = NodeStatusProto.getDefaultInstance();
NodeStatusProto.Builder builder = null;
boolean viaProto = false;
@ -45,6 +50,7 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
private NodeId nodeId = null;
private List<ContainerStatus> containers = null;
private NodeHealthStatus nodeHealthStatus = null;
private List<ApplicationId> keepAliveApplications = null;
public NodeStatusPBImpl() {
builder = NodeStatusProto.newBuilder();
@ -55,15 +61,14 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
viaProto = true;
}
public NodeStatusProto getProto() {
mergeLocalToProto();
public synchronized NodeStatusProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
return proto;
}
private void mergeLocalToBuilder() {
private synchronized void mergeLocalToBuilder() {
if (this.nodeId != null) {
builder.setNodeId(convertToProtoFormat(this.nodeId));
}
@ -73,9 +78,12 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
if (this.nodeHealthStatus != null) {
builder.setNodeHealthStatus(convertToProtoFormat(this.nodeHealthStatus));
}
if (this.keepAliveApplications != null) {
addKeepAliveApplicationsToProto();
}
}
private void mergeLocalToProto() {
private synchronized void mergeLocalToProto() {
if (viaProto)
maybeInitBuilder();
mergeLocalToBuilder();
@ -84,14 +92,14 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
viaProto = true;
}
private void maybeInitBuilder() {
private synchronized void maybeInitBuilder() {
if (viaProto || builder == null) {
builder = NodeStatusProto.newBuilder(proto);
}
viaProto = false;
}
private void addContainersToProto() {
private synchronized void addContainersToProto() {
maybeInitBuilder();
builder.clearContainersStatuses();
if (containers == null)
@ -124,19 +132,53 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
};
builder.addAllContainersStatuses(iterable);
}
private synchronized void addKeepAliveApplicationsToProto() {
maybeInitBuilder();
builder.clearKeepAliveApplications();
if (keepAliveApplications == null)
return;
Iterable<ApplicationIdProto> iterable = new Iterable<ApplicationIdProto>() {
@Override
public Iterator<ApplicationIdProto> iterator() {
return new Iterator<ApplicationIdProto>() {
Iterator<ApplicationId> iter = keepAliveApplications.iterator();
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public ApplicationIdProto next() {
return convertToProtoFormat(iter.next());
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
};
builder.addAllKeepAliveApplications(iterable);
}
@Override
public int getResponseId() {
public synchronized int getResponseId() {
NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
return p.getResponseId();
}
@Override
public void setResponseId(int responseId) {
public synchronized void setResponseId(int responseId) {
maybeInitBuilder();
builder.setResponseId(responseId);
}
@Override
public NodeId getNodeId() {
public synchronized NodeId getNodeId() {
NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
if (this.nodeId != null) {
return this.nodeId;
@ -148,8 +190,9 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
return this.nodeId;
}
@Override
public void setNodeId(NodeId nodeId) {
public synchronized void setNodeId(NodeId nodeId) {
maybeInitBuilder();
if (nodeId == null)
builder.clearNodeId();
@ -158,20 +201,35 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
}
@Override
public List<ContainerStatus> getContainersStatuses() {
public synchronized List<ContainerStatus> getContainersStatuses() {
initContainers();
return this.containers;
}
@Override
public void setContainersStatuses(List<ContainerStatus> containers) {
public synchronized void setContainersStatuses(
List<ContainerStatus> containers) {
if (containers == null) {
builder.clearContainersStatuses();
}
this.containers = containers;
}
@Override
public synchronized List<ApplicationId> getKeepAliveApplications() {
initKeepAliveApplications();
return this.keepAliveApplications;
}
@Override
public synchronized void setKeepAliveApplications(List<ApplicationId> appIds) {
if (appIds == null) {
builder.clearKeepAliveApplications();
}
this.keepAliveApplications = appIds;
}
private void initContainers() {
private synchronized void initContainers() {
if (this.containers != null) {
return;
}
@ -185,8 +243,22 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
}
private synchronized void initKeepAliveApplications() {
if (this.keepAliveApplications != null) {
return;
}
NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
List<ApplicationIdProto> list = p.getKeepAliveApplicationsList();
this.keepAliveApplications = new ArrayList<ApplicationId>();
for (ApplicationIdProto c : list) {
this.keepAliveApplications.add(convertFromProtoFormat(c));
}
}
@Override
public NodeHealthStatus getNodeHealthStatus() {
public synchronized NodeHealthStatus getNodeHealthStatus() {
NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
if (nodeHealthStatus != null) {
return nodeHealthStatus;
@ -199,7 +271,7 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
}
@Override
public void setNodeHealthStatus(NodeHealthStatus healthStatus) {
public synchronized void setNodeHealthStatus(NodeHealthStatus healthStatus) {
maybeInitBuilder();
if (healthStatus == null) {
builder.clearNodeHealthStatus();
@ -231,4 +303,12 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
private ContainerStatusProto convertToProtoFormat(ContainerStatus c) {
return ((ContainerStatusPBImpl)c).getProto();
}
}
private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto c) {
return new ApplicationIdPBImpl(c);
}
private ApplicationIdProto convertToProtoFormat(ApplicationId c) {
return ((ApplicationIdPBImpl)c).getProto();
}
}

View File

@ -34,6 +34,7 @@ message NodeStatusProto {
optional int32 response_id = 2;
repeated ContainerStatusProto containersStatuses = 3;
optional NodeHealthStatusProto nodeHealthStatus = 4;
repeated ApplicationIdProto keep_alive_applications = 5;
}
message RegistrationResponseProto {

Some files were not shown because too many files have changed in this diff Show More