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:
commit
b77c107ccb
|
@ -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)
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -30,6 +30,7 @@
|
|||
<modules>
|
||||
<module>hadoop-hdfs</module>
|
||||
<module>hadoop-hdfs-httpfs</module>
|
||||
<module>hadoop-hdfs/src/contrib/bkjournal</module>
|
||||
</modules>
|
||||
|
||||
<build>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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...
|
||||
|
|
|
@ -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, ""));
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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()).
|
||||
_();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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._()._();
|
||||
}
|
||||
|
|
|
@ -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._()._();
|
||||
}
|
||||
|
|
|
@ -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))._().
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
||||
}
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.";
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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"));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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().
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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().");
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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;
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
._();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue