Merge trunk into HA branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1171315 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2011-09-15 23:07:48 +00:00
commit 6af0c27930
87 changed files with 2691 additions and 1651 deletions

View File

@ -11,6 +11,9 @@ Trunk (unreleased changes)
HADOOP-7607. Simplify the RPC proxy cleanup process. (atm)
HADOOP-7635. RetryInvocationHandler should release underlying resources on
close (atm)
BUGS
HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
@ -18,6 +21,8 @@ Trunk (unreleased changes)
HADOOP-7610. Fix for hadoop debian package (Eric Yang via gkesavan)
HADOOP-7641. Add Apache License to template config files (Eric Yang via atm)
Release 0.23.0 - Unreleased
INCOMPATIBLE CHANGES
@ -583,6 +588,9 @@ Release 0.23.0 - Unreleased
HADOOP-7626. Bugfix for a config generator (Eric Yang via ddas)
HADOOP-7629. Allow immutable FsPermission objects to be used as IPC
parameters. (todd)
Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -44,18 +44,12 @@ public class FsPermission implements Writable {
};
static { // register a ctor
WritableFactories.setFactory(FsPermission.class, FACTORY);
WritableFactories.setFactory(ImmutableFsPermission.class, FACTORY);
}
/** Create an immutable {@link FsPermission} object. */
public static FsPermission createImmutable(short permission) {
return new FsPermission(permission) {
public FsPermission applyUMask(FsPermission umask) {
throw new UnsupportedOperationException();
}
public void readFields(DataInput in) throws IOException {
throw new UnsupportedOperationException();
}
};
return new ImmutableFsPermission(permission);
}
//POSIX permission style
@ -301,4 +295,16 @@ else if (unixSymbolicPermission.length() != 10) {
return new FsPermission((short)n);
}
private static class ImmutableFsPermission extends FsPermission {
public ImmutableFsPermission(short permission) {
super(permission);
}
public FsPermission applyUMask(FsPermission umask) {
throw new UnsupportedOperationException();
}
public void readFields(DataInput in) throws IOException {
throw new UnsupportedOperationException();
}
}
}

View File

@ -17,7 +17,10 @@
*/
package org.apache.hadoop.io.retry;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.ipc.RPC;
/**
* An implementation of {@link FailoverProxyProvider} which does nothing in the
@ -49,4 +52,9 @@ public void performFailover(Object currentProxy) {
// Nothing to do.
}
@Override
public void close() throws IOException {
RPC.stopProxy(proxy);
}
}

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.io.retry;
import java.io.Closeable;
import org.apache.hadoop.classification.InterfaceStability;
/**
@ -27,7 +29,7 @@
* {@link RetryPolicy}.
*/
@InterfaceStability.Evolving
public interface FailoverProxyProvider {
public interface FailoverProxyProvider extends Closeable {
/**
* Get the proxy object which should be used until the next failover event

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.io.retry;
import java.io.Closeable;
import java.io.IOException;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
@ -27,7 +29,7 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
class RetryInvocationHandler implements InvocationHandler {
class RetryInvocationHandler implements InvocationHandler, Closeable {
public static final Log LOG = LogFactory.getLog(RetryInvocationHandler.class);
private FailoverProxyProvider proxyProvider;
@ -103,4 +105,9 @@ private Object invokeMethod(Method method, Object[] args) throws Throwable {
}
}
@Override
public void close() throws IOException {
proxyProvider.close();
}
}

View File

@ -1,5 +1,25 @@
<?xml version="1.0"?>
<!--
Copyright 2011 The Apache Software Foundation
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<!-- This is the configuration file for the resource manager in Hadoop. -->
<!-- You can configure various scheduling parameters related to queues. -->
<!-- The properties for a queue follow a naming convention,such as, -->

View File

@ -1,3 +1,21 @@
# Copyright 2011 The Apache Software Foundation
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#Logging Implementation
#Log4J

View File

@ -1,6 +1,26 @@
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
Copyright 2011 The Apache Software Foundation
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<!-- Put site-specific property overrides in this file. -->
<configuration>

View File

@ -1,3 +1,21 @@
# Copyright 2011 The Apache Software Foundation
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# Set Hadoop-specific environment variables here.
# The only required environment variable is JAVA_HOME. All others are

View File

@ -1,5 +1,26 @@
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
Copyright 2011 The Apache Software Foundation
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<!-- Put site-specific property overrides in this file. -->

View File

@ -1,5 +1,26 @@
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
Copyright 2011 The Apache Software Foundation
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<configuration>

View File

@ -1,5 +1,26 @@
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
Copyright 2011 The Apache Software Foundation
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<configuration>
<property>
<name>mapred.queue.default.acl-submit-job</name>

View File

@ -1,5 +1,26 @@
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
Copyright 2011 The Apache Software Foundation
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<!-- Put site-specific property overrides in this file. -->

View File

@ -1,3 +1,21 @@
# Copyright 2011 The Apache Software Foundation
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
mapreduce.cluster.local.dir=${HADOOP_MAPRED_DIR}
mapreduce.tasktracker.group=${HADOOP_GROUP}
hadoop.log.dir=${HADOOP_LOG_DIR}/${HADOOP_MR_USER}

View File

@ -58,6 +58,11 @@ public Class<?> getInterface() {
return iface;
}
@Override
public void close() throws IOException {
// Nothing to do.
}
}
public static class FailOverOnceOnAnyExceptionPolicy implements RetryPolicy {

View File

@ -18,28 +18,38 @@
package org.apache.hadoop.ipc;
import java.io.Closeable;
import java.io.IOException;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.lang.management.ManagementFactory;
import java.lang.management.ThreadInfo;
import java.lang.management.ThreadMXBean;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.util.Arrays;
import junit.framework.TestCase;
import javax.net.SocketFactory;
import org.apache.commons.logging.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.io.UTF8;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.authorize.AuthorizationException;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.Service;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.junit.Test;
import static org.junit.Assert.*;
import com.google.protobuf.DescriptorProtos;
import com.google.protobuf.DescriptorProtos.EnumDescriptorProto;
@ -49,7 +59,8 @@
import static org.mockito.Mockito.*;
/** Unit tests for RPC. */
public class TestRPC extends TestCase {
@SuppressWarnings("deprecation")
public class TestRPC {
private static final String ADDRESS = "0.0.0.0";
public static final Log LOG =
@ -57,11 +68,14 @@ public class TestRPC extends TestCase {
private static Configuration conf = new Configuration();
static {
conf.setClass("rpc.engine." + StoppedProtocol.class.getName(),
StoppedRpcEngine.class, RpcEngine.class);
}
int datasize = 1024*100;
int numThreads = 50;
public TestRPC(String name) { super(name); }
public interface TestProtocol extends VersionedProtocol {
public static final long versionID = 1L;
@ -207,6 +221,74 @@ public void run() {
}
}
/**
* A basic interface for testing client-side RPC resource cleanup.
*/
private static interface StoppedProtocol {
long versionID = 0;
public void stop();
}
/**
* A class used for testing cleanup of client side RPC resources.
*/
private static class StoppedRpcEngine implements RpcEngine {
@Override
public Object[] call(Method method, Object[][] params, InetSocketAddress[] addrs,
UserGroupInformation ticket, Configuration conf)
throws IOException, InterruptedException {
return null;
}
@SuppressWarnings("unchecked")
@Override
public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
SocketFactory factory, int rpcTimeout) throws IOException {
T proxy = (T) Proxy.newProxyInstance(protocol.getClassLoader(),
new Class[] { protocol }, new StoppedInvocationHandler());
return new ProtocolProxy<T>(protocol, proxy, false);
}
@Override
public org.apache.hadoop.ipc.RPC.Server getServer(Class<?> protocol,
Object instance, String bindAddress, int port, int numHandlers,
int numReaders, int queueSizePerHandler, boolean verbose, Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager) throws IOException {
return null;
}
}
/**
* An invocation handler which does nothing when invoking methods, and just
* counts the number of times close() is called.
*/
private static class StoppedInvocationHandler
implements InvocationHandler, Closeable {
private int closeCalled = 0;
@Override
public Object invoke(Object proxy, Method method, Object[] args)
throws Throwable {
return null;
}
@Override
public void close() throws IOException {
closeCalled++;
}
public int getCloseCalled() {
return closeCalled;
}
}
@Test
public void testConfRpc() throws Exception {
Server server = RPC.getServer(TestProtocol.class,
new TestImpl(), ADDRESS, 0, 1, false, conf, null);
@ -229,6 +311,7 @@ public void testConfRpc() throws Exception {
server.stop();
}
@Test
public void testSlowRpc() throws Exception {
System.out.println("Testing Slow RPC");
// create a server with two handlers
@ -273,11 +356,12 @@ public void testSlowRpc() throws Exception {
}
}
public void testRPCConf(Configuration conf) throws Exception {
@Test
public void testCalls() throws Exception {
testCallsInternal(conf);
}
public void testCalls(Configuration conf) throws Exception {
private void testCallsInternal(Configuration conf) throws Exception {
Server server = RPC.getServer(TestProtocol.class,
new TestImpl(), ADDRESS, 0, conf);
TestProtocol proxy = null;
@ -384,6 +468,7 @@ public void testCalls(Configuration conf) throws Exception {
}
}
@Test
public void testStandaloneClient() throws IOException {
try {
TestProtocol proxy = RPC.waitForProxy(TestProtocol.class,
@ -450,6 +535,7 @@ private void doRPCs(Configuration conf, boolean expectFailure) throws Exception
}
}
@Test
public void testAuthorization() throws Exception {
Configuration conf = new Configuration();
conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
@ -481,20 +567,48 @@ public void testNoPings() throws Exception {
Configuration conf = new Configuration();
conf.setBoolean("ipc.client.ping", false);
new TestRPC("testnoPings").testCalls(conf);
new TestRPC().testCallsInternal(conf);
conf.setInt(CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, 2);
new TestRPC("testnoPings").testCalls(conf);
new TestRPC().testCallsInternal(conf);
}
/**
* Test stopping a non-registered proxy
* @throws Exception
*/
@Test
public void testStopNonRegisteredProxy() throws Exception {
RPC.stopProxy(mock(TestProtocol.class));
}
@Test
public void testStopProxy() throws IOException {
StoppedProtocol proxy = (StoppedProtocol) RPC.getProxy(StoppedProtocol.class,
StoppedProtocol.versionID, null, conf);
StoppedInvocationHandler invocationHandler = (StoppedInvocationHandler)
Proxy.getInvocationHandler(proxy);
assertEquals(invocationHandler.getCloseCalled(), 0);
RPC.stopProxy(proxy);
assertEquals(invocationHandler.getCloseCalled(), 1);
}
@Test
public void testWrappedStopProxy() throws IOException {
StoppedProtocol wrappedProxy = (StoppedProtocol) RPC.getProxy(StoppedProtocol.class,
StoppedProtocol.versionID, null, conf);
StoppedInvocationHandler invocationHandler = (StoppedInvocationHandler)
Proxy.getInvocationHandler(wrappedProxy);
StoppedProtocol proxy = (StoppedProtocol) RetryProxy.create(StoppedProtocol.class,
wrappedProxy, RetryPolicies.RETRY_FOREVER);
assertEquals(invocationHandler.getCloseCalled(), 0);
RPC.stopProxy(proxy);
assertEquals(invocationHandler.getCloseCalled(), 1);
}
@Test
public void testErrorMsgForInsecureClient() throws Exception {
final Server server = RPC.getServer(TestProtocol.class,
new TestImpl(), ADDRESS, 0, 5, true, conf, null);
@ -567,10 +681,10 @@ private static int countThreads(String search) {
return count;
}
/**
* Test that server.stop() properly stops all threads
*/
@Test
public void testStopsAllThreads() throws Exception {
int threadsBefore = countThreads("Server$Listener$Reader");
assertEquals("Expect no Reader threads running before test",
@ -591,8 +705,7 @@ public void testStopsAllThreads() throws Exception {
}
public static void main(String[] args) throws Exception {
new TestRPC("test").testCalls(conf);
new TestRPC().testCallsInternal(conf);
}
}

View File

@ -35,6 +35,12 @@ Trunk (unreleased changes)
HDFS-2314. MRV1 test compilation broken after HDFS-2197 (todd)
HDFS-2331. Fix WebHdfsFileSystem compilation problems for a bug in JDK
version < 1.6.0_26. (Abhijit Suresh Shingate via szetszwo)
HDFS-2333. Change DFSOutputStream back to package private, otherwise,
there are two SC_START_IN_CTOR findbugs warnings. (szetszwo)
Release 0.23.0 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -42,6 +42,7 @@
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsServerDefaults;
@ -79,7 +80,6 @@
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.io.DataOutputBuffer;
@ -797,12 +797,20 @@ private DFSOutputStream callAppend(HdfsFileStatus stat, String src,
*
* @param src file name
* @param buffersize buffer size
* @param progress for reporting write-progress
* @param progress for reporting write-progress; null is acceptable.
* @param statistics file system statistics; null is acceptable.
* @return an output stream for writing into the file
*
* @see ClientProtocol#append(String, String)
*/
public DFSOutputStream append(String src, int buffersize, Progressable progress)
public FSDataOutputStream append(final String src, final int buffersize,
final Progressable progress, final FileSystem.Statistics statistics
) throws IOException {
final DFSOutputStream out = append(src, buffersize, progress);
return new FSDataOutputStream(out, statistics, out.getInitialLen());
}
private DFSOutputStream append(String src, int buffersize, Progressable progress)
throws IOException {
checkOpen();
HdfsFileStatus stat = getFileInfo(src);

View File

@ -36,6 +36,7 @@
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.fs.FileAlreadyExistsException;
@ -98,7 +99,8 @@
* datanode from the original pipeline. The DataStreamer now
* starts sending packets from the dataQueue.
****************************************************************/
public class DFSOutputStream extends FSOutputSummer implements Syncable {
@InterfaceAudience.Private
class DFSOutputStream extends FSOutputSummer implements Syncable {
private final DFSClient dfsClient;
private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
private Socket s;
@ -1537,7 +1539,7 @@ public synchronized void hsync() throws IOException {
* write pipeline have failed.
* @return the number of valid replicas of the current block
*/
public synchronized int getNumCurrentReplicas() throws IOException {
synchronized int getNumCurrentReplicas() throws IOException {
dfsClient.checkOpen();
isClosed();
if (streamer == null) {
@ -1707,7 +1709,7 @@ synchronized void setTestFilename(String newname) {
/**
* Returns the size of a file as it was when this stream was opened
*/
public long getInitialLen() {
long getInitialLen() {
return initialFileSize;
}

View File

@ -240,10 +240,8 @@ public FSDataInputStream open(Path f, int bufferSize) throws IOException {
@Override
public FSDataOutputStream append(Path f, int bufferSize,
Progressable progress) throws IOException {
statistics.incrementWriteOps(1);
final DFSOutputStream op = dfs.append(getPathName(f), bufferSize, progress);
return new FSDataOutputStream(op, statistics, op.getInitialLen());
return dfs.append(getPathName(f), bufferSize, progress, statistics);
}
@Override

View File

@ -47,7 +47,6 @@
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@ -158,10 +157,8 @@ public Response post(
final Configuration conf = new Configuration(datanode.getConf());
final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
final DFSOutputStream dfsout = dfsclient.append(fullpath,
bufferSize.getValue(), null);
final FSDataOutputStream out = new FSDataOutputStream(dfsout, null,
dfsout.getInitialLen());
final FSDataOutputStream out = dfsclient.append(fullpath,
bufferSize.getValue(), null, null);
try {
IOUtils.copyBytes(in, out, bufferSize.getValue());
} finally {

View File

@ -206,7 +206,7 @@ private <T> T run(final HttpOpParam.Op op, final Path fspath,
final HttpURLConnection conn = httpConnect(op, fspath, parameters);
validateResponse(op, conn);
try {
return jsonParse(conn.getInputStream());
return WebHdfsFileSystem.<T>jsonParse(conn.getInputStream());
} finally {
conn.disconnect();
}

View File

@ -286,6 +286,18 @@ Release 0.23.0 - Unreleased
org.apache.hadoop.yarn.api.records.* to be get/set only. Added javadocs to
all public records. (acmurthy)
MAPREDUCE-2676. MR-279: JobHistory Job page needs reformatted. (Robert Evans via
mahadev)
MAPREDUCE-2899. Replace major parts of ApplicationSubmissionContext with a
ContainerLaunchContext (Arun Murthy via mahadev)
MAPREDUCE-2966. Added ShutDown hooks for MRV2 processes so that they can
gracefully exit. (Abhijit Suresh Shingate via vinodkv)
MAPREDUCE-2672. MR-279: JobHistory Server needs Analysis this job.
(Robert Evans via mahadev)
OPTIMIZATIONS
MAPREDUCE-2026. Make JobTracker.getJobCounters() and
@ -1312,6 +1324,15 @@ Release 0.23.0 - Unreleased
MAPREDUCE-2995. Fixed race condition in ContainerLauncher. (vinodkv via
acmurthy)
MAPREDUCE-2949. Fixed NodeManager to shut-down correctly if a service
startup fails. (Ravi Teja via vinodkv)
MAPREDUCE-3005. Fix both FifoScheduler and CapacityScheduler to correctly
enforce locality constraints. (acmurthy)
MAPREDUCE-3007. Fixed Yarn Mapreduce client to be able to connect to
JobHistoryServer in secure mode. (vinodkv)
Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES
@ -1888,6 +1909,9 @@ Release 0.22.0 - Unreleased
MAPREDUCE-2767. Remove Linux task-controller. (Milind Bhandarkar via shv)
MAPREDUCE-2991. queueinfo.jsp fails to show queue status for Capacity
scheduler if queue names contain special symbols. (Priyo Mustafi via shv)
Release 0.21.1 - Unreleased
NEW FEATURES

View File

@ -559,12 +559,14 @@ public void handle(SpeculatorEvent event) {
public static void main(String[] args) {
try {
//Configuration.addDefaultResource("job.xml");
ApplicationId applicationId = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class);
ApplicationId applicationId = RecordFactoryProvider
.getRecordFactory(null).newRecordInstance(ApplicationId.class);
applicationId.setClusterTimestamp(Long.valueOf(args[0]));
applicationId.setId(Integer.valueOf(args[1]));
int failCount = Integer.valueOf(args[2]);
MRAppMaster appMaster = new MRAppMaster(applicationId, failCount);
Runtime.getRuntime().addShutdownHook(
new CompositeServiceShutdownHook(appMaster));
YarnConfiguration conf = new YarnConfiguration(new JobConf());
conf.addResource(new Path(MRConstants.JOB_CONF_FILE));
conf.set(MRJobConfig.USER_NAME,
@ -573,7 +575,7 @@ public static void main(String[] args) {
appMaster.init(conf);
appMaster.start();
} catch (Throwable t) {
LOG.error("Caught throwable. Exiting:", t);
LOG.fatal("Error starting MRAppMaster", t);
System.exit(1);
}
}

View File

@ -21,6 +21,7 @@
import java.util.List;
import java.util.Map;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@ -30,6 +31,7 @@
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
/**
@ -53,6 +55,16 @@ public interface Job {
boolean isUber();
String getUserName();
/**
* @return a path to where the config file for this job is located.
*/
Path getConfFile();
/**
* @return the ACLs for this job for each type of JobACL given.
*/
Map<JobACL, AccessControlList> getJobACLs();
TaskAttemptCompletionEvent[]
getTaskAttemptCompletionEvents(int fromEventId, int maxEvents);

View File

@ -38,32 +38,51 @@ public interface TaskAttempt {
float getProgress();
TaskAttemptState getState();
/** Has attempt reached the final state or not.
/**
* Has attempt reached the final state or not.
* @return true if it has finished, else false
*/
boolean isFinished();
/**If container Assigned then return container ID, otherwise null.
/**
* @return the container ID if a container is assigned, otherwise null.
*/
ContainerId getAssignedContainerID();
/**If container Assigned then return container mgr address, otherwise null.
/**
* @return container mgr address if a container is assigned, otherwise null.
*/
String getAssignedContainerMgrAddress();
/**If container Assigned then return the node's http address, otherwise null.
/**
* @return node's http address if a container is assigned, otherwise null.
*/
String getNodeHttpAddress();
/** Returns time at which container is launched. If container is not launched
/**
* @return time at which container is launched. If container is not launched
* yet, returns 0.
*/
long getLaunchTime();
/** Returns attempt's finish time. If attempt is not finished
/**
* @return attempt's finish time. If attempt is not finished
* yet, returns 0.
*/
long getFinishTime();
/**
* @return The attempt's shuffle finish time if the attempt is a reduce. If
* attempt is not finished yet, returns 0.
*/
long getShuffleFinishTime();
/**
* @return The attempt's sort or merge finish time if the attempt is a reduce.
* If attempt is not finished yet, returns 0.
*/
long getSortFinishTime();
/**
* @return the port shuffle is on.
*/

View File

@ -772,6 +772,15 @@ public String getUserName() {
return userName;
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile()
*/
@Override
public Path getConfFile() {
return remoteJobConfFile;
}
@Override
public String getName() {
return jobName;
@ -788,6 +797,15 @@ public int getTotalReduces() {
return reduceTasks.size(); //FIXME: why indirection? return numReduceTasks
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.job.Job#getJobACLs()
*/
@Override
public Map<JobACL, AccessControlList> getJobACLs() {
return Collections.unmodifiableMap(jobACLs);
}
public static class InitTransition
implements MultipleArcTransition<JobImpl, JobEvent, JobState> {

View File

@ -579,13 +579,12 @@ private ContainerLaunchContext createContainerLaunchContext() {
+ remoteJobConfPath.toUri().toASCIIString());
// //////////// End of JobConf setup
// Setup DistributedCache
setupDistributedCache(remoteFS, conf, localResources, environment);
MRApps.setupDistributedCache(conf, localResources, environment);
// Set local-resources and environment
container.setLocalResources(localResources);
container.setEnv(environment);
container.setEnvironment(environment);
// Setup up tokens
Credentials taskCredentials = new Credentials();
@ -618,7 +617,7 @@ private ContainerLaunchContext createContainerLaunchContext() {
ShuffleHandler.serializeServiceData(jobToken));
container.setServiceData(serviceData);
MRApps.addToClassPath(container.getEnv(), getInitialClasspath());
MRApps.addToClassPath(container.getEnvironment(), getInitialClasspath());
} catch (IOException e) {
throw new YarnException(e);
}
@ -645,7 +644,7 @@ private ContainerLaunchContext createContainerLaunchContext() {
taskAttemptListener.getAddress(), remoteTask, javaHome,
workDir.toString(), containerLogDir, childTmpDir, jvmID));
MapReduceChildJVM.setVMEnv(container.getEnv(), classPaths,
MapReduceChildJVM.setVMEnv(container.getEnvironment(), classPaths,
workDir.toString(), containerLogDir, nmLdLibraryPath, remoteTask,
localizedApplicationTokensFile);
@ -656,116 +655,6 @@ private ContainerLaunchContext createContainerLaunchContext() {
return container;
}
private static long[] parseTimeStamps(String[] strs) {
if (null == strs) {
return null;
}
long[] result = new long[strs.length];
for(int i=0; i < strs.length; ++i) {
result[i] = Long.parseLong(strs[i]);
}
return result;
}
private void setupDistributedCache(FileSystem remoteFS,
Configuration conf,
Map<String, LocalResource> localResources,
Map<String, String> env)
throws IOException {
// Cache archives
parseDistributedCacheArtifacts(remoteFS, localResources, env,
LocalResourceType.ARCHIVE,
DistributedCache.getCacheArchives(conf),
parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)),
getFileSizes(conf, MRJobConfig.CACHE_ARCHIVES_SIZES),
DistributedCache.getArchiveVisibilities(conf),
DistributedCache.getArchiveClassPaths(conf));
// Cache files
parseDistributedCacheArtifacts(remoteFS,
localResources, env,
LocalResourceType.FILE,
DistributedCache.getCacheFiles(conf),
parseTimeStamps(DistributedCache.getFileTimestamps(conf)),
getFileSizes(conf, MRJobConfig.CACHE_FILES_SIZES),
DistributedCache.getFileVisibilities(conf),
DistributedCache.getFileClassPaths(conf));
}
// TODO - Move this to MR!
// Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[],
// long[], boolean[], Path[], FileType)
private void parseDistributedCacheArtifacts(
FileSystem remoteFS,
Map<String, LocalResource> localResources,
Map<String, String> env,
LocalResourceType type,
URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[],
Path[] pathsToPutOnClasspath) throws IOException {
if (uris != null) {
// Sanity check
if ((uris.length != timestamps.length) || (uris.length != sizes.length) ||
(uris.length != visibilities.length)) {
throw new IllegalArgumentException("Invalid specification for " +
"distributed-cache artifacts of type " + type + " :" +
" #uris=" + uris.length +
" #timestamps=" + timestamps.length +
" #visibilities=" + visibilities.length
);
}
Map<String, Path> classPaths = new HashMap<String, Path>();
if (pathsToPutOnClasspath != null) {
for (Path p : pathsToPutOnClasspath) {
p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
remoteFS.getWorkingDirectory()));
classPaths.put(p.toUri().getPath().toString(), p);
}
}
for (int i = 0; i < uris.length; ++i) {
URI u = uris[i];
Path p = new Path(u);
p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
remoteFS.getWorkingDirectory()));
// Add URI fragment or just the filename
Path name = new Path((null == u.getFragment())
? p.getName()
: u.getFragment());
if (name.isAbsolute()) {
throw new IllegalArgumentException("Resource name must be relative");
}
String linkName = name.toUri().getPath();
localResources.put(
linkName,
BuilderUtils.newLocalResource(
p.toUri(), type,
visibilities[i]
? LocalResourceVisibility.PUBLIC
: LocalResourceVisibility.PRIVATE,
sizes[i], timestamps[i])
);
if (classPaths.containsKey(u.getPath())) {
MRApps.addToClassPath(env, linkName);
}
}
}
}
// TODO - Move this to MR!
private static long[] getFileSizes(Configuration conf, String key) {
String[] strs = conf.getStrings(key);
if (strs == null) {
return null;
}
long[] result = new long[strs.length];
for(int i=0; i < strs.length; ++i) {
result[i] = Long.parseLong(strs[i]);
}
return result;
}
@Override
public ContainerId getAssignedContainerID() {
readLock.lock();
@ -806,6 +695,25 @@ public long getFinishTime() {
}
}
@Override
public long getShuffleFinishTime() {
readLock.lock();
try {
return this.reportedStatus.shuffleFinishTime;
} finally {
readLock.unlock();
}
}
@Override
public long getSortFinishTime() {
readLock.lock();
try {
return this.reportedStatus.sortFinishTime;
} finally {
readLock.unlock();
}
}
@Override
public int getShufflePort() {
@ -862,6 +770,7 @@ public TaskAttemptReport getReport() {
result.setProgress(reportedStatus.progress);
result.setStartTime(launchTime);
result.setFinishTime(finishTime);
result.setShuffleFinishTime(this.reportedStatus.shuffleFinishTime);
result.setDiagnosticInfo(reportedStatus.diagnosticInfo);
result.setPhase(reportedStatus.phase);
result.setStateString(reportedStatus.stateString);

View File

@ -177,11 +177,12 @@ public void attempts() {
}
setTitle(join(attemptState, " ",
MRApps.taskType(taskType).toString(), " attempts in ", $(JOB_ID)));
render(attemptsPage());
} catch (Exception e) {
badRequest(e.getMessage());
}
}
render(attemptsPage());
}
/**
@ -205,7 +206,7 @@ void notFound(String s) {
/**
* Ensure that a JOB_ID was passed into the page.
*/
void requireJob() {
public void requireJob() {
try {
if ($(JOB_ID).isEmpty()) {
throw new RuntimeException("missing job ID");
@ -216,14 +217,15 @@ void requireJob() {
notFound($(JOB_ID));
}
} catch (Exception e) {
badRequest(e.getMessage() == null ? e.getClass().getName() : e.getMessage());
badRequest(e.getMessage() == null ?
e.getClass().getName() : e.getMessage());
}
}
/**
* Ensure that a TASK_ID was passed into the page.
*/
void requireTask() {
public void requireTask() {
try {
if ($(TASK_ID).isEmpty()) {
throw new RuntimeException("missing task ID");

View File

@ -0,0 +1,110 @@
/**
* 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 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.util.MRApps;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import com.google.inject.Inject;
/**
* Render the configuration for this job.
*/
public class ConfBlock extends HtmlBlock {
final AppContext appContext;
final Configuration conf;
@Inject ConfBlock(AppContext appctx, Configuration conf) {
appContext = appctx;
this.conf = conf;
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block)
*/
@Override protected void render(Block html) {
String jid = $(JOB_ID);
if (jid.isEmpty()) {
html.
p()._("Sorry, can't do anything without a JobID.")._();
return;
}
JobId jobID = MRApps.toJobID(jid);
Job job = appContext.getJob(jobID);
if (job == null) {
html.
p()._("Sorry, ", jid, " not found.")._();
return;
}
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));
html.div().h3(confPath.toString())._();
TBODY<TABLE<Hamlet>> tbody = html.
// Tasks table
table("#conf").
thead().
tr().
th(_TH, "key").
th(_TH, "value").
_().
_().
tbody();
for(Map.Entry<String, String> entry : jobConf) {
tbody.
tr().
td(entry.getKey()).
td(entry.getValue()).
_();
}
tbody._().
tfoot().
tr().
th().input("search_init").$type(InputType.text).$name("key").$value("key")._()._().
th().input("search_init").$type(InputType.text).$name("value").$value("value")._()._().
_().
_().
_();
} catch(IOException e) {
LOG.error("Error while reading "+confPath, e);
html.p()._("Sorry got an error while reading conf file. ",confPath);
}
}
}

View File

@ -22,10 +22,12 @@
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.ShuffleHandler;
import org.apache.hadoop.mapreduce.FileSystemCounter;
import org.apache.hadoop.mapreduce.JobACL;
@ -50,6 +52,7 @@
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -258,6 +261,16 @@ public List<String> getDiagnostics() {
public String getAssignedContainerMgrAddress() {
return "localhost:9998";
}
@Override
public long getShuffleFinishTime() {
return 0;
}
@Override
public long getSortFinishTime() {
return 0;
}
};
}
@ -452,7 +465,7 @@ public Map<TaskId, Task> getTasks(TaskType taskType) {
@Override
public List<String> getDiagnostics() {
throw new UnsupportedOperationException("Not supported yet.");
return Collections.<String>emptyList();
}
@Override
@ -463,8 +476,18 @@ public boolean checkAccess(UserGroupInformation callerUGI,
@Override
public String getUserName() {
return "mock";
}
@Override
public Path getConfFile() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public Map<JobACL, AccessControlList> getJobACLs() {
return Collections.<JobACL, AccessControlList>emptyMap();
}
};
}
}

View File

@ -31,6 +31,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@ -58,6 +59,7 @@
import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
import org.apache.hadoop.mapreduce.v2.app.speculate.TaskRuntimeEstimator;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.SystemClock;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@ -461,6 +463,16 @@ public boolean checkAccess(UserGroupInformation callerUGI,
public String getUserName() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public Path getConfFile() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public Map<JobACL, AccessControlList> getJobACLs() {
throw new UnsupportedOperationException("Not supported yet.");
}
}
/*
@ -681,6 +693,16 @@ public long getFinishTime() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public long getShuffleFinishTime() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public long getSortFinishTime() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public String getAssignedContainerMgrAddress() {
throw new UnsupportedOperationException("Not supported yet.");

View File

@ -21,12 +21,17 @@
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID;
import static org.junit.Assert.assertEquals;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
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.AppContext;
import org.apache.hadoop.mapreduce.v2.app.MockJobs;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -115,18 +120,42 @@ public long getStartTime() {
WebAppTests.testPage(AppView.class, AppContext.class, new TestAppContext());
}
@Test public void testJobView() {
WebAppTests.testPage(JobPage.class, AppContext.class, new TestAppContext());
AppContext appContext = new TestAppContext();
Map<String, String> params = getJobParams(appContext);
WebAppTests.testPage(JobPage.class, AppContext.class, appContext, params);
}
@Test public void testTasksView() {
WebAppTests.testPage(TasksPage.class, AppContext.class,
new TestAppContext());
AppContext appContext = new TestAppContext();
Map<String, String> params = getTaskParams(appContext);
WebAppTests.testPage(TasksPage.class, AppContext.class, appContext, params);
}
@Test public void testTaskView() {
WebAppTests.testPage(TaskPage.class, AppContext.class,
new TestAppContext());
AppContext appContext = new TestAppContext();
Map<String, String> params = getTaskParams(appContext);
WebAppTests.testPage(TaskPage.class, AppContext.class, appContext, params);
}
public static Map<String, String> getJobParams(AppContext appContext) {
JobId jobId = appContext.getAllJobs().entrySet().iterator().next().getKey();
Map<String, String> params = new HashMap<String, String>();
params.put(AMParams.JOB_ID, MRApps.toString(jobId));
return params;
}
public static Map<String, String> getTaskParams(AppContext appContext) {
JobId jobId = appContext.getAllJobs().entrySet().iterator().next().getKey();
Entry<TaskId, Task> e = appContext.getJob(jobId).getTasks().entrySet().iterator().next();
e.getValue().getType();
Map<String, String> params = new HashMap<String, String>();
params.put(AMParams.JOB_ID, MRApps.toString(jobId));
params.put(AMParams.TASK_ID, e.getKey().toString());
params.put(AMParams.TASK_TYPE, MRApps.taskSymbol(e.getValue().getType()));
return params;
}
public static void main(String[] args) {

View File

@ -24,6 +24,10 @@ public interface TaskAttemptReport {
public abstract float getProgress();
public abstract long getStartTime();
public abstract long getFinishTime();
/** @return the shuffle finish time. Applicable only for reduce attempts */
public abstract long getShuffleFinishTime();
/** @return the sort/merge finish time. Applicable only for reduce attempts */
public abstract long getSortFinishTime();
public abstract Counters getCounters();
public abstract String getDiagnosticInfo();
public abstract String getStateString();
@ -39,4 +43,14 @@ public interface TaskAttemptReport {
public abstract void setStateString(String stateString);
public abstract void setPhase(Phase phase);
/**
* Set the shuffle finish time. Applicable only for reduce attempts
* @param time the time the shuffle finished.
*/
public abstract void setShuffleFinishTime(long time);
/**
* Set the sort/merge finish time. Applicable only for reduce attempts
* @param time the time the shuffle finished.
*/
public abstract void setSortFinishTime(long time);
}

View File

@ -127,6 +127,31 @@ public void setFinishTime(long finishTime) {
maybeInitBuilder();
builder.setFinishTime((finishTime));
}
@Override
public long getShuffleFinishTime() {
TaskAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
return (p.getShuffleFinishTime());
}
@Override
public void setShuffleFinishTime(long time) {
maybeInitBuilder();
builder.setShuffleFinishTime(time);
}
@Override
public long getSortFinishTime() {
TaskAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
return (p.getSortFinishTime());
}
@Override
public void setSortFinishTime(long time) {
maybeInitBuilder();
builder.setSortFinishTime(time);
}
@Override
public TaskAttemptId getTaskAttemptId() {
TaskAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
@ -262,7 +287,4 @@ private PhaseProto convertToProtoFormat(Phase e) {
private Phase convertFromProtoFormat(PhaseProto e) {
return MRProtoUtils.convertFromProtoFormat(e);
}
}

View File

@ -132,6 +132,7 @@ public void setStartTime(long startTime) {
maybeInitBuilder();
builder.setStartTime((startTime));
}
@Override
public long getFinishTime() {
TaskReportProtoOrBuilder p = viaProto ? proto : builder;
@ -143,6 +144,7 @@ public void setFinishTime(long finishTime) {
maybeInitBuilder();
builder.setFinishTime((finishTime));
}
@Override
public TaskId getTaskId() {
TaskReportProtoOrBuilder p = viaProto ? proto : builder;

View File

@ -25,14 +25,20 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.net.URI;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.mapreduce.v2.MRConstants;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@ -42,12 +48,18 @@
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.util.Apps;
import org.apache.hadoop.yarn.util.BuilderUtils;
/**
* Helper class for MR applications
*/
@Private
@Unstable
public class MRApps extends Apps {
public static final String JOB = "job";
public static final String TASK = "task";
@ -232,4 +244,121 @@ public static String getJobFile(Configuration conf, String user,
jobId.toString() + Path.SEPARATOR + MRConstants.JOB_CONF_FILE);
return jobFile.toString();
}
private static long[] parseTimeStamps(String[] strs) {
if (null == strs) {
return null;
}
long[] result = new long[strs.length];
for(int i=0; i < strs.length; ++i) {
result[i] = Long.parseLong(strs[i]);
}
return result;
}
public static void setupDistributedCache(
Configuration conf,
Map<String, LocalResource> localResources,
Map<String, String> env)
throws IOException {
// Cache archives
parseDistributedCacheArtifacts(conf, localResources, env,
LocalResourceType.ARCHIVE,
DistributedCache.getCacheArchives(conf),
parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)),
getFileSizes(conf, MRJobConfig.CACHE_ARCHIVES_SIZES),
DistributedCache.getArchiveVisibilities(conf),
DistributedCache.getArchiveClassPaths(conf));
// Cache files
parseDistributedCacheArtifacts(conf,
localResources, env,
LocalResourceType.FILE,
DistributedCache.getCacheFiles(conf),
parseTimeStamps(DistributedCache.getFileTimestamps(conf)),
getFileSizes(conf, MRJobConfig.CACHE_FILES_SIZES),
DistributedCache.getFileVisibilities(conf),
DistributedCache.getFileClassPaths(conf));
}
// TODO - Move this to MR!
// Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[],
// long[], boolean[], Path[], FileType)
private static void parseDistributedCacheArtifacts(
Configuration conf,
Map<String, LocalResource> localResources,
Map<String, String> env,
LocalResourceType type,
URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[],
Path[] pathsToPutOnClasspath) throws IOException {
if (uris != null) {
// Sanity check
if ((uris.length != timestamps.length) || (uris.length != sizes.length) ||
(uris.length != visibilities.length)) {
throw new IllegalArgumentException("Invalid specification for " +
"distributed-cache artifacts of type " + type + " :" +
" #uris=" + uris.length +
" #timestamps=" + timestamps.length +
" #visibilities=" + visibilities.length
);
}
Map<String, Path> classPaths = new HashMap<String, Path>();
if (pathsToPutOnClasspath != null) {
for (Path p : pathsToPutOnClasspath) {
FileSystem remoteFS = p.getFileSystem(conf);
p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
remoteFS.getWorkingDirectory()));
classPaths.put(p.toUri().getPath().toString(), p);
}
}
for (int i = 0; i < uris.length; ++i) {
URI u = uris[i];
Path p = new Path(u);
FileSystem remoteFS = p.getFileSystem(conf);
p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
remoteFS.getWorkingDirectory()));
// Add URI fragment or just the filename
Path name = new Path((null == u.getFragment())
? p.getName()
: u.getFragment());
if (name.isAbsolute()) {
throw new IllegalArgumentException("Resource name must be relative");
}
String linkName = name.toUri().getPath();
localResources.put(
linkName,
BuilderUtils.newLocalResource(
p.toUri(), type,
visibilities[i]
? LocalResourceVisibility.PUBLIC
: LocalResourceVisibility.PRIVATE,
sizes[i], timestamps[i])
);
if (classPaths.containsKey(u.getPath())) {
MRApps.addToClassPath(env, linkName);
}
}
}
}
// TODO - Move this to MR!
private static long[] getFileSizes(Configuration conf, String key) {
String[] strs = conf.getStrings(key);
if (strs == null) {
return null;
}
long[] result = new long[strs.length];
for(int i=0; i < strs.length; ++i) {
result[i] = Long.parseLong(strs[i]);
}
return result;
}
}

View File

@ -117,6 +117,8 @@ message TaskAttemptReportProto {
optional string diagnostic_info = 7;
optional string state_string = 8;
optional PhaseProto phase = 9;
optional int64 shuffle_finish_time = 10;
optional int64 sort_finish_time=11;
}
enum JobStateProto {

View File

@ -379,46 +379,46 @@ public void printAll() {
}
}
/** Get the job submit time */
/** @return the job submit time */
public long getSubmitTime() { return submitTime; }
/** Get the job finish time */
/** @return the job finish time */
public long getFinishTime() { return finishTime; }
/** Get the job id */
/** @return the job id */
public JobID getJobId() { return jobid; }
/** Get the user name */
/** @return the user name */
public String getUsername() { return username; }
/** Get the job name */
/** @return the job name */
public String getJobname() { return jobname; }
/** Get the job queue name */
/** @return the job queue name */
public String getJobQueueName() { return jobQueueName; }
/** Get the path for the job configuration file */
/** @return the path for the job configuration file */
public String getJobConfPath() { return jobConfPath; }
/** Get the job launch time */
/** @return the job launch time */
public long getLaunchTime() { return launchTime; }
/** Get the total number of maps */
/** @return the total number of maps */
public long getTotalMaps() { return totalMaps; }
/** Get the total number of reduces */
/** @return the total number of reduces */
public long getTotalReduces() { return totalReduces; }
/** Get the total number of failed maps */
/** @return the total number of failed maps */
public long getFailedMaps() { return failedMaps; }
/** Get the number of failed reduces */
/** @return the number of failed reduces */
public long getFailedReduces() { return failedReduces; }
/** Get the number of finished maps */
/** @return the number of finished maps */
public long getFinishedMaps() { return finishedMaps; }
/** Get the number of finished reduces */
/** @return the number of finished reduces */
public long getFinishedReduces() { return finishedReduces; }
/** Get the job status */
/** @return the job status */
public String getJobStatus() { return jobStatus; }
public String getErrorInfo() { return errorInfo; }
/** Get the counters for the job */
/** @return the counters for the job */
public Counters getTotalCounters() { return totalCounters; }
/** Get the map counters for the job */
/** @return the map counters for the job */
public Counters getMapCounters() { return mapCounters; }
/** Get the reduce counters for the job */
/** @return the reduce counters for the job */
public Counters getReduceCounters() { return reduceCounters; }
/** Get the map of all tasks in this job */
/** @return the map of all tasks in this job */
public Map<TaskID, TaskInfo> getAllTasks() { return tasksMap; }
/** Get the priority of this job */
/** @return the priority of this job */
public String getPriority() { return priority.toString(); }
public Map<JobACL, AccessControlList> getJobACLs() { return jobACLs; }
}
@ -458,27 +458,27 @@ public void printAll() {
}
}
/** Get the Task ID */
/** @return the Task ID */
public TaskID getTaskId() { return taskId; }
/** Get the start time of this task */
/** @return the start time of this task */
public long getStartTime() { return startTime; }
/** Get the finish time of this task */
/** @return the finish time of this task */
public long getFinishTime() { return finishTime; }
/** Get the task type */
/** @return the task type */
public TaskType getTaskType() { return taskType; }
/** Get the split locations */
/** @return the split locations */
public String getSplitLocations() { return splitLocations; }
/** Get the counters for this task */
/** @return the counters for this task */
public Counters getCounters() { return counters; }
/** Get the task status */
/** @return the task status */
public String getTaskStatus() { return status; }
/** Get the attempt Id that caused this task to fail */
/** @return the attempt Id that caused this task to fail */
public TaskAttemptID getFailedDueToAttemptId() {
return failedDueToAttemptId;
}
/** Get the error */
/** @return the error */
public String getError() { return error; }
/** Get the map of all attempts for this task */
/** @return the map of all attempts for this task */
public Map<TaskAttemptID, TaskAttemptInfo> getAllTaskAttempts() {
return attemptsMap;
}
@ -530,33 +530,33 @@ public void printAll() {
}
}
/** Get the attempt Id */
/** @return the attempt Id */
public TaskAttemptID getAttemptId() { return attemptId; }
/** Get the start time of the attempt */
/** @return the start time of the attempt */
public long getStartTime() { return startTime; }
/** Get the finish time of the attempt */
/** @return the finish time of the attempt */
public long getFinishTime() { return finishTime; }
/** Get the shuffle finish time. Applicable only for reduce attempts */
/** @return the shuffle finish time. Applicable only for reduce attempts */
public long getShuffleFinishTime() { return shuffleFinishTime; }
/** Get the sort finish time. Applicable only for reduce attempts */
/** @return the sort finish time. Applicable only for reduce attempts */
public long getSortFinishTime() { return sortFinishTime; }
/** Get the map finish time. Applicable only for map attempts */
/** @return the map finish time. Applicable only for map attempts */
public long getMapFinishTime() { return mapFinishTime; }
/** Get the error string */
/** @return the error string */
public String getError() { return error; }
/** Get the state */
/** @return the state */
public String getState() { return state; }
/** Get the task status */
/** @return the task status */
public String getTaskStatus() { return status; }
/** Get the task type */
/** @return the task type */
public TaskType getTaskType() { return taskType; }
/** Get the tracker name where the attempt executed */
/** @return the tracker name where the attempt executed */
public String getTrackerName() { return trackerName; }
/** Get the host name */
/** @return the host name */
public String getHostname() { return hostname; }
/** Get the counters for the attempt */
/** @return the counters for the attempt */
public Counters getCounters() { return counters; }
/** Get the HTTP port for the tracker */
/** @return the HTTP port for the tracker */
public int getHttpPort() { return httpPort; }
}
}

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.mapreduce.jobhistory;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.TaskStatus;
@ -28,7 +26,6 @@
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapred.ProgressSplitsBlock;
import org.apache.hadoop.mapred.TaskStatus;
import org.apache.avro.util.Utf8;

View File

@ -70,22 +70,26 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
private final Map<TaskId, Task> mapTasks = new HashMap<TaskId, Task>();
private final Map<TaskId, Task> reduceTasks = new HashMap<TaskId, Task>();
private final String user;
private final Path confFile;
private List<TaskAttemptCompletionEvent> completionEvents = null;
private JobInfo jobInfo;
public CompletedJob(Configuration conf, JobId jobId, Path historyFile,
boolean loadTasks, String userName) throws IOException {
boolean loadTasks, String userName, Path confFile) throws IOException {
LOG.info("Loading job: " + jobId + " from file: " + historyFile);
this.conf = conf;
this.jobId = jobId;
this.confFile = confFile;
loadFullHistoryData(loadTasks, historyFile);
user = userName;
counters = TypeConverter.toYarn(jobInfo.getTotalCounters());
diagnostics.add(jobInfo.getErrorInfo());
report = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobReport.class);
report =
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
JobReport.class);
report.setJobId(jobId);
report.setJobState(JobState.valueOf(jobInfo.getJobStatus()));
report.setStartTime(jobInfo.getLaunchTime());
@ -192,11 +196,12 @@ public int compare(TaskAttempt o1, TaskAttempt o2) {
int attemptRunTime = -1;
if (taskAttempt.getLaunchTime() != 0 && taskAttempt.getFinishTime() != 0) {
attemptRunTime = (int) (taskAttempt.getFinishTime() - taskAttempt
.getLaunchTime());
attemptRunTime =
(int) (taskAttempt.getFinishTime() - taskAttempt.getLaunchTime());
}
// Default to KILLED
TaskAttemptCompletionEventStatus taceStatus = TaskAttemptCompletionEventStatus.KILLED;
TaskAttemptCompletionEventStatus taceStatus =
TaskAttemptCompletionEventStatus.KILLED;
String taStateString = taskAttempt.getState().toString();
try {
taceStatus = TaskAttemptCompletionEventStatus.valueOf(taStateString);
@ -222,7 +227,8 @@ public Map<TaskId, Task> getTasks() {
}
//History data is leisurely loaded when task level data is requested
private synchronized void loadFullHistoryData(boolean loadTasks, Path historyFileAbsolute) throws IOException {
private synchronized void loadFullHistoryData(boolean loadTasks,
Path historyFileAbsolute) throws IOException {
LOG.info("Loading history file: [" + historyFileAbsolute + "]");
if (jobInfo != null) {
return; //data already loaded
@ -230,11 +236,13 @@ private synchronized void loadFullHistoryData(boolean loadTasks, Path historyFil
if (historyFileAbsolute != null) {
try {
JobHistoryParser parser = new JobHistoryParser(historyFileAbsolute.getFileSystem(conf), historyFileAbsolute);
JobHistoryParser parser =
new JobHistoryParser(historyFileAbsolute.getFileSystem(conf),
historyFileAbsolute);
jobInfo = parser.parse();
} catch (IOException e) {
throw new YarnException("Could not load history file " + historyFileAbsolute,
e);
throw new YarnException("Could not load history file "
+ historyFileAbsolute, e);
}
} else {
throw new IOException("History file not found");
@ -293,7 +301,8 @@ public Map<TaskId, Task> getTasks(TaskType taskType) {
}
@Override
public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) {
public
boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) {
if (!UserGroupInformation.isSecurityEnabled()) {
return true;
}
@ -304,8 +313,26 @@ public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation)
jobInfo.getUsername(), jobACL);
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.job.Job#getJobACLs()
*/
@Override
public Map<JobACL, AccessControlList> getJobACLs() {
return jobInfo.getJobACLs();
}
@Override
public String getUserName() {
return user;
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile()
*/
@Override
public Path getConfFile() {
return confFile;
}
}

View File

@ -71,6 +71,8 @@ public class CompletedTaskAttempt implements TaskAttempt {
report.setStartTime(attemptInfo.getStartTime());
report.setFinishTime(attemptInfo.getFinishTime());
report.setShuffleFinishTime(attemptInfo.getShuffleFinishTime());
report.setSortFinishTime(attemptInfo.getSortFinishTime());
if (localDiagMessage != null) {
report.setDiagnosticInfo(attemptInfo.getError() + ", " + localDiagMessage);
} else {
@ -159,9 +161,18 @@ public long getFinishTime() {
return report.getFinishTime();
}
@Override
public long getShuffleFinishTime() {
return report.getShuffleFinishTime();
}
@Override
public long getSortFinishTime() {
return report.getSortFinishTime();
}
@Override
public int getShufflePort() {
throw new UnsupportedOperationException("Not supported yet.");
}
}

View File

@ -21,7 +21,6 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Calendar;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
@ -36,8 +35,6 @@
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -87,18 +84,18 @@ public class JobHistory extends AbstractService implements HistoryContext {
private static final Log SUMMARY_LOG = LogFactory.getLog(JobSummary.class);
private static final Pattern DATE_PATTERN = Pattern
.compile("([0-1]?[0-9])/([0-3]?[0-9])/((?:2[0-9])[0-9][0-9])");
/*
* TODO Get rid of this once JobId has it's own comparator
*/
private static final Comparator<JobId> JOB_ID_COMPARATOR = new Comparator<JobId>() {
private static final Comparator<JobId> JOB_ID_COMPARATOR =
new Comparator<JobId>() {
@Override
public int compare(JobId o1, JobId o2) {
if (o1.getAppId().getClusterTimestamp() > o2.getAppId().getClusterTimestamp()) {
if (o1.getAppId().getClusterTimestamp() >
o2.getAppId().getClusterTimestamp()) {
return 1;
} else if (o1.getAppId().getClusterTimestamp() < o2.getAppId().getClusterTimestamp()) {
} else if (o1.getAppId().getClusterTimestamp() <
o2.getAppId().getClusterTimestamp()) {
return -1;
} else {
return o1.getId() - o2.getId();
@ -106,7 +103,8 @@ public int compare(JobId o1, JobId o2) {
}
};
private static String DONE_BEFORE_SERIAL_TAIL = JobHistoryUtils.doneSubdirsBeforeSerialTail();
private static String DONE_BEFORE_SERIAL_TAIL =
JobHistoryUtils.doneSubdirsBeforeSerialTail();
/**
* Maps between a serial number (generated based on jobId) and the timestamp
@ -114,29 +112,32 @@ public int compare(JobId o1, JobId o2) {
* Facilitates jobId based searches.
* If a jobId is not found in this list - it will not be found.
*/
private final SortedMap<String, Set<String>> idToDateString = new ConcurrentSkipListMap<String, Set<String>>();
private final SortedMap<String, Set<String>> idToDateString =
new ConcurrentSkipListMap<String, Set<String>>();
//Maintains minimal details for recent jobs (parsed from history file name).
//Sorted on Job Completion Time.
private final SortedMap<JobId, MetaInfo> jobListCache = new ConcurrentSkipListMap<JobId, MetaInfo>(
JOB_ID_COMPARATOR);
private final SortedMap<JobId, MetaInfo> jobListCache =
new ConcurrentSkipListMap<JobId, MetaInfo>(JOB_ID_COMPARATOR);
// Re-use exisiting MetaInfo objects if they exist for the specific JobId. (synchronization on MetaInfo)
// Check for existance of the object when using iterators.
private final SortedMap<JobId, MetaInfo> intermediateListCache = new ConcurrentSkipListMap<JobId, JobHistory.MetaInfo>(
JOB_ID_COMPARATOR);
private final SortedMap<JobId, MetaInfo> intermediateListCache =
new ConcurrentSkipListMap<JobId, JobHistory.MetaInfo>(JOB_ID_COMPARATOR);
//Maintains a list of known done subdirectories. Not currently used.
private final Set<Path> existingDoneSubdirs = new HashSet<Path>();
private final SortedMap<JobId, Job> loadedJobCache = new ConcurrentSkipListMap<JobId, Job>(
JOB_ID_COMPARATOR);
private final SortedMap<JobId, Job> loadedJobCache =
new ConcurrentSkipListMap<JobId, Job>(JOB_ID_COMPARATOR);
/**
* Maintains a mapping between intermediate user directories and the last known modification time.
* Maintains a mapping between intermediate user directories and the last
* known modification time.
*/
private Map<String, Long> userDirModificationTimeMap = new HashMap<String, Long>();
private Map<String, Long> userDirModificationTimeMap =
new HashMap<String, Long>();
//The number of jobs to maintain in the job list cache.
private int jobListCacheSize;
@ -187,7 +188,8 @@ public void init(Configuration conf) throws YarnException {
debugMode = conf.getBoolean(JHAdminConfig.MR_HISTORY_DEBUG_MODE, false);
serialNumberLowDigits = debugMode ? 1 : 3;
serialNumberFormat = ("%0"
+ (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits) + "d");
+ (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS
+ serialNumberLowDigits) + "d");
String doneDirPrefix = null;
doneDirPrefix = JobHistoryUtils.getConfiguredHistoryServerDoneDirPrefix(conf);
@ -195,9 +197,11 @@ public void init(Configuration conf) throws YarnException {
doneDirPrefixPath = FileContext.getFileContext(conf).makeQualified(
new Path(doneDirPrefix));
doneDirFc = FileContext.getFileContext(doneDirPrefixPath.toUri(), conf);
mkdir(doneDirFc, doneDirPrefixPath, new FsPermission(JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION));
mkdir(doneDirFc, doneDirPrefixPath, new FsPermission(
JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION));
} catch (IOException e) {
throw new YarnException("Error creating done directory: [" + doneDirPrefixPath + "]", e);
throw new YarnException("Error creating done directory: [" +
doneDirPrefixPath + "]", e);
}
String intermediateDoneDirPrefix = null;
@ -208,21 +212,27 @@ public void init(Configuration conf) throws YarnException {
.makeQualified(new Path(intermediateDoneDirPrefix));
intermediateDoneDirFc = FileContext.getFileContext(
intermediateDoneDirPath.toUri(), conf);
mkdir(intermediateDoneDirFc, intermediateDoneDirPath, new FsPermission(JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS.toShort()));
mkdir(intermediateDoneDirFc, intermediateDoneDirPath, new FsPermission(
JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS.toShort()));
} catch (IOException e) {
LOG.info("error creating done directory on dfs " + e);
throw new YarnException("Error creating intermediate done directory: [" + intermediateDoneDirPath + "]", e);
throw new YarnException("Error creating intermediate done directory: ["
+ intermediateDoneDirPath + "]", e);
}
jobListCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_JOBLIST_CACHE_SIZE, DEFAULT_JOBLIST_CACHE_SIZE);
loadedJobCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_LOADED_JOB_CACHE_SIZE, DEFAULT_LOADEDJOB_CACHE_SIZE);
dateStringCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_DATESTRING_CACHE_SIZE, DEFAULT_DATESTRING_CACHE_SIZE);
jobListCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_JOBLIST_CACHE_SIZE,
DEFAULT_JOBLIST_CACHE_SIZE);
loadedJobCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_LOADED_JOB_CACHE_SIZE,
DEFAULT_LOADEDJOB_CACHE_SIZE);
dateStringCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_DATESTRING_CACHE_SIZE,
DEFAULT_DATESTRING_CACHE_SIZE);
moveThreadInterval =
conf.getLong(JHAdminConfig.MR_HISTORY_MOVE_INTERVAL_MS,
DEFAULT_MOVE_THREAD_INTERVAL);
numMoveThreads = conf.getInt(JHAdminConfig.MR_HISTORY_MOVE_THREAD_COUNT, DEFAULT_MOVE_THREAD_COUNT);
numMoveThreads = conf.getInt(JHAdminConfig.MR_HISTORY_MOVE_THREAD_COUNT,
DEFAULT_MOVE_THREAD_COUNT);
try {
initExisting();
} catch (IOException e) {
@ -254,19 +264,21 @@ private void mkdir(FileContext fc, Path path, FsPermission fsp)
@Override
public void start() {
//Start moveIntermediatToDoneThread
moveIntermediateToDoneRunnable = new MoveIntermediateToDoneRunnable(moveThreadInterval, numMoveThreads);
moveIntermediateToDoneRunnable =
new MoveIntermediateToDoneRunnable(moveThreadInterval, numMoveThreads);
moveIntermediateToDoneThread = new Thread(moveIntermediateToDoneRunnable);
moveIntermediateToDoneThread.setName("MoveIntermediateToDoneScanner");
moveIntermediateToDoneThread.start();
//Start historyCleaner
boolean startCleanerService = conf.getBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, true);
boolean startCleanerService = conf.getBoolean(
JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, true);
if (startCleanerService) {
long maxAgeOfHistoryFiles = conf.getLong(JHAdminConfig.MR_HISTORY_MAX_AGE_MS,
DEFAULT_HISTORY_MAX_AGE);
long maxAgeOfHistoryFiles = conf.getLong(
JHAdminConfig.MR_HISTORY_MAX_AGE_MS, DEFAULT_HISTORY_MAX_AGE);
cleanerScheduledExecutor = new ScheduledThreadPoolExecutor(1);
long runInterval = conf.getLong(JHAdminConfig.MR_HISTORY_CLEANER_INTERVAL_MS,
DEFAULT_RUN_INTERVAL);
long runInterval = conf.getLong(
JHAdminConfig.MR_HISTORY_CLEANER_INTERVAL_MS, DEFAULT_RUN_INTERVAL);
cleanerScheduledExecutor
.scheduleAtFixedRate(new HistoryCleaner(maxAgeOfHistoryFiles),
30 * 1000l, runInterval, TimeUnit.MILLISECONDS);
@ -331,13 +343,16 @@ private void initExisting() throws IOException {
private void removeDirectoryFromSerialNumberIndex(Path serialDirPath) {
String serialPart = serialDirPath.getName();
String timeStampPart = JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
String timeStampPart =
JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
if (timeStampPart == null) {
LOG.warn("Could not find timestamp portion from path: " + serialDirPath.toString() +". Continuing with next");
LOG.warn("Could not find timestamp portion from path: " +
serialDirPath.toString() +". Continuing with next");
return;
}
if (serialPart == null) {
LOG.warn("Could not find serial portion from path: " + serialDirPath.toString() + ". Continuing with next");
LOG.warn("Could not find serial portion from path: " +
serialDirPath.toString() + ". Continuing with next");
return;
}
if (idToDateString.containsKey(serialPart)) {
@ -355,13 +370,16 @@ private void addDirectoryToSerialNumberIndex(Path serialDirPath) {
LOG.debug("Adding "+serialDirPath+" to serial index");
}
String serialPart = serialDirPath.getName();
String timestampPart = JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
String timestampPart =
JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
if (timestampPart == null) {
LOG.warn("Could not find timestamp portion from path: " + serialDirPath.toString() +". Continuing with next");
LOG.warn("Could not find timestamp portion from path: " +
serialDirPath.toString() +". Continuing with next");
return;
}
if (serialPart == null) {
LOG.warn("Could not find serial portion from path: " + serialDirPath.toString() + ". Continuing with next");
LOG.warn("Could not find serial portion from path: " +
serialDirPath.toString() + ". Continuing with next");
}
addToSerialNumberIndex(serialPart, timestampPart);
}
@ -400,7 +418,8 @@ private void addDirectoryToJobListCache(Path path) throws IOException {
}
}
private static List<FileStatus> scanDirectory(Path path, FileContext fc, PathFilter pathFilter) throws IOException {
private static List<FileStatus> scanDirectory(Path path, FileContext fc,
PathFilter pathFilter) throws IOException {
path = fc.makeQualified(path);
List<FileStatus> jhStatusList = new ArrayList<FileStatus>();
RemoteIterator<FileStatus> fileStatusIter = fc.listStatus(path);
@ -414,7 +433,8 @@ private static List<FileStatus> scanDirectory(Path path, FileContext fc, PathFil
return jhStatusList;
}
private static List<FileStatus> scanDirectoryForHistoryFiles(Path path, FileContext fc) throws IOException {
private static List<FileStatus> scanDirectoryForHistoryFiles(Path path,
FileContext fc) throws IOException {
return scanDirectory(path, fc, JobHistoryUtils.getHistoryFileFilter());
}
@ -425,7 +445,8 @@ private static List<FileStatus> scanDirectoryForHistoryFiles(Path path, FileCont
* @return
*/
private List<FileStatus> findTimestampedDirectories() throws IOException {
List<FileStatus> fsList = JobHistoryUtils.localGlobber(doneDirFc, doneDirPrefixPath, DONE_BEFORE_SERIAL_TAIL);
List<FileStatus> fsList = JobHistoryUtils.localGlobber(doneDirFc,
doneDirPrefixPath, DONE_BEFORE_SERIAL_TAIL);
return fsList;
}
@ -434,7 +455,8 @@ private List<FileStatus> findTimestampedDirectories() throws IOException {
*/
private void addToJobListCache(JobId jobId, MetaInfo metaInfo) {
if(LOG.isDebugEnabled()) {
LOG.debug("Adding "+jobId+" to job list cache with "+metaInfo.getJobIndexInfo());
LOG.debug("Adding "+jobId+" to job list cache with "
+metaInfo.getJobIndexInfo());
}
jobListCache.put(jobId, metaInfo);
if (jobListCache.size() > jobListCacheSize) {
@ -462,14 +484,16 @@ private void addToLoadedJobCache(Job job) {
* @throws IOException
*/
private void scanIntermediateDirectory() throws IOException {
List<FileStatus> userDirList = JobHistoryUtils.localGlobber(intermediateDoneDirFc, intermediateDoneDirPath, "");
List<FileStatus> userDirList =
JobHistoryUtils.localGlobber(intermediateDoneDirFc, intermediateDoneDirPath, "");
for (FileStatus userDir : userDirList) {
String name = userDir.getPath().getName();
long newModificationTime = userDir.getModificationTime();
boolean shouldScan = false;
synchronized (userDirModificationTimeMap) {
if (!userDirModificationTimeMap.containsKey(name) || newModificationTime > userDirModificationTimeMap.get(name)) {
if (!userDirModificationTimeMap.containsKey(name) || newModificationTime
> userDirModificationTimeMap.get(name)) {
shouldScan = true;
userDirModificationTimeMap.put(name, newModificationTime);
}
@ -514,9 +538,11 @@ private void scanIntermediateDirectory(final Path absPath)
* @return A MetaInfo object for the jobId, null if not found.
* @throws IOException
*/
private MetaInfo getJobMetaInfo(List<FileStatus> fileStatusList, JobId jobId) throws IOException {
private MetaInfo getJobMetaInfo(List<FileStatus> fileStatusList, JobId jobId)
throws IOException {
for (FileStatus fs : fileStatusList) {
JobIndexInfo jobIndexInfo = FileNameIndexUtils.getIndexInfo(fs.getPath().getName());
JobIndexInfo jobIndexInfo =
FileNameIndexUtils.getIndexInfo(fs.getPath().getName());
if (jobIndexInfo.getJobId().equals(jobId)) {
String confFileName = JobHistoryUtils
.getIntermediateConfFileName(jobIndexInfo.getJobId());
@ -549,7 +575,8 @@ private MetaInfo scanOldDirsForJob(JobId jobId) throws IOException {
}
for (String timestampPart : dateStringSet) {
Path logDir = canonicalHistoryLogPath(jobId, timestampPart);
List<FileStatus> fileStatusList = scanDirectoryForHistoryFiles(logDir, doneDirFc);
List<FileStatus> fileStatusList = scanDirectoryForHistoryFiles(logDir,
doneDirFc);
MetaInfo metaInfo = getJobMetaInfo(fileStatusList, jobId);
if (metaInfo != null) {
return metaInfo;
@ -559,7 +586,8 @@ private MetaInfo scanOldDirsForJob(JobId jobId) throws IOException {
}
/**
* Checks for the existence of the job history file in the interemediate directory.
* Checks for the existence of the job history file in the intermediate
* directory.
* @param jobId
* @return
* @throws IOException
@ -586,7 +614,8 @@ public void stop() {
MoveIntermediateToDoneRunnable(long sleepTime, int numMoveThreads) {
this.sleepTime = sleepTime;
moveToDoneExecutor = new ThreadPoolExecutor(1, numMoveThreads, 1, TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
moveToDoneExecutor = new ThreadPoolExecutor(1, numMoveThreads, 1,
TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
running = true;
}
@ -604,7 +633,8 @@ public void run() {
try {
moveToDone(metaInfo);
} catch (IOException e) {
LOG.info("Failed to process metaInfo for job: " + metaInfo.jobIndexInfo.getJobId(), e);
LOG.info("Failed to process metaInfo for job: " +
metaInfo.jobIndexInfo.getJobId(), e);
}
}
});
@ -629,38 +659,17 @@ private Job loadJob(MetaInfo metaInfo) {
synchronized(metaInfo) {
try {
Job job = new CompletedJob(conf, metaInfo.getJobIndexInfo().getJobId(),
metaInfo.getHistoryFile(), true, metaInfo.getJobIndexInfo().getUser());
metaInfo.getHistoryFile(), true, metaInfo.getJobIndexInfo().getUser(),
metaInfo.getConfFile());
addToLoadedJobCache(job);
return job;
} catch (IOException e) {
throw new YarnException("Could not find/load job: " + metaInfo.getJobIndexInfo().getJobId(), e);
throw new YarnException("Could not find/load job: " +
metaInfo.getJobIndexInfo().getJobId(), e);
}
}
}
private SortedMap<JobId, JobIndexInfo> getAllJobsMetaInfo() {
SortedMap<JobId, JobIndexInfo> result = new TreeMap<JobId, JobIndexInfo>(JOB_ID_COMPARATOR);
try {
scanIntermediateDirectory();
} catch (IOException e) {
LOG.warn("Failed to scan intermediate directory", e);
throw new YarnException(e);
}
for (JobId jobId : intermediateListCache.keySet()) {
MetaInfo mi = intermediateListCache.get(jobId);
if (mi != null) {
result.put(jobId, mi.getJobIndexInfo());
}
}
for (JobId jobId : jobListCache.keySet()) {
MetaInfo mi = jobListCache.get(jobId);
if (mi != null) {
result.put(jobId, mi.getJobIndexInfo());
}
}
return result;
}
private Map<JobId, Job> getAllJobsInternal() {
//TODO This should ideally be using getAllJobsMetaInfo
// or get rid of that method once Job has APIs for user, finishTime etc.
@ -746,108 +755,6 @@ private Job findJob(JobId jobId) throws IOException {
return null;
}
/**
* Searches cached jobs for the specified criteria (AND). Ignores the criteria if null.
* @param soughtUser
* @param soughtJobNameSubstring
* @param soughtDateStrings
* @return
*/
private Map<JobId, Job> findJobs(String soughtUser, String soughtJobNameSubstring, String[] soughtDateStrings) {
boolean searchUser = true;
boolean searchJobName = true;
boolean searchDates = true;
List<Calendar> soughtCalendars = null;
if (soughtUser == null) {
searchUser = false;
}
if (soughtJobNameSubstring == null) {
searchJobName = false;
}
if (soughtDateStrings == null) {
searchDates = false;
} else {
soughtCalendars = getSoughtDateAsCalendar(soughtDateStrings);
}
Map<JobId, Job> resultMap = new TreeMap<JobId, Job>();
SortedMap<JobId, JobIndexInfo> allJobs = getAllJobsMetaInfo();
for (Map.Entry<JobId, JobIndexInfo> entry : allJobs.entrySet()) {
JobId jobId = entry.getKey();
JobIndexInfo indexInfo = entry.getValue();
String jobName = indexInfo.getJobName();
String jobUser = indexInfo.getUser();
long finishTime = indexInfo.getFinishTime();
if (searchUser) {
if (!soughtUser.equals(jobUser)) {
continue;
}
}
if (searchJobName) {
if (!jobName.contains(soughtJobNameSubstring)) {
continue;
}
}
if (searchDates) {
boolean matchedDate = false;
Calendar jobCal = Calendar.getInstance();
jobCal.setTimeInMillis(finishTime);
for (Calendar cal : soughtCalendars) {
if (jobCal.get(Calendar.YEAR) == cal.get(Calendar.YEAR) &&
jobCal.get(Calendar.MONTH) == cal.get(Calendar.MONTH) &&
jobCal.get(Calendar.DAY_OF_MONTH) == cal.get(Calendar.DAY_OF_MONTH)) {
matchedDate = true;
break;
}
}
if (!matchedDate) {
break;
}
}
resultMap.put(jobId, new PartialJob(indexInfo, jobId));
}
return resultMap;
}
private List<Calendar> getSoughtDateAsCalendar(String [] soughtDateStrings) {
List<Calendar> soughtCalendars = new ArrayList<Calendar>();
for (int i = 0 ; i < soughtDateStrings.length ; i++) {
String soughtDate = soughtDateStrings[i];
if (soughtDate.length() != 0) {
Matcher m = DATE_PATTERN.matcher(soughtDate);
if (m.matches()) {
String yyyyPart = m.group(3);
String mmPart = m.group(1);
String ddPart = m.group(2);
if (yyyyPart.length() == 2) {
yyyyPart = "20" + yyyyPart;
}
if (mmPart.length() == 1) {
mmPart = "0" + mmPart;
}
if (ddPart.length() == 1) {
ddPart = "0" + ddPart;
}
Calendar soughtCal = Calendar.getInstance();
soughtCal.set(Calendar.YEAR, Integer.parseInt(yyyyPart));
soughtCal.set(Calendar.MONTH, Integer.parseInt(mmPart) - 1);
soughtCal.set(Calendar.DAY_OF_MONTH, Integer.parseInt(ddPart) -1);
soughtCalendars.add(soughtCal);
}
}
}
return soughtCalendars;
}
private void moveToDone(MetaInfo metaInfo) throws IOException {
long completeTime = metaInfo.getJobIndexInfo().getFinishTime();
if (completeTime == 0) completeTime = System.currentTimeMillis();
@ -890,26 +797,31 @@ private void moveToDone(MetaInfo metaInfo) throws IOException {
try {
maybeMakeSubdirectory(targetDir);
} catch (IOException e) {
LOG.warn("Failed creating subdirectory: " + targetDir + " while attempting to move files for jobId: " + jobId);
LOG.warn("Failed creating subdirectory: " + targetDir +
" while attempting to move files for jobId: " + jobId);
throw e;
}
synchronized (metaInfo) {
if (historyFile != null) {
Path toPath = doneDirFc.makeQualified(new Path(targetDir, historyFile.getName()));
Path toPath = doneDirFc.makeQualified(new Path(targetDir,
historyFile.getName()));
try {
moveToDoneNow(historyFile, toPath);
} catch (IOException e) {
LOG.warn("Failed to move file: " + historyFile + " for jobId: " + jobId);
LOG.warn("Failed to move file: " + historyFile + " for jobId: "
+ jobId);
throw e;
}
metaInfo.setHistoryFile(toPath);
}
if (confFile != null) {
Path toPath = doneDirFc.makeQualified(new Path(targetDir, confFile.getName()));
Path toPath = doneDirFc.makeQualified(new Path(targetDir,
confFile.getName()));
try {
moveToDoneNow(confFile, toPath);
} catch (IOException e) {
LOG.warn("Failed to move file: " + historyFile + " for jobId: " + jobId);
LOG.warn("Failed to move file: " + historyFile + " for jobId: "
+ jobId);
throw e;
}
metaInfo.setConfFile(toPath);
@ -953,7 +865,8 @@ private void maybeMakeSubdirectory(Path path) throws IOException {
}
} catch (FileNotFoundException fnfE) {
try {
FsPermission fsp = new FsPermission(JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION);
FsPermission fsp =
new FsPermission(JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION);
doneDirFc.mkdir(path, fsp, true);
FileStatus fsStatus = doneDirFc.getFileStatus(path);
LOG.info("Perms after creating " + fsStatus.getPermission().toShort()
@ -972,12 +885,15 @@ private void maybeMakeSubdirectory(Path path) throws IOException {
}
private Path canonicalHistoryLogPath(JobId id, String timestampComponent) {
return new Path(doneDirPrefixPath, JobHistoryUtils.historyLogSubdirectory(id, timestampComponent, serialNumberFormat));
return new Path(doneDirPrefixPath,
JobHistoryUtils.historyLogSubdirectory(id, timestampComponent, serialNumberFormat));
}
private Path canonicalHistoryLogPath(JobId id, long millisecondTime) {
String timestampComponent = JobHistoryUtils.timestampDirectoryComponent(millisecondTime, debugMode);
return new Path(doneDirPrefixPath, JobHistoryUtils.historyLogSubdirectory(id, timestampComponent, serialNumberFormat));
String timestampComponent =
JobHistoryUtils.timestampDirectoryComponent(millisecondTime, debugMode);
return new Path(doneDirPrefixPath,
JobHistoryUtils.historyLogSubdirectory(id, timestampComponent, serialNumberFormat));
}
@ -1033,7 +949,8 @@ static class MetaInfo {
private Path summaryFile;
JobIndexInfo jobIndexInfo;
MetaInfo(Path historyFile, Path confFile, Path summaryFile, JobIndexInfo jobIndexInfo) {
MetaInfo(Path historyFile, Path confFile, Path summaryFile,
JobIndexInfo jobIndexInfo) {
this.historyFile = historyFile;
this.confFile = confFile;
this.summaryFile = summaryFile;
@ -1073,13 +990,19 @@ public void run() {
//Sort in ascending order. Relies on YYYY/MM/DD/Serial
Collections.sort(serialDirList);
for (FileStatus serialDir : serialDirList) {
List<FileStatus> historyFileList = scanDirectoryForHistoryFiles(serialDir.getPath(), doneDirFc);
List<FileStatus> historyFileList =
scanDirectoryForHistoryFiles(serialDir.getPath(), doneDirFc);
for (FileStatus historyFile : historyFileList) {
JobIndexInfo jobIndexInfo = FileNameIndexUtils.getIndexInfo(historyFile.getPath().getName());
long effectiveTimestamp = getEffectiveTimestamp(jobIndexInfo.getFinishTime(), historyFile);
JobIndexInfo jobIndexInfo =
FileNameIndexUtils.getIndexInfo(historyFile.getPath().getName());
long effectiveTimestamp =
getEffectiveTimestamp(jobIndexInfo.getFinishTime(), historyFile);
if (shouldDelete(effectiveTimestamp)) {
String confFileName = JobHistoryUtils.getIntermediateConfFileName(jobIndexInfo.getJobId());
MetaInfo metaInfo = new MetaInfo(historyFile.getPath(), new Path(historyFile.getPath().getParent(), confFileName), null, jobIndexInfo);
String confFileName =
JobHistoryUtils.getIntermediateConfFileName(jobIndexInfo.getJobId());
MetaInfo metaInfo = new MetaInfo(historyFile.getPath(),
new Path(historyFile.getPath().getParent(), confFileName),
null, jobIndexInfo);
delete(metaInfo);
} else {
halted = true;

View File

@ -74,14 +74,15 @@ protected void doSecureLogin(Configuration conf) throws IOException {
public static void main(String[] args) {
StringUtils.startupShutdownMessage(JobHistoryServer.class, args, LOG);
JobHistoryServer server = null;
try {
server = new JobHistoryServer();
JobHistoryServer jobHistoryServer = new JobHistoryServer();
Runtime.getRuntime().addShutdownHook(
new CompositeServiceShutdownHook(jobHistoryServer));
YarnConfiguration conf = new YarnConfiguration(new JobConf());
server.init(conf);
server.start();
} catch (Throwable e) {
LOG.fatal(StringUtils.stringifyException(e));
jobHistoryServer.init(conf);
jobHistoryServer.start();
} catch (Throwable t) {
LOG.fatal("Error starting JobHistoryServer", t);
System.exit(-1);
}
}

View File

@ -21,6 +21,7 @@
import java.util.List;
import java.util.Map;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@ -32,6 +33,7 @@
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import clover.org.apache.log4j.Logger;
@ -147,4 +149,14 @@ public String getUserName() {
return jobIndexInfo.getUser();
}
@Override
public Path getConfFile() {
throw new IllegalStateException("Not implemented yet");
}
@Override
public Map<JobACL, AccessControlList> getJobACLs() {
throw new IllegalStateException("Not implemented yet");
}
}

View File

@ -0,0 +1,97 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.ATTEMPT_STATE;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
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.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.App;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
import org.apache.hadoop.yarn.webapp.SubView;
import com.google.inject.Inject;
/**
* Render a page showing the attempts made of a given type and a given job.
*/
public class HsAttemptsPage extends HsTaskPage {
static class FewAttemptsBlock extends HsTaskPage.AttemptsBlock {
@Inject
FewAttemptsBlock(App ctx) {
super(ctx);
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsTaskPage.AttemptsBlock#isValidRequest()
* Verify that a job is given.
*/
@Override
protected boolean isValidRequest() {
return app.getJob() != null;
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsTaskPage.AttemptsBlock#getTaskAttempts()
* @return the attempts that are for a given job and a specific type/state.
*/
@Override
protected Collection<TaskAttempt> getTaskAttempts() {
List<TaskAttempt> fewTaskAttemps = new ArrayList<TaskAttempt>();
String taskTypeStr = $(TASK_TYPE);
TaskType taskType = MRApps.taskType(taskTypeStr);
String attemptStateStr = $(ATTEMPT_STATE);
TaskAttemptStateUI neededState = MRApps
.taskAttemptState(attemptStateStr);
Job j = app.getJob();
Map<TaskId, Task> tasks = j.getTasks(taskType);
for (Task task : tasks.values()) {
Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
for (TaskAttempt attempt : attempts.values()) {
if (neededState.correspondsTo(attempt.getState())) {
fewTaskAttemps.add(attempt);
}
}
}
return fewTaskAttemps;
}
}
/**
* The content will render a different set of task attempts.
* @return FewAttemptsBlock.class
*/
@Override
protected Class<? extends SubView> content() {
return FewAttemptsBlock.class;
}
}

View File

@ -0,0 +1,99 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.JOB_ID;
import static org.apache.hadoop.yarn.util.StringHelper.join;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.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.postInitID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
import org.apache.hadoop.mapreduce.v2.app.webapp.ConfBlock;
import org.apache.hadoop.yarn.webapp.SubView;
/**
* Render a page with the configuration for a give job in it.
*/
public class HsConfPage extends HsView {
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
*/
@Override protected void preHead(Page.HTML<_> html) {
String jobID = $(JOB_ID);
set(TITLE, jobID.isEmpty() ? "Bad request: missing job ID"
: join("Configuration for MapReduce Job ", $(JOB_ID)));
commonPreHead(html);
set(DATATABLES_ID, "conf");
set(initID(DATATABLES, "conf"), confTableInit());
set(postInitID(DATATABLES, "conf"), confPostTableInit());
setTableStyles(html, "conf");
//Override the default nav config
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
}
/**
* The body of this block is the configuration block.
* @return HsConfBlock.class
*/
@Override protected Class<? extends SubView> content() {
return ConfBlock.class;
}
/**
* @return the end of the JS map that is the jquery datatable config for the
* conf table.
*/
private String confTableInit() {
return tableInit().append("}").toString();
}
/**
* @return the java script code to allow the jquery conf datatable to filter
* by column.
*/
private String confPostTableInit() {
return "var confInitVals = new Array();\n" +
"$('tfoot input').keyup( function () \n{"+
" confDataTable.fnFilter( this.value, $('tfoot input').index(this) );\n"+
"} );\n"+
"$('tfoot input').each( function (i) {\n"+
" confInitVals[i] = this.value;\n"+
"} );\n"+
"$('tfoot input').focus( function () {\n"+
" if ( this.className == 'search_init' )\n"+
" {\n"+
" this.className = '';\n"+
" this.value = '';\n"+
" }\n"+
"} );\n"+
"$('tfoot input').blur( function (i) {\n"+
" if ( this.value == '' )\n"+
" {\n"+
" this.className = 'search_init';\n"+
" this.value = confInitVals[$('tfoot input').index(this)];\n"+
" }\n"+
"} );\n";
}
}

View File

@ -79,6 +79,15 @@ protected Class<? extends View> taskPage() {
return HsTaskPage.class;
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#attemptsPage()
*/
@Override
protected Class<? extends View> attemptsPage() {
return HsAttemptsPage.class;
}
// Need all of these methods here also as Guice doesn't look into parent
// classes.
@ -127,6 +136,21 @@ public void attempts() {
super.attempts();
}
/**
* @return the page that will be used to render the /conf page
*/
protected Class<? extends View> confPage() {
return HsConfPage.class;
}
/**
* Render the /conf page
*/
public void conf() {
requireJob();
render(confPage());
}
/**
* @return the page about the current server.
*/

View File

@ -20,11 +20,14 @@
import com.google.inject.Inject;
import java.util.Date;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.mapreduce.JobACL;
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.TaskAttemptState;
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;
@ -32,12 +35,13 @@
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.util.StringUtils;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.webapp.ResponseInfo;
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.*;
/**
@ -46,21 +50,18 @@
public class HsJobBlock extends HtmlBlock {
final AppContext appContext;
int runningMapTasks = 0;
int pendingMapTasks = 0;
int runningReduceTasks = 0;
int pendingReduceTasks = 0;
int newMapAttempts = 0;
int runningMapAttempts = 0;
int killedMapAttempts = 0;
int failedMapAttempts = 0;
int successfulMapAttempts = 0;
int newReduceAttempts = 0;
int runningReduceAttempts = 0;
int killedReduceAttempts = 0;
int failedReduceAttempts = 0;
int successfulReduceAttempts = 0;
long avgMapTime = 0;
long avgReduceTime = 0;
long avgShuffleTime = 0;
long avgSortTime = 0;
int numMaps;
int numReduces;
@Inject HsJobBlock(AppContext appctx) {
appContext = appctx;
@ -84,9 +85,9 @@ public class HsJobBlock extends HtmlBlock {
p()._("Sorry, ", jid, " not found.")._();
return;
}
Map<JobACL, AccessControlList> acls = job.getJobACLs();
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();
@ -94,13 +95,38 @@ public class HsJobBlock extends HtmlBlock {
long startTime = jobReport.getStartTime();
long finishTime = jobReport.getFinishTime();
countTasksAndAttempts(job);
info("Job Overview").
ResponseInfo infoBlock = info("Job Overview").
_("Job Name:", job.getName()).
_("User Name:", job.getUserName()).
_("State:", job.getState()).
_("Uberized:", job.isUber()).
_("Started:", new Date(startTime)).
_("Finished:", new Date(finishTime)).
_("Elapsed:", StringUtils.formatTime(
Times.elapsed(startTime, finishTime)));
Times.elapsed(startTime, finishTime, false)));
List<String> diagnostics = job.getDiagnostics();
if(diagnostics != null && !diagnostics.isEmpty()) {
StringBuffer b = new StringBuffer();
for(String diag: diagnostics) {
b.append(diag);
}
infoBlock._("Diagnostics:", b.toString());
}
if(numMaps > 0) {
infoBlock._("Average Map Time", StringUtils.formatTime(avgMapTime));
}
if(numReduces > 0) {
infoBlock._("Average Reduce Time", StringUtils.formatTime(avgReduceTime));
infoBlock._("Average Shuffle Time", StringUtils.formatTime(avgShuffleTime));
infoBlock._("Average Merge Time", StringUtils.formatTime(avgSortTime));
}
for(Map.Entry<JobACL, AccessControlList> entry : acls.entrySet()) {
infoBlock._("ACL "+entry.getKey().getAclName()+":",
entry.getValue().getAclString());
}
html.
_(InfoBlock.class).
div(_INFO_WRAP).
@ -109,34 +135,17 @@ public class HsJobBlock extends HtmlBlock {
table("#job").
tr().
th(_TH, "Task Type").
th(_TH, "Progress").
th(_TH, "Total").
th(_TH, "Pending").
th(_TH, "Running").
th(_TH, "Complete")._().
tr(_ODD).
th().
a(url("tasks", jid, "m"), "Map")._().
td().
div(_PROGRESSBAR).
$title(join(mapPct, '%')). // tooltip
div(_PROGRESSBAR_VALUE).
$style(join("width:", mapPct, '%'))._()._()._().
td(String.valueOf(mapTasks)).
td(String.valueOf(pendingMapTasks)).
td(String.valueOf(runningMapTasks)).
td(String.valueOf(mapTasksComplete))._().
tr(_EVEN).
th().
a(url("tasks", jid, "r"), "Reduce")._().
td().
div(_PROGRESSBAR).
$title(join(reducePct, '%')). // tooltip
div(_PROGRESSBAR_VALUE).
$style(join("width:", reducePct, '%'))._()._()._().
td(String.valueOf(reduceTasks)).
td(String.valueOf(pendingReduceTasks)).
td(String.valueOf(runningReduceTasks)).
td(String.valueOf(reducesTasksComplete))._()
._().
@ -144,19 +153,11 @@ public class HsJobBlock extends HtmlBlock {
table("#job").
tr().
th(_TH, "Attempt Type").
th(_TH, "New").
th(_TH, "Running").
th(_TH, "Failed").
th(_TH, "Killed").
th(_TH, "Successful")._().
tr(_ODD).
th("Maps").
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.NEW.toString()),
String.valueOf(newMapAttempts))._().
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.RUNNING.toString()),
String.valueOf(runningMapAttempts))._().
td().a(url("attempts", jid, "m",
TaskAttemptStateUI.FAILED.toString()),
String.valueOf(failedMapAttempts))._().
@ -169,12 +170,6 @@ public class HsJobBlock extends HtmlBlock {
_().
tr(_EVEN).
th("Reduces").
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.NEW.toString()),
String.valueOf(newReduceAttempts))._().
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.RUNNING.toString()),
String.valueOf(runningReduceAttempts))._().
td().a(url("attempts", jid, "r",
TaskAttemptStateUI.FAILED.toString()),
String.valueOf(failedReduceAttempts))._().
@ -195,44 +190,21 @@ public class HsJobBlock extends HtmlBlock {
* @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()) {
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;
int successful = 0, failed = 0, killed =0;
if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
++newAttempts;
//Do Nothing
} else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
.getState())) {
++running;
//Do Nothing
} else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
.getState())) {
++successful;
@ -246,21 +218,41 @@ private void countTasksAndAttempts(Job job) {
switch (task.getType()) {
case MAP:
newMapAttempts += newAttempts;
runningMapAttempts += running;
successfulMapAttempts += successful;
failedMapAttempts += failed;
killedMapAttempts += killed;
if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
numMaps++;
avgMapTime += (attempt.getFinishTime() -
attempt.getLaunchTime());
}
break;
case REDUCE:
newReduceAttempts += newAttempts;
runningReduceAttempts += running;
successfulReduceAttempts += successful;
failedReduceAttempts += failed;
killedReduceAttempts += killed;
if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
numReduces++;
avgShuffleTime += (attempt.getShuffleFinishTime() -
attempt.getLaunchTime());
avgSortTime += attempt.getSortFinishTime() -
attempt.getLaunchTime();
avgReduceTime += (attempt.getFinishTime() -
attempt.getShuffleFinishTime());
}
break;
}
}
}
if(numMaps > 0) {
avgMapTime = avgMapTime / numMaps;
}
if(numReduces > 0) {
avgReduceTime = avgReduceTime / numReduces;
avgShuffleTime = avgShuffleTime / numReduces;
avgSortTime = avgSortTime / numReduces;
}
}
}

View File

@ -52,6 +52,7 @@ public class HsNavBlock extends HtmlBlock {
ul().
li().a(url("job", jobid), "Overview")._().
li().a(url("jobcounters", jobid), "Counters")._().
li().a(url("conf", jobid), "Configuration")._().
li().a(url("tasks", jobid, "m"), "Map tasks")._().
li().a(url("tasks", jobid, "r"), "Reduce tasks")._()._();
}

View File

@ -18,28 +18,32 @@
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.yarn.util.StringHelper.percent;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
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.postInitID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
import java.util.Collection;
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.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.webapp.App;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.webapp.SubView;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TD;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TFOOT;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.THEAD;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import com.google.common.base.Joiner;
@ -68,48 +72,160 @@ protected void render(Block html) {
h2($(TITLE));
return;
}
TBODY<TABLE<Hamlet>> tbody = html.
TaskType type = null;
String symbol = $(TASK_TYPE);
if (!symbol.isEmpty()) {
type = MRApps.taskType(symbol);
} else {
type = app.getTask().getType();
}
TR<THEAD<TABLE<Hamlet>>> headRow = html.
table("#attempts").
thead().
tr().
tr();
headRow.
th(".id", "Attempt").
th(".progress", "Progress").
th(".state", "State").
th(".node", "node").
th(".tsh", "Started").
th(".tsh", "Finished").
th(".tsh", "Elapsed").
th(".note", "Note")._()._().
tbody();
th(".tsh", "Start Time");
if(type == TaskType.REDUCE) {
headRow.th("Shuffle Finish Time");
headRow.th("Merge Finish Time");
}
headRow.th("Finish Time"); //Attempt
if(type == TaskType.REDUCE) {
headRow.th("Elapsed Time Shuffle"); //Attempt
headRow.th("Elapsed Time Merge"); //Attempt
headRow.th("Elapsed Time Reduce"); //Attempt
}
headRow.th("Elapsed Time").
th(".note", "Note");
TBODY<TABLE<Hamlet>> tbody = headRow._()._().tbody();
for (TaskAttempt ta : getTaskAttempts()) {
String taid = MRApps.toString(ta.getID());
String progress = percent(ta.getProgress());
ContainerId containerId = ta.getAssignedContainerID();
String nodeHttpAddr = ta.getNodeHttpAddress();
long startTime = ta.getLaunchTime();
long finishTime = ta.getFinishTime();
long elapsed = Times.elapsed(startTime, finishTime);
TD<TR<TBODY<TABLE<Hamlet>>>> nodeTd = tbody.
tr().
td(".id", taid).
td(".progress", progress).
td(".state", ta.getState().toString()).
long attemptStartTime = ta.getLaunchTime();
long shuffleFinishTime = -1;
long sortFinishTime = -1;
long attemptFinishTime = ta.getFinishTime();
long elapsedShuffleTime = -1;
long elapsedSortTime = -1;
long elapsedReduceTime = -1;
if(type == TaskType.REDUCE) {
shuffleFinishTime = ta.getShuffleFinishTime();
sortFinishTime = ta.getSortFinishTime();
elapsedShuffleTime =
Times.elapsed(attemptStartTime, shuffleFinishTime, false);
elapsedSortTime =
Times.elapsed(shuffleFinishTime, sortFinishTime, false);
elapsedReduceTime =
Times.elapsed(sortFinishTime, attemptFinishTime, false);
}
long attemptElapsed =
Times.elapsed(attemptStartTime, attemptFinishTime, false);
int sortId = ta.getID().getId() + (ta.getID().getTaskId().getId() * 10000);
TR<TBODY<TABLE<Hamlet>>> row = tbody.tr();
row.
td().
a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr);
if (containerId != null) {
String containerIdStr = ConverterUtils.toString(containerId);
nodeTd._(" ").
a(".logslink", url("http://", nodeHttpAddr, "yarn", "containerlogs",
containerIdStr), "logs");
br().$title(String.valueOf(sortId))._(). // sorting
_(taid)._().
td(ta.getState().toString()).
td().a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr)._();
row.td().
br().$title(String.valueOf(attemptStartTime))._().
_(Times.format(attemptStartTime))._();
if(type == TaskType.REDUCE) {
row.td().
br().$title(String.valueOf(shuffleFinishTime))._().
_(Times.format(shuffleFinishTime))._();
row.td().
br().$title(String.valueOf(sortFinishTime))._().
_(Times.format(sortFinishTime))._();
}
nodeTd._().
td(".ts", Times.format(startTime)).
td(".ts", Times.format(finishTime)).
td(".dt", StringUtils.formatTime(elapsed)).
td(".note", Joiner.on('\n').join(ta.getDiagnostics()))._();
row.
td().
br().$title(String.valueOf(attemptFinishTime))._().
_(Times.format(attemptFinishTime))._();
if(type == TaskType.REDUCE) {
row.td().
br().$title(String.valueOf(elapsedShuffleTime))._().
_(formatTime(elapsedShuffleTime))._();
row.td().
br().$title(String.valueOf(elapsedSortTime))._().
_(formatTime(elapsedSortTime))._();
row.td().
br().$title(String.valueOf(elapsedReduceTime))._().
_(formatTime(elapsedReduceTime))._();
}
tbody._()._();
row.
td().
br().$title(String.valueOf(attemptElapsed))._().
_(formatTime(attemptElapsed))._().
td(".note", Joiner.on('\n').join(ta.getDiagnostics()));
row._();
}
TR<TFOOT<TABLE<Hamlet>>> footRow = tbody._().tfoot().tr();
footRow.
th().input("search_init").$type(InputType.text).
$name("attempt_name").$value("Attempt")._()._().
th().input("search_init").$type(InputType.text).
$name("attempt_state").$value("State")._()._().
th().input("search_init").$type(InputType.text).
$name("attempt_node").$value("Node")._()._().
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")._()._();
footRow.
th().input("search_init").$type(InputType.text).
$name("merge_time").$value("Merge Time")._()._();
}
footRow.
th().input("search_init").$type(InputType.text).
$name("attempt_finish").$value("Finish Time")._()._();
if(type == TaskType.REDUCE) {
footRow.
th().input("search_init").$type(InputType.text).
$name("elapsed_shuffle_time").$value("Elapsed Shuffle Time")._()._();
footRow.
th().input("search_init").$type(InputType.text).
$name("elapsed_merge_time").$value("Elapsed Merge Time")._()._();
footRow.
th().input("search_init").$type(InputType.text).
$name("elapsed_reduce_time").$value("Elapsed Reduce Time")._()._();
}
footRow.
th().input("search_init").$type(InputType.text).
$name("attempt_elapsed").$value("Elapsed Time")._()._().
th().input("search_init").$type(InputType.text).
$name("note").$value("Note")._()._();
footRow._()._()._();
}
private String formatTime(long elapsed) {
return elapsed < 0 ? "N/A" : StringUtils.formatTime(elapsed);
}
/**
@ -138,6 +254,7 @@ protected Collection<TaskAttempt> getTaskAttempts() {
//Set up the java script and CSS for the attempts table
set(DATATABLES_ID, "attempts");
set(initID(DATATABLES, "attempts"), attemptsTableInit());
set(postInitID(DATATABLES, "attempts"), attemptsPostTableInit());
setTableStyles(html, "attempts");
}
@ -154,6 +271,49 @@ protected Collection<TaskAttempt> getTaskAttempts() {
* attempts table.
*/
private String attemptsTableInit() {
return tableInit().append("}").toString();
TaskType type = null;
String symbol = $(TASK_TYPE);
if (!symbol.isEmpty()) {
type = MRApps.taskType(symbol);
} else {
TaskId taskID = MRApps.toTaskID($(TASK_ID));
type = taskID.getTaskType();
}
StringBuilder b = tableInit().
append(",aoColumnDefs:[");
b.append("{'sType':'title-numeric', 'aTargets': [ 0");
if(type == TaskType.REDUCE) {
b.append(", 7, 8, 9, 10");
} else { //MAP
b.append(", 5");
}
b.append(" ] }");
b.append("]}");
return b.toString();
}
private String attemptsPostTableInit() {
return "var asInitVals = new Array();\n" +
"$('tfoot input').keyup( function () \n{"+
" attemptsDataTable.fnFilter( this.value, $('tfoot input').index(this) );\n"+
"} );\n"+
"$('tfoot input').each( function (i) {\n"+
" asInitVals[i] = this.value;\n"+
"} );\n"+
"$('tfoot input').focus( function () {\n"+
" if ( this.className == 'search_init' )\n"+
" {\n"+
" this.className = '';\n"+
" this.value = '';\n"+
" }\n"+
"} );\n"+
"$('tfoot input').blur( function (i) {\n"+
" if ( this.value == '' )\n"+
" {\n"+
" this.className = 'search_init';\n"+
" this.value = asInitVals[$('tfoot input').index(this)];\n"+
" }\n"+
"} );\n";
}
}

View File

@ -0,0 +1,235 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.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.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.TABLE;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TFOOT;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.THEAD;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import com.google.inject.Inject;
/**
* Render the a table of tasks for a given type.
*/
public class HsTasksBlock extends HtmlBlock {
final App app;
@Inject HsTasksBlock(App app) {
this.app = app;
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block)
*/
@Override protected void render(Block html) {
if (app.getJob() == null) {
html.
h2($(TITLE));
return;
}
TaskType type = null;
String symbol = $(TASK_TYPE);
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;
thead.tr().
th().$colspan(5).$class("ui-state-default")._("Task")._().
th().$colspan(attemptColSpan).$class("ui-state-default").
_("Successful Attempt")._().
_();
TR<THEAD<TABLE<Hamlet>>> theadRow = thead.
tr().
th("Name").
th("State").
th("Start Time").
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);
long attemptStartTime = -1;
long shuffleFinishTime = -1;
long sortFinishTime = -1;
long attemptFinishTime = -1;
long elapsedShuffleTime = -1;
long elapsedSortTime = -1;;
long elapsedReduceTime = -1;
long attemptElapsed = -1;
TaskAttempt successful = getSuccessfulAttempt(task);
if(successful != null) {
attemptStartTime = successful.getLaunchTime();
attemptFinishTime = successful.getFinishTime();
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);
}
attemptElapsed =
Times.elapsed(attemptStartTime, attemptFinishTime, false);
}
TR<TBODY<TABLE<Hamlet>>> row = tbody.tr();
row.
td().
br().$title(String.valueOf(task.getID().getId()))._(). // sorting
a(url("task", tid), tid)._().
td(report.getTaskState().toString()).
td().
br().$title(String.valueOf(startTime))._().
_(Times.format(startTime))._().
td().
br().$title(String.valueOf(finishTime))._().
_(Times.format(finishTime))._().
td().
br().$title(String.valueOf(elapsed))._().
_(formatTime(elapsed))._().
td().
br().$title(String.valueOf(attemptStartTime))._().
_(Times.format(attemptStartTime))._();
if(type == TaskType.REDUCE) {
row.td().
br().$title(String.valueOf(shuffleFinishTime))._().
_(Times.format(shuffleFinishTime))._();
row.td().
br().$title(String.valueOf(sortFinishTime))._().
_(Times.format(sortFinishTime))._();
}
row.
td().
br().$title(String.valueOf(attemptFinishTime))._().
_(Times.format(attemptFinishTime))._();
if(type == TaskType.REDUCE) {
row.td().
br().$title(String.valueOf(elapsedShuffleTime))._().
_(formatTime(elapsedShuffleTime))._();
row.td().
br().$title(String.valueOf(elapsedSortTime))._().
_(formatTime(elapsedSortTime))._();
row.td().
br().$title(String.valueOf(elapsedReduceTime))._().
_(formatTime(elapsedReduceTime))._();
}
row.td().
br().$title(String.valueOf(attemptElapsed))._().
_(formatTime(attemptElapsed))._();
row._();
}
TR<TFOOT<TABLE<Hamlet>>> footRow = tbody._().tfoot().tr();
footRow.th().input("search_init").$type(InputType.text).$name("task")
.$value("ID")._()._().th().input("search_init").$type(InputType.text)
.$name("state").$value("State")._()._().th().input("search_init")
.$type(InputType.text).$name("start_time").$value("Start Time")._()._()
.th().input("search_init").$type(InputType.text).$name("finish_time")
.$value("Finish Time")._()._().th().input("search_init")
.$type(InputType.text).$name("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")._()._();
footRow.th().input("search_init").$type(InputType.text)
.$name("merge_time").$value("Merge Time")._()._();
}
footRow.th().input("search_init").$type(InputType.text)
.$name("attempt_finish").$value("Finish Time")._()._();
if(type == TaskType.REDUCE) {
footRow.th().input("search_init").$type(InputType.text)
.$name("elapsed_shuffle_time").$value("Elapsed Shuffle Time")._()._();
footRow.th().input("search_init").$type(InputType.text)
.$name("elapsed_merge_time").$value("Elapsed Merge Time")._()._();
footRow.th().input("search_init").$type(InputType.text)
.$name("elapsed_reduce_time").$value("Elapsed Reduce Time")._()._();
}
footRow.th().input("search_init").$type(InputType.text)
.$name("attempt_elapsed").$value("Elapsed Time")._()._();
footRow._()._()._();
}
private String formatTime(long elapsed) {
return elapsed < 0 ? "N/A" : StringUtils.formatTime(elapsed);
}
private TaskAttempt getSuccessfulAttempt(Task task) {
for(TaskAttempt attempt: task.getAttempts().values()) {
if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
return attempt;
}
}
return null;
}
}

View File

@ -18,13 +18,16 @@
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
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.postInitID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
import org.apache.hadoop.mapreduce.v2.app.webapp.TasksBlock;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.webapp.SubView;
/**
@ -41,15 +44,16 @@ public class HsTasksPage extends HsView {
set(DATATABLES_ID, "tasks");
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
set(initID(DATATABLES, "tasks"), tasksTableInit());
set(postInitID(DATATABLES, "tasks"), jobsPostTableInit());
setTableStyles(html, "tasks");
}
/**
* The content of this page is the TasksBlock
* @return TasksBlock.class
* @return HsTasksBlock.class
*/
@Override protected Class<? extends SubView> content() {
return TasksBlock.class;
return HsTasksBlock.class;
}
/**
@ -57,9 +61,45 @@ public class HsTasksPage extends HsView {
* for the tasks table.
*/
private String tasksTableInit() {
return tableInit().
append(",aoColumns:[{sType:'title-numeric'},{sType:'title-numeric',").
append("bSearchable:false},null,{sType:'title-numeric'},").
append("{sType:'title-numeric'},{sType:'title-numeric'}]}").toString();
TaskType type = null;
String symbol = $(TASK_TYPE);
if (!symbol.isEmpty()) {
type = MRApps.taskType(symbol);
}
StringBuilder b = tableInit().
append(",aoColumnDefs:[");
b.append("{'sType':'title-numeric', 'aTargets': [ 0, 4");
if(type == TaskType.REDUCE) {
b.append(", 9, 10, 11, 12");
} else { //MAP
b.append(", 7");
}
b.append(" ] }");
b.append("]}");
return b.toString();
}
private String jobsPostTableInit() {
return "var asInitVals = new Array();\n" +
"$('tfoot input').keyup( function () \n{"+
" tasksDataTable.fnFilter( this.value, $('tfoot input').index(this) );\n"+
"} );\n"+
"$('tfoot input').each( function (i) {\n"+
" asInitVals[i] = this.value;\n"+
"} );\n"+
"$('tfoot input').focus( function () {\n"+
" if ( this.className == 'search_init' )\n"+
" {\n"+
" this.className = '';\n"+
" this.value = '';\n"+
" }\n"+
"} );\n"+
"$('tfoot input').blur( function (i) {\n"+
" if ( this.value == '' )\n"+
" {\n"+
" this.className = 'search_init';\n"+
" this.value = asInitVals[$('tfoot input').index(this)];\n"+
" }\n"+
"} );\n";
}
}

View File

@ -39,6 +39,7 @@ public void setup() {
route("/", HsController.class);
route("/app", HsController.class);
route(pajoin("/job", JOB_ID), HsController.class, "job");
route(pajoin("/conf", JOB_ID), HsController.class, "conf");
route(pajoin("/jobcounters", JOB_ID), HsController.class, "jobCounters");
route(pajoin("/tasks", JOB_ID, TASK_TYPE), HsController.class, "tasks");
route(pajoin("/attempts", JOB_ID, TASK_TYPE, ATTEMPT_STATE),

View File

@ -19,25 +19,36 @@
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.ATTEMPT_STATE;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.JOB_ID;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
import static org.junit.Assert.assertEquals;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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.AppContext;
import org.apache.hadoop.mapreduce.v2.app.MockJobs;
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.webapp.AMParams;
import org.apache.hadoop.mapreduce.v2.app.webapp.TestAMWebApp;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.Apps;
import org.apache.hadoop.yarn.webapp.test.WebAppTests;
import org.junit.Test;
import com.google.inject.Injector;
public class TestHSWebApp {
private static final Log LOG = LogFactory.getLog(TestHSWebApp.class);
static class TestAppContext implements AppContext {
final ApplicationAttemptId appAttemptID;
@ -111,16 +122,53 @@ public long getStartTime() {
}
@Test public void testJobView() {
WebAppTests.testPage(HsJobPage.class, AppContext.class, new TestAppContext());
LOG.info("HsJobPage");
AppContext appContext = new TestAppContext();
Map<String, String> params = TestAMWebApp.getJobParams(appContext);
WebAppTests.testPage(HsJobPage.class, AppContext.class, appContext, params);
}
@Test public void testTasksView() {
WebAppTests.testPage(HsTasksPage.class, AppContext.class,
new TestAppContext());
@Test
public void testTasksView() {
LOG.info("HsTasksPage");
AppContext appContext = new TestAppContext();
Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
WebAppTests.testPage(HsTasksPage.class, AppContext.class, appContext,
params);
}
@Test public void testTaskView() {
WebAppTests.testPage(HsTaskPage.class, AppContext.class,
@Test
public void testTaskView() {
LOG.info("HsTaskPage");
AppContext appContext = new TestAppContext();
Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
WebAppTests
.testPage(HsTaskPage.class, AppContext.class, appContext, params);
}
@Test public void testAttemptsWithJobView() {
LOG.info("HsAttemptsPage with data");
TestAppContext ctx = new TestAppContext();
JobId id = ctx.getAllJobs().keySet().iterator().next();
Map<String, String> params = new HashMap<String,String>();
params.put(JOB_ID, id.toString());
params.put(TASK_TYPE, "m");
params.put(ATTEMPT_STATE, "SUCCESSFUL");
WebAppTests.testPage(HsAttemptsPage.class, AppContext.class,
ctx, params);
}
@Test public void testAttemptsView() {
LOG.info("HsAttemptsPage");
AppContext appContext = new TestAppContext();
Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
WebAppTests.testPage(HsAttemptsPage.class, AppContext.class,
appContext, params);
}
@Test public void testConfView() {
LOG.info("HsConfPage");
WebAppTests.testPage(HsConfPage.class, AppContext.class,
new TestAppContext());
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.mapred;
import java.io.IOException;
import java.security.PrivilegedAction;
import java.util.HashMap;
import java.util.Map;
@ -28,12 +29,13 @@
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo;
public class ClientCache {
@ -72,16 +74,21 @@ synchronized ClientServiceDelegate getClient(JobID jobId) {
private MRClientProtocol instantiateHistoryProxy()
throws IOException {
String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
final String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
LOG.info("Connecting to HistoryServer at: " + serviceAddr);
Configuration myConf = new Configuration(conf);
//TODO This should ideally be using it's own class (instead of ClientRMSecurityInfo)
final Configuration myConf = new Configuration(conf);
myConf.setClass(YarnConfiguration.YARN_SECURITY_INFO,
ClientRMSecurityInfo.class, SecurityInfo.class);
YarnRPC rpc = YarnRPC.create(myConf);
ClientHSSecurityInfo.class, SecurityInfo.class);
final YarnRPC rpc = YarnRPC.create(myConf);
LOG.info("Connected to HistoryServer at: " + serviceAddr);
UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
return currentUser.doAs(new PrivilegedAction<MRClientProtocol>() {
@Override
public MRClientProtocol run() {
return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
NetUtils.createSocketAddr(serviceAddr), myConf);
}
});
}
}

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.mapred;
import java.io.IOException;
import java.net.URI;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
@ -33,7 +32,6 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.io.DataOutputBuffer;
@ -55,7 +53,6 @@
import org.apache.hadoop.mapreduce.TaskTrackerInfo;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.mapreduce.v2.MRConstants;
@ -72,6 +69,7 @@
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@ -237,7 +235,6 @@ public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts)
// Construct necessary information to start the MR AM
ApplicationSubmissionContext appContext =
createApplicationSubmissionContext(conf, jobSubmitDir, ts);
setupDistributedCache(conf, appContext);
// XXX Remove
in.close();
@ -273,15 +270,17 @@ private LocalResource createApplicationResource(FileContext fs, Path p)
public ApplicationSubmissionContext createApplicationSubmissionContext(
Configuration jobConf,
String jobSubmitDir, Credentials ts) throws IOException {
ApplicationSubmissionContext appContext =
recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
ApplicationId applicationId = resMgrDelegate.getApplicationId();
appContext.setApplicationId(applicationId);
// Setup resource requirements
Resource capability = recordFactory.newRecordInstance(Resource.class);
capability.setMemory(conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
MRJobConfig.DEFAULT_MR_AM_VMEM_MB));
LOG.info("AppMaster capability = " + capability);
appContext.setMasterCapability(capability);
// Setup LocalResources
Map<String, LocalResource> localResources =
new HashMap<String, LocalResource>();
Path jobConfPath = new Path(jobSubmitDir, MRConstants.JOB_CONF_FILE);
@ -292,14 +291,11 @@ public ApplicationSubmissionContext createApplicationSubmissionContext(
LOG.debug("Creating setup context, jobSubmitDir url is "
+ yarnUrlForJobSubmitDir);
appContext.setResource(MRConstants.JOB_SUBMIT_DIR,
yarnUrlForJobSubmitDir);
appContext.setResourceTodo(MRConstants.JOB_CONF_FILE,
localResources.put(MRConstants.JOB_CONF_FILE,
createApplicationResource(defaultFileContext,
jobConfPath));
if (jobConf.get(MRJobConfig.JAR) != null) {
appContext.setResourceTodo(MRConstants.JOB_JAR,
localResources.put(MRConstants.JOB_JAR,
createApplicationResource(defaultFileContext,
new Path(jobSubmitDir, MRConstants.JOB_JAR)));
} else {
@ -312,30 +308,21 @@ public ApplicationSubmissionContext createApplicationSubmissionContext(
// TODO gross hack
for (String s : new String[] { "job.split", "job.splitmetainfo",
MRConstants.APPLICATION_TOKENS_FILE }) {
appContext.setResourceTodo(
localResources.put(
MRConstants.JOB_SUBMIT_DIR + "/" + s,
createApplicationResource(defaultFileContext, new Path(jobSubmitDir, s)));
createApplicationResource(defaultFileContext,
new Path(jobSubmitDir, s)));
}
// TODO: Only if security is on.
List<String> fsTokens = new ArrayList<String>();
for (Token<? extends TokenIdentifier> token : ts.getAllTokens()) {
fsTokens.add(token.encodeToUrlString());
}
// TODO - Remove this!
appContext.addAllFsTokens(fsTokens);
// Setup security tokens
ByteBuffer securityTokens = null;
if (UserGroupInformation.isSecurityEnabled()) {
DataOutputBuffer dob = new DataOutputBuffer();
ts.writeTokenStorageToStream(dob);
appContext.setFsTokensTodo(ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
}
// Add queue information
appContext.setQueue(jobConf.get(JobContext.QUEUE_NAME, JobConf.DEFAULT_QUEUE_NAME));
// Add job name
appContext.setApplicationName(jobConf.get(JobContext.JOB_NAME, "N/A"));
// Add the command line
// Setup the command to run the AM
String javaHome = "$JAVA_HOME";
Vector<CharSequence> vargs = new Vector<CharSequence>(8);
vargs.add(javaHome + "/bin/java");
@ -346,13 +333,6 @@ public ApplicationSubmissionContext createApplicationSubmissionContext(
vargs.add(conf.get(MRJobConfig.MR_AM_COMMAND_OPTS,
MRJobConfig.DEFAULT_MR_AM_COMMAND_OPTS));
// Add { job jar, MR app jar } to classpath.
Map<String, String> environment = new HashMap<String, String>();
MRApps.setInitialClasspath(environment);
MRApps.addToClassPath(environment, MRConstants.JOB_JAR);
MRApps.addToClassPath(environment,
MRConstants.YARN_MAPREDUCE_APP_JAR_PATH);
appContext.addAllEnvironment(environment);
vargs.add("org.apache.hadoop.mapreduce.v2.app.MRAppMaster");
vargs.add(String.valueOf(applicationId.getClusterTimestamp()));
vargs.add(String.valueOf(applicationId.getId()));
@ -371,139 +351,42 @@ public ApplicationSubmissionContext createApplicationSubmissionContext(
LOG.info("Command to launch container for ApplicationMaster is : "
+ mergedCommand);
appContext.addAllCommands(vargsFinal);
// TODO: RM should get this from RPC.
appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
// Setup the environment - Add { job jar, MR app jar } to classpath.
Map<String, String> environment = new HashMap<String, String>();
MRApps.setInitialClasspath(environment);
MRApps.addToClassPath(environment, MRConstants.JOB_JAR);
MRApps.addToClassPath(environment,
MRConstants.YARN_MAPREDUCE_APP_JAR_PATH);
// Parse distributed cache
MRApps.setupDistributedCache(jobConf, localResources, environment);
// Setup ContainerLaunchContext for AM container
ContainerLaunchContext amContainer =
recordFactory.newRecordInstance(ContainerLaunchContext.class);
amContainer.setResource(capability); // Resource (mem) required
amContainer.setLocalResources(localResources); // Local resources
amContainer.setEnvironment(environment); // Environment
amContainer.setCommands(vargsFinal); // Command for AM
amContainer.setContainerTokens(securityTokens); // Security tokens
// Set up the ApplicationSubmissionContext
ApplicationSubmissionContext appContext =
recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
appContext.setApplicationId(applicationId); // ApplicationId
appContext.setUser( // User name
UserGroupInformation.getCurrentUser().getShortUserName());
appContext.setQueue( // Queue name
jobConf.get(JobContext.QUEUE_NAME,
YarnConfiguration.DEFAULT_QUEUE_NAME));
appContext.setApplicationName( // Job name
jobConf.get(JobContext.JOB_NAME,
YarnConfiguration.DEFAULT_APPLICATION_NAME));
appContext.setAMContainerSpec(amContainer); // AM Container
return appContext;
}
/**
* * TODO: Copied for now from TaskAttemptImpl.java ... fixme
* @param strs
* @return
*/
private static long[] parseTimeStamps(String[] strs) {
if (null == strs) {
return null;
}
long[] result = new long[strs.length];
for(int i=0; i < strs.length; ++i) {
result[i] = Long.parseLong(strs[i]);
}
return result;
}
/**
* TODO: Copied for now from TaskAttemptImpl.java ... fixme
*
* TODO: This is currently needed in YarnRunner as user code like setupJob,
* cleanupJob may need access to dist-cache. Once we separate distcache for
* maps, reduces, setup etc, this can include only a subset of artificats.
* This is also needed for uberAM case where we run everything inside AM.
*/
private void setupDistributedCache(Configuration conf,
ApplicationSubmissionContext container) throws IOException {
// Cache archives
parseDistributedCacheArtifacts(conf, container, LocalResourceType.ARCHIVE,
DistributedCache.getCacheArchives(conf),
parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)),
getFileSizes(conf, MRJobConfig.CACHE_ARCHIVES_SIZES),
DistributedCache.getArchiveVisibilities(conf),
DistributedCache.getArchiveClassPaths(conf));
// Cache files
parseDistributedCacheArtifacts(conf, container, LocalResourceType.FILE,
DistributedCache.getCacheFiles(conf),
parseTimeStamps(DistributedCache.getFileTimestamps(conf)),
getFileSizes(conf, MRJobConfig.CACHE_FILES_SIZES),
DistributedCache.getFileVisibilities(conf),
DistributedCache.getFileClassPaths(conf));
}
// TODO - Move this to MR!
// Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[], long[], boolean[], Path[], FileType)
private void parseDistributedCacheArtifacts(Configuration conf,
ApplicationSubmissionContext container, LocalResourceType type,
URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[],
Path[] pathsToPutOnClasspath) throws IOException {
if (uris != null) {
// Sanity check
if ((uris.length != timestamps.length) || (uris.length != sizes.length) ||
(uris.length != visibilities.length)) {
throw new IllegalArgumentException("Invalid specification for " +
"distributed-cache artifacts of type " + type + " :" +
" #uris=" + uris.length +
" #timestamps=" + timestamps.length +
" #visibilities=" + visibilities.length
);
}
Map<String, Path> classPaths = new HashMap<String, Path>();
if (pathsToPutOnClasspath != null) {
for (Path p : pathsToPutOnClasspath) {
FileSystem fs = p.getFileSystem(conf);
p = p.makeQualified(fs.getUri(), fs.getWorkingDirectory());
classPaths.put(p.toUri().getPath().toString(), p);
}
}
for (int i = 0; i < uris.length; ++i) {
URI u = uris[i];
Path p = new Path(u);
FileSystem fs = p.getFileSystem(conf);
p = fs.resolvePath(
p.makeQualified(fs.getUri(), fs.getWorkingDirectory()));
// Add URI fragment or just the filename
Path name = new Path((null == u.getFragment())
? p.getName()
: u.getFragment());
if (name.isAbsolute()) {
throw new IllegalArgumentException("Resource name must be relative");
}
String linkName = name.toUri().getPath();
container.setResourceTodo(
linkName,
createLocalResource(
p.toUri(), type,
visibilities[i]
? LocalResourceVisibility.PUBLIC
: LocalResourceVisibility.PRIVATE,
sizes[i], timestamps[i])
);
if (classPaths.containsKey(u.getPath())) {
Map<String, String> environment = container.getAllEnvironment();
MRApps.addToClassPath(environment, linkName);
}
}
}
}
// TODO - Move this to MR!
private static long[] getFileSizes(Configuration conf, String key) {
String[] strs = conf.getStrings(key);
if (strs == null) {
return null;
}
long[] result = new long[strs.length];
for(int i=0; i < strs.length; ++i) {
result[i] = Long.parseLong(strs[i]);
}
return result;
}
private LocalResource createLocalResource(URI uri,
LocalResourceType type, LocalResourceVisibility visibility,
long size, long timestamp) throws IOException {
LocalResource resource = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(LocalResource.class);
resource.setResource(ConverterUtils.getYarnUrlFromURI(uri));
resource.setType(type);
resource.setVisibility(visibility);
resource.setSize(size);
resource.setTimestamp(timestamp);
return resource;
}
@Override
public void setJobPriority(JobID arg0, String arg1) throws IOException,
InterruptedException {

View File

@ -0,0 +1 @@
org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo

View File

@ -18,14 +18,8 @@
package org.apache.hadoop.yarn.api.records;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.ClientRMProtocol;
/**
@ -36,26 +30,17 @@
* <p>It includes details such as:
* <ul>
* <li>{@link ApplicationId} of the application.</li>
* <li>
* {@link Resource} necessary to run the <code>ApplicationMaster</code>.
* </li>
* <li>Application user.</li>
* <li>Application name.</li>
* <li>{@link Priority} of the application.</li>
* <li>Security tokens (if security is enabled).</li>
* <li>
* {@link LocalResource} necessary for running the
* <code>ApplicationMaster</code> container such
* as binaries, jar, shared-objects, side-files etc.
* {@link ContainerLaunchContext} of the container in which the
* <code>ApplicationMaster</code> is executed.
* </li>
* <li>
* Environment variables for the launched <code>ApplicationMaster</code>
* process.
* </li>
* <li>Command to launch the <code>ApplicationMaster</code>.</li>
* </ul>
* </p>
*
* @see ContainerLaunchContext
* @see ClientRMProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)
*/
@Public
@ -143,198 +128,25 @@ public interface ApplicationSubmissionContext {
public void setUser(String user);
/**
* Get the <code>Resource</code> required to run the
* <code>ApplicationMaster</code>.
* @return <code>Resource</code> required to run the
* <code>ApplicationMaster</code>
* Get the <code>ContainerLaunchContext</code> to describe the
* <code>Container</code> with which the <code>ApplicationMaster</code> is
* launched.
* @return <code>ContainerLaunchContext</code> for the
* <code>ApplicationMaster</code> container
*/
@Public
@Stable
public Resource getMasterCapability();
public ContainerLaunchContext getAMContainerSpec();
/**
* Set <code>Resource</code> required to run the
* <code>ApplicationMaster</code>.
* @param masterCapability <code>Resource</code> required to run the
* <code>ApplicationMaster</code>
* Set the <code>ContainerLaunchContext</code> to describe the
* <code>Container</code> with which the <code>ApplicationMaster</code> is
* launched.
* @param amContainer <code>ContainerLaunchContext</code> for the
* <code>ApplicationMaster</code> container
*/
@Public
@Stable
public void setMasterCapability(Resource masterCapability);
public void setAMContainerSpec(ContainerLaunchContext amContainer);
@Private
@Unstable
public Map<String, URL> getAllResources();
@Private
@Unstable
public URL getResource(String key);
@Private
@Unstable
public void addAllResources(Map<String, URL> resources);
@Private
@Unstable
public void setResource(String key, URL url);
@Private
@Unstable
public void removeResource(String key);
@Private
@Unstable
public void clearResources();
/**
* Get all the <code>LocalResource</code> required to run the
* <code>ApplicationMaster</code>.
* @return <code>LocalResource</code> required to run the
* <code>ApplicationMaster</code>
*/
@Public
@Stable
public Map<String, LocalResource> getAllResourcesTodo();
@Private
@Unstable
public LocalResource getResourceTodo(String key);
/**
* Add all the <code>LocalResource</code> required to run the
* <code>ApplicationMaster</code>.
* @param resources all <code>LocalResource</code> required to run the
* <code>ApplicationMaster</code>
*/
@Public
@Stable
public void addAllResourcesTodo(Map<String, LocalResource> resources);
@Private
@Unstable
public void setResourceTodo(String key, LocalResource localResource);
@Private
@Unstable
public void removeResourceTodo(String key);
@Private
@Unstable
public void clearResourcesTodo();
@Private
@Unstable
public List<String> getFsTokenList();
@Private
@Unstable
public String getFsToken(int index);
@Private
@Unstable
public int getFsTokenCount();
@Private
@Unstable
public void addAllFsTokens(List<String> fsTokens);
@Private
@Unstable
public void addFsToken(String fsToken);
@Private
@Unstable
public void removeFsToken(int index);
@Private
@Unstable
public void clearFsTokens();
/**
* Get <em>file-system tokens</em> for the <code>ApplicationMaster</code>.
* @return file-system tokens for the <code>ApplicationMaster</code>
*/
@Public
@Stable
public ByteBuffer getFsTokensTodo();
/**
* Set <em>file-system tokens</em> for the <code>ApplicationMaster</code>.
* @param fsTokens file-system tokens for the <code>ApplicationMaster</code>
*/
@Public
@Stable
public void setFsTokensTodo(ByteBuffer fsTokens);
/**
* Get the <em>environment variables</em> for the
* <code>ApplicationMaster</code>.
* @return environment variables for the <code>ApplicationMaster</code>
*/
@Public
@Stable
public Map<String, String> getAllEnvironment();
@Private
@Unstable
public String getEnvironment(String key);
/**
* Add all of the <em>environment variables</em> for the
* <code>ApplicationMaster</code>.
* @param environment environment variables for the
* <code>ApplicationMaster</code>
*/
@Public
@Stable
public void addAllEnvironment(Map<String, String> environment);
@Private
@Unstable
public void setEnvironment(String key, String env);
@Private
@Unstable
public void removeEnvironment(String key);
@Private
@Unstable
public void clearEnvironment();
/**
* Get the <em>commands</em> to launch the <code>ApplicationMaster</code>.
* @return commands to launch the <code>ApplicationMaster</code>
*/
@Public
@Stable
public List<String> getCommandList();
@Private
@Unstable
public String getCommand(int index);
@Private
@Unstable
public int getCommandCount();
/**
* Add all of the <em>commands</em> to launch the
* <code>ApplicationMaster</code>.
* @param commands commands to launch the <code>ApplicationMaster</code>
*/
@Public
@Stable
public void addAllCommands(List<String> commands);
@Private
@Unstable
public void addCommand(String command);
@Private
@Unstable
public void removeCommand(int index);
@Private
@Unstable
public void clearCommands();
}

View File

@ -156,7 +156,7 @@ public interface ContainerLaunchContext {
*/
@Public
@Stable
Map<String, String> getEnv();
Map<String, String> getEnvironment();
/**
* Add <em>environment variables</em> for the container.
@ -164,7 +164,7 @@ public interface ContainerLaunchContext {
*/
@Public
@Stable
void setEnv(Map<String, String> environment);
void setEnvironment(Map<String, String> environment);
/**
* Get the list of <em>commands</em> for launching the container.

View File

@ -18,56 +18,35 @@
package org.apache.hadoop.yarn.api.records.impl.pb;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.ProtoBase;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
import org.apache.hadoop.yarn.proto.YarnProtos.StringLocalResourceMapProto;
import org.apache.hadoop.yarn.proto.YarnProtos.StringStringMapProto;
import org.apache.hadoop.yarn.proto.YarnProtos.StringURLMapProto;
import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSubmissionContextProto> implements ApplicationSubmissionContext {
ApplicationSubmissionContextProto proto = ApplicationSubmissionContextProto.getDefaultInstance();
public class ApplicationSubmissionContextPBImpl
extends ProtoBase<ApplicationSubmissionContextProto>
implements ApplicationSubmissionContext {
ApplicationSubmissionContextProto proto =
ApplicationSubmissionContextProto.getDefaultInstance();
ApplicationSubmissionContextProto.Builder builder = null;
boolean viaProto = false;
private ApplicationId applicationId = null;
private Resource masterCapability = null;
private Map<String, URL> resources = null;
private Map<String, LocalResource> resourcesTodo = null;
private List<String> fsTokenList = null;
private ByteBuffer fsTokenTodo = null;
private Map<String, String> environment = null;
private List<String> commandList = null;
private Priority priority = null;
private ContainerLaunchContext amContainer = null;
public ApplicationSubmissionContextPBImpl() {
builder = ApplicationSubmissionContextProto.newBuilder();
}
public ApplicationSubmissionContextPBImpl(ApplicationSubmissionContextProto proto) {
public ApplicationSubmissionContextPBImpl(
ApplicationSubmissionContextProto proto) {
this.proto = proto;
viaProto = true;
}
@ -83,30 +62,12 @@ private void mergeLocalToBuilder() {
if (this.applicationId != null) {
builder.setApplicationId(convertToProtoFormat(this.applicationId));
}
if (this.masterCapability != null) {
builder.setMasterCapability(convertToProtoFormat(this.masterCapability));
}
if (this.resources != null) {
addResourcesToProto();
}
if (this.resourcesTodo != null) {
addResourcesTodoToProto();
}
if (this.fsTokenList != null) {
addFsTokenListToProto();
}
if (this.fsTokenTodo != null) {
builder.setFsTokensTodo(convertToProtoFormat(this.fsTokenTodo));
}
if (this.environment != null) {
addEnvironmentToProto();
}
if (this.commandList != null) {
addCommandsToProto();
}
if (this.priority != null) {
builder.setPriority(convertToProtoFormat(this.priority));
}
if (this.amContainer != null) {
builder.setAmContainerSpec(convertToProtoFormat(this.amContainer));
}
}
private void mergeLocalToProto() {
@ -145,6 +106,7 @@ public void setPriority(Priority priority) {
builder.clearPriority();
this.priority = priority;
}
@Override
public ApplicationId getApplicationId() {
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@ -165,6 +127,7 @@ public void setApplicationId(ApplicationId applicationId) {
builder.clearApplicationId();
this.applicationId = applicationId;
}
@Override
public String getApplicationName() {
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@ -183,403 +146,7 @@ public void setApplicationName(String applicationName) {
}
builder.setApplicationName((applicationName));
}
@Override
public Resource getMasterCapability() {
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
if (this.masterCapability != null) {
return masterCapability;
} // Else via proto
if (!p.hasMasterCapability()) {
return null;
}
masterCapability = convertFromProtoFormat(p.getMasterCapability());
return this.masterCapability;
}
@Override
public void setMasterCapability(Resource masterCapability) {
maybeInitBuilder();
if (masterCapability == null)
builder.clearMasterCapability();
this.masterCapability = masterCapability;
}
@Override
public Map<String, URL> getAllResources() {
initResources();
return this.resources;
}
@Override
public URL getResource(String key) {
initResources();
return this.resources.get(key);
}
private void initResources() {
if (this.resources != null) {
return;
}
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
List<StringURLMapProto> mapAsList = p.getResourcesList();
this.resources = new HashMap<String, URL>();
for (StringURLMapProto c : mapAsList) {
this.resources.put(c.getKey(), convertFromProtoFormat(c.getValue()));
}
}
@Override
public void addAllResources(final Map<String, URL> resources) {
if (resources == null)
return;
initResources();
this.resources.putAll(resources);
}
private void addResourcesToProto() {
maybeInitBuilder();
builder.clearResources();
if (this.resources == null)
return;
Iterable<StringURLMapProto> iterable = new Iterable<StringURLMapProto>() {
@Override
public Iterator<StringURLMapProto> iterator() {
return new Iterator<StringURLMapProto>() {
Iterator<String> keyIter = resources.keySet().iterator();
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public StringURLMapProto next() {
String key = keyIter.next();
return StringURLMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(resources.get(key))).build();
}
@Override
public boolean hasNext() {
return keyIter.hasNext();
}
};
}
};
builder.addAllResources(iterable);
}
@Override
public void setResource(String key, URL val) {
initResources();
this.resources.put(key, val);
}
@Override
public void removeResource(String key) {
initResources();
this.resources.remove(key);
}
@Override
public void clearResources() {
initResources();
this.resources.clear();
}
@Override
public Map<String, LocalResource> getAllResourcesTodo() {
initResourcesTodo();
return this.resourcesTodo;
}
@Override
public LocalResource getResourceTodo(String key) {
initResourcesTodo();
return this.resourcesTodo.get(key);
}
private void initResourcesTodo() {
if (this.resourcesTodo != null) {
return;
}
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
List<StringLocalResourceMapProto> mapAsList = p.getResourcesTodoList();
this.resourcesTodo = new HashMap<String, LocalResource>();
for (StringLocalResourceMapProto c : mapAsList) {
this.resourcesTodo.put(c.getKey(), convertFromProtoFormat(c.getValue()));
}
}
@Override
public void addAllResourcesTodo(final Map<String, LocalResource> resourcesTodo) {
if (resourcesTodo == null)
return;
initResourcesTodo();
this.resourcesTodo.putAll(resourcesTodo);
}
private void addResourcesTodoToProto() {
maybeInitBuilder();
builder.clearResourcesTodo();
if (resourcesTodo == null)
return;
Iterable<StringLocalResourceMapProto> iterable = new Iterable<StringLocalResourceMapProto>() {
@Override
public Iterator<StringLocalResourceMapProto> iterator() {
return new Iterator<StringLocalResourceMapProto>() {
Iterator<String> keyIter = resourcesTodo.keySet().iterator();
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public StringLocalResourceMapProto next() {
String key = keyIter.next();
return StringLocalResourceMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(resourcesTodo.get(key))).build();
}
@Override
public boolean hasNext() {
return keyIter.hasNext();
}
};
}
};
builder.addAllResourcesTodo(iterable);
}
@Override
public void setResourceTodo(String key, LocalResource val) {
initResourcesTodo();
this.resourcesTodo.put(key, val);
}
@Override
public void removeResourceTodo(String key) {
initResourcesTodo();
this.resourcesTodo.remove(key);
}
@Override
public void clearResourcesTodo() {
initResourcesTodo();
this.resourcesTodo.clear();
}
@Override
public List<String> getFsTokenList() {
initFsTokenList();
return this.fsTokenList;
}
@Override
public String getFsToken(int index) {
initFsTokenList();
return this.fsTokenList.get(index);
}
@Override
public int getFsTokenCount() {
initFsTokenList();
return this.fsTokenList.size();
}
private void initFsTokenList() {
if (this.fsTokenList != null) {
return;
}
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
List<String> list = p.getFsTokensList();
this.fsTokenList = new ArrayList<String>();
for (String c : list) {
this.fsTokenList.add(c);
}
}
@Override
public void addAllFsTokens(final List<String> fsTokens) {
if (fsTokens == null)
return;
initFsTokenList();
this.fsTokenList.addAll(fsTokens);
}
private void addFsTokenListToProto() {
maybeInitBuilder();
builder.clearFsTokens();
builder.addAllFsTokens(this.fsTokenList);
}
@Override
public void addFsToken(String fsTokens) {
initFsTokenList();
this.fsTokenList.add(fsTokens);
}
@Override
public void removeFsToken(int index) {
initFsTokenList();
this.fsTokenList.remove(index);
}
@Override
public void clearFsTokens() {
initFsTokenList();
this.fsTokenList.clear();
}
@Override
public ByteBuffer getFsTokensTodo() {
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
if (this.fsTokenTodo != null) {
return this.fsTokenTodo;
}
if (!p.hasFsTokensTodo()) {
return null;
}
this.fsTokenTodo = convertFromProtoFormat(p.getFsTokensTodo());
return this.fsTokenTodo;
}
@Override
public void setFsTokensTodo(ByteBuffer fsTokensTodo) {
maybeInitBuilder();
if (fsTokensTodo == null)
builder.clearFsTokensTodo();
this.fsTokenTodo = fsTokensTodo;
}
@Override
public Map<String, String> getAllEnvironment() {
initEnvironment();
return this.environment;
}
@Override
public String getEnvironment(String key) {
initEnvironment();
return this.environment.get(key);
}
private void initEnvironment() {
if (this.environment != null) {
return;
}
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
List<StringStringMapProto> mapAsList = p.getEnvironmentList();
this.environment = new HashMap<String, String>();
for (StringStringMapProto c : mapAsList) {
this.environment.put(c.getKey(), c.getValue());
}
}
@Override
public void addAllEnvironment(Map<String, String> environment) {
if (environment == null)
return;
initEnvironment();
this.environment.putAll(environment);
}
private void addEnvironmentToProto() {
maybeInitBuilder();
builder.clearEnvironment();
if (environment == null)
return;
Iterable<StringStringMapProto> iterable = new Iterable<StringStringMapProto>() {
@Override
public Iterator<StringStringMapProto> iterator() {
return new Iterator<StringStringMapProto>() {
Iterator<String> keyIter = environment.keySet().iterator();
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public StringStringMapProto next() {
String key = keyIter.next();
return StringStringMapProto.newBuilder().setKey(key).setValue((environment.get(key))).build();
}
@Override
public boolean hasNext() {
return keyIter.hasNext();
}
};
}
};
builder.addAllEnvironment(iterable);
}
@Override
public void setEnvironment(String key, String val) {
initEnvironment();
this.environment.put(key, val);
}
@Override
public void removeEnvironment(String key) {
initEnvironment();
this.environment.remove(key);
}
@Override
public void clearEnvironment() {
initEnvironment();
this.environment.clear();
}
@Override
public List<String> getCommandList() {
initCommandList();
return this.commandList;
}
@Override
public String getCommand(int index) {
initCommandList();
return this.commandList.get(index);
}
@Override
public int getCommandCount() {
initCommandList();
return this.commandList.size();
}
private void initCommandList() {
if (this.commandList != null) {
return;
}
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
List<String> list = p.getCommandList();
this.commandList = new ArrayList<String>();
for (String c : list) {
this.commandList.add(c);
}
}
@Override
public void addAllCommands(final List<String> command) {
if (command == null)
return;
initCommandList();
this.commandList.addAll(command);
}
private void addCommandsToProto() {
maybeInitBuilder();
builder.clearCommand();
if (this.commandList == null)
return;
builder.addAllCommand(this.commandList);
}
@Override
public void addCommand(String command) {
initCommandList();
this.commandList.add(command);
}
@Override
public void removeCommand(int index) {
initCommandList();
this.commandList.remove(index);
}
@Override
public void clearCommands() {
initCommandList();
this.commandList.clear();
}
@Override
public String getQueue() {
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@ -598,6 +165,7 @@ public void setQueue(String queue) {
}
builder.setQueue((queue));
}
@Override
public String getUser() {
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@ -617,6 +185,28 @@ public void setUser(String user) {
builder.setUser((user));
}
@Override
public ContainerLaunchContext getAMContainerSpec() {
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
if (this.amContainer != null) {
return amContainer;
} // Else via proto
if (!p.hasAmContainerSpec()) {
return null;
}
amContainer = convertFromProtoFormat(p.getAmContainerSpec());
return amContainer;
}
@Override
public void setAMContainerSpec(ContainerLaunchContext amContainer) {
maybeInitBuilder();
if (amContainer == null) {
builder.clearAmContainerSpec();
}
this.amContainer = amContainer;
}
private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
return new PriorityPBImpl(p);
}
@ -633,28 +223,12 @@ private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
return ((ApplicationIdPBImpl)t).getProto();
}
private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
return new ResourcePBImpl(p);
private ContainerLaunchContextPBImpl convertFromProtoFormat(
ContainerLaunchContextProto p) {
return new ContainerLaunchContextPBImpl(p);
}
private ResourceProto convertToProtoFormat(Resource t) {
return ((ResourcePBImpl)t).getProto();
private ContainerLaunchContextProto convertToProtoFormat(ContainerLaunchContext t) {
return ((ContainerLaunchContextPBImpl)t).getProto();
}
private URLPBImpl convertFromProtoFormat(URLProto p) {
return new URLPBImpl(p);
}
private URLProto convertToProtoFormat(URL t) {
return ((URLPBImpl)t).getProto();
}
private LocalResourcePBImpl convertFromProtoFormat(LocalResourceProto p) {
return new LocalResourcePBImpl(p);
}
private LocalResourceProto convertToProtoFormat(LocalResource t) {
return ((LocalResourcePBImpl)t).getProto();
}
}

View File

@ -39,8 +39,6 @@
import org.apache.hadoop.yarn.proto.YarnProtos.StringLocalResourceMapProto;
import org.apache.hadoop.yarn.proto.YarnProtos.StringStringMapProto;
public class ContainerLaunchContextPBImpl
extends ProtoBase<ContainerLaunchContextProto>
implements ContainerLaunchContext {
@ -54,10 +52,9 @@ public class ContainerLaunchContextPBImpl
private Map<String, LocalResource> localResources = null;
private ByteBuffer containerTokens = null;
private Map<String, ByteBuffer> serviceData = null;
private Map<String, String> env = null;
private Map<String, String> environment = null;
private List<String> commands = null;
public ContainerLaunchContextPBImpl() {
builder = ContainerLaunchContextProto.newBuilder();
}
@ -94,7 +91,7 @@ private void mergeLocalToBuilder() {
if (this.serviceData != null) {
addServiceDataToProto();
}
if (this.env != null) {
if (this.environment != null) {
addEnvToProto();
}
if (this.commands != null) {
@ -364,37 +361,37 @@ public boolean hasNext() {
}
@Override
public Map<String, String> getEnv() {
public Map<String, String> getEnvironment() {
initEnv();
return this.env;
return this.environment;
}
private void initEnv() {
if (this.env != null) {
if (this.environment != null) {
return;
}
ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder;
List<StringStringMapProto> list = p.getEnvList();
this.env = new HashMap<String, String>();
List<StringStringMapProto> list = p.getEnvironmentList();
this.environment = new HashMap<String, String>();
for (StringStringMapProto c : list) {
this.env.put(c.getKey(), c.getValue());
this.environment.put(c.getKey(), c.getValue());
}
}
@Override
public void setEnv(final Map<String, String> env) {
public void setEnvironment(final Map<String, String> env) {
if (env == null)
return;
initEnv();
this.env.clear();
this.env.putAll(env);
this.environment.clear();
this.environment.putAll(env);
}
private void addEnvToProto() {
maybeInitBuilder();
builder.clearEnv();
if (env == null)
builder.clearEnvironment();
if (environment == null)
return;
Iterable<StringStringMapProto> iterable =
new Iterable<StringStringMapProto>() {
@ -403,7 +400,7 @@ private void addEnvToProto() {
public Iterator<StringStringMapProto> iterator() {
return new Iterator<StringStringMapProto>() {
Iterator<String> keyIter = env.keySet().iterator();
Iterator<String> keyIter = environment.keySet().iterator();
@Override
public void remove() {
@ -414,7 +411,7 @@ public void remove() {
public StringStringMapProto next() {
String key = keyIter.next();
return StringStringMapProto.newBuilder().setKey(key).setValue(
(env.get(key))).build();
(environment.get(key))).build();
}
@Override
@ -424,7 +421,7 @@ public boolean hasNext() {
};
}
};
builder.addAllEnv(iterable);
builder.addAllEnvironment(iterable);
}
private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {

View File

@ -188,17 +188,11 @@ message AMResponseProto {
////////////////////////////////////////////////////////////////////////
message ApplicationSubmissionContextProto {
optional ApplicationIdProto application_id = 1;
optional string application_name = 2;
optional ResourceProto master_capability = 3;
repeated StringURLMapProto resources = 4;
repeated StringLocalResourceMapProto resources_todo = 5;
repeated string fs_tokens = 6;
optional bytes fs_tokens_todo = 7;
repeated StringStringMapProto environment = 8;
repeated string command = 9;
optional string queue = 10;
optional PriorityProto priority = 11;
optional string user = 12;
optional string application_name = 2 [default = "N/A"];
optional string user = 3;
optional string queue = 4 [default = "default"];
optional PriorityProto priority = 5;
optional ContainerLaunchContextProto am_container_spec = 6;
}
message YarnClusterMetricsProto {
@ -242,7 +236,7 @@ message ContainerLaunchContextProto {
repeated StringLocalResourceMapProto localResources = 4;
optional bytes container_tokens = 5;
repeated StringBytesMapProto service_data = 6;
repeated StringStringMapProto env = 7;
repeated StringStringMapProto environment = 7;
repeated string command = 8;
}

View File

@ -219,6 +219,12 @@ public class YarnConfiguration extends Configuration {
RM_PREFIX + "max-completed-applications";
public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 10000;
/** Default application name */
public static final String DEFAULT_APPLICATION_NAME = "N/A";
/** Default queue name */
public static final String DEFAULT_QUEUE_NAME = "default";
////////////////////////////////
// Node Manager Configs
////////////////////////////////

View File

@ -67,28 +67,59 @@ public synchronized void start() {
Service service = serviceList.get(i);
service.start();
}
} catch(Throwable e) {
super.start();
} catch (Throwable e) {
LOG.error("Error starting services " + getName(), e);
for (int j = i-1; j >= 0; j--) {
Service service = serviceList.get(j);
try {
service.stop();
} catch(Throwable t) {
LOG.info("Error stopping " + service.getName(), t);
}
}
// Note that the state of the failed service is still INITED and not
// STARTED. Even though the last service is not started completely, still
// call stop() on all services including failed service to make sure cleanup
// happens.
stop(i);
throw new YarnException("Failed to Start " + getName(), e);
}
super.start();
}
public synchronized void stop() {
//stop in reserve order of start
for (int i = serviceList.size() - 1; i >= 0; i--) {
Service service = serviceList.get(i);
service.stop();
if (serviceList.size() > 0) {
stop(serviceList.size() - 1);
}
super.stop();
}
private synchronized void stop(int numOfServicesStarted) {
// stop in reserve order of start
for (int i = numOfServicesStarted; i >= 0; i--) {
Service service = serviceList.get(i);
try {
service.stop();
} catch (Throwable t) {
LOG.info("Error stopping " + service.getName(), t);
}
}
}
/**
* JVM Shutdown hook for CompositeService which will stop the give
* CompositeService gracefully in case of JVM shutdown.
*/
public static class CompositeServiceShutdownHook extends Thread {
private CompositeService compositeService;
public CompositeServiceShutdownHook(CompositeService compositeService) {
this.compositeService = compositeService;
}
@Override
public void run() {
try {
// Stop the Composite Service
compositeService.stop();
} catch (Throwable t) {
LOG.info("Error stopping " + compositeService.getName(), t);
}
}
}
}

View File

@ -30,10 +30,18 @@ public class Times {
};
public static long elapsed(long started, long finished) {
return Times.elapsed(started, finished, true);
}
public static long elapsed(long started, long finished, boolean isRunning) {
if (finished > 0) {
return finished - started;
}
if (isRunning) {
return started > 0 ? System.currentTimeMillis() - started : 0;
} else {
return -1;
}
}
public static String format(long ts) {

View File

@ -0,0 +1,247 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.util;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service.STATE;
import org.junit.Test;
public class TestCompositeService {
private static final int NUM_OF_SERVICES = 5;
private static final int FAILED_SERVICE_SEQ_NUMBER = 2;
@Test
public void testCallSequence() {
ServiceManager serviceManager = new ServiceManager("ServiceManager");
// Add services
for (int i = 0; i < NUM_OF_SERVICES; i++) {
CompositeServiceImpl service = new CompositeServiceImpl(i);
serviceManager.addTestService(service);
}
CompositeServiceImpl[] services = serviceManager.getServices().toArray(
new CompositeServiceImpl[0]);
assertEquals("Number of registered services ", NUM_OF_SERVICES,
services.length);
Configuration conf = new Configuration();
// Initialise the composite service
serviceManager.init(conf);
// Verify the init() call sequence numbers for every service
for (int i = 0; i < NUM_OF_SERVICES; i++) {
assertEquals("For " + services[i]
+ " service, init() call sequence number should have been ", i,
services[i].getCallSequenceNumber());
}
// Reset the call sequence numbers
for (int i = 0; i < NUM_OF_SERVICES; i++) {
services[i].reset();
}
serviceManager.start();
// Verify the start() call sequence numbers for every service
for (int i = 0; i < NUM_OF_SERVICES; i++) {
assertEquals("For " + services[i]
+ " service, start() call sequence number should have been ", i,
services[i].getCallSequenceNumber());
}
// Reset the call sequence numbers
for (int i = 0; i < NUM_OF_SERVICES; i++) {
services[i].reset();
}
serviceManager.stop();
// Verify the stop() call sequence numbers for every service
for (int i = 0; i < NUM_OF_SERVICES; i++) {
assertEquals("For " + services[i]
+ " service, stop() call sequence number should have been ",
((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber());
}
}
@Test
public void testServiceStartup() {
ServiceManager serviceManager = new ServiceManager("ServiceManager");
// Add services
for (int i = 0; i < NUM_OF_SERVICES; i++) {
CompositeServiceImpl service = new CompositeServiceImpl(i);
if (i == FAILED_SERVICE_SEQ_NUMBER) {
service.setThrowExceptionOnStart(true);
}
serviceManager.addTestService(service);
}
CompositeServiceImpl[] services = serviceManager.getServices().toArray(
new CompositeServiceImpl[0]);
Configuration conf = new Configuration();
// Initialise the composite service
serviceManager.init(conf);
// Start the composite service
try {
serviceManager.start();
fail("Exception should have been thrown due to startup failure of last service");
} catch (YarnException e) {
for (int i = 0; i < NUM_OF_SERVICES - 1; i++) {
if (i >= FAILED_SERVICE_SEQ_NUMBER) {
// Failed service state should be INITED
assertEquals("Service state should have been ", STATE.INITED,
services[NUM_OF_SERVICES - 1].getServiceState());
} else {
assertEquals("Service state should have been ", STATE.STOPPED,
services[i].getServiceState());
}
}
}
}
@Test
public void testServiceStop() {
ServiceManager serviceManager = new ServiceManager("ServiceManager");
// Add services
for (int i = 0; i < NUM_OF_SERVICES; i++) {
CompositeServiceImpl service = new CompositeServiceImpl(i);
if (i == FAILED_SERVICE_SEQ_NUMBER) {
service.setThrowExceptionOnStop(true);
}
serviceManager.addTestService(service);
}
CompositeServiceImpl[] services = serviceManager.getServices().toArray(
new CompositeServiceImpl[0]);
Configuration conf = new Configuration();
// Initialise the composite service
serviceManager.init(conf);
serviceManager.start();
// Start the composite service
try {
serviceManager.stop();
} catch (YarnException e) {
for (int i = 0; i < NUM_OF_SERVICES - 1; i++) {
assertEquals("Service state should have been ", STATE.STOPPED,
services[NUM_OF_SERVICES].getServiceState());
}
}
}
public static class CompositeServiceImpl extends CompositeService {
private static int counter = -1;
private int callSequenceNumber = -1;
private boolean throwExceptionOnStart;
private boolean throwExceptionOnStop;
public CompositeServiceImpl(int sequenceNumber) {
super(Integer.toString(sequenceNumber));
}
@Override
public synchronized void init(Configuration conf) {
counter++;
callSequenceNumber = counter;
super.init(conf);
}
@Override
public synchronized void start() {
if (throwExceptionOnStart) {
throw new YarnException("Fake service start exception");
}
counter++;
callSequenceNumber = counter;
super.start();
}
@Override
public synchronized void stop() {
counter++;
callSequenceNumber = counter;
if (throwExceptionOnStop) {
throw new YarnException("Fake service stop exception");
}
super.stop();
}
public static int getCounter() {
return counter;
}
public int getCallSequenceNumber() {
return callSequenceNumber;
}
public void reset() {
callSequenceNumber = -1;
counter = -1;
}
public void setThrowExceptionOnStart(boolean throwExceptionOnStart) {
this.throwExceptionOnStart = throwExceptionOnStart;
}
public void setThrowExceptionOnStop(boolean throwExceptionOnStop) {
this.throwExceptionOnStop = throwExceptionOnStop;
}
@Override
public String toString() {
return "Service " + getName();
}
}
public static class ServiceManager extends CompositeService {
public void addTestService(CompositeService service) {
addService(service);
}
public ServiceManager(String name) {
super(name);
}
}
}

View File

@ -24,6 +24,8 @@
import org.apache.hadoop.yarn.webapp.WebAppException;
import java.lang.reflect.Method;
import java.util.Map;
import com.google.inject.Module;
import com.google.inject.Scopes;
import com.google.inject.servlet.RequestScoped;
@ -126,22 +128,31 @@ public static <T> Injector testController(Class<? extends Controller> ctrlr,
}
}
@SuppressWarnings("unchecked")
public static <T> Injector testController(Class<? extends Controller> ctrlr,
String methodName) {
return testController(ctrlr, methodName, null, null);
}
public static <T> Injector testPage(Class<? extends View> page, Class<T> api,
T impl, Module... modules) {
T impl, Map<String,String> params, Module... modules) {
Injector injector = createMockInjector(api, impl, modules);
injector.getInstance(page).render();
View view = injector.getInstance(page);
if(params != null) {
for(Map.Entry<String, String> entry: params.entrySet()) {
view.set(entry.getKey(), entry.getValue());
}
}
view.render();
flushOutput(injector);
return injector;
}
public static <T> Injector testPage(Class<? extends View> page, Class<T> api,
T impl, Module... modules) {
return testPage(page, api, impl, null, modules);
}
// convenience
@SuppressWarnings("unchecked")
public static <T> Injector testPage(Class<? extends View> page) {
return testPage(page, null, null);
}
@ -155,7 +166,6 @@ public static <T> Injector testBlock(Class<? extends SubView> block,
}
// convenience
@SuppressWarnings("unchecked")
public static <T> Injector testBlock(Class<? extends SubView> block) {
return testBlock(block, null, null);
}

View File

@ -133,13 +133,6 @@ public void init(Configuration conf) {
dispatcher.register(ContainerManagerEventType.class, containerManager);
addService(dispatcher);
Runtime.getRuntime().addShutdownHook(new Thread() {
@Override
public void run() {
NodeManager.this.stop();
}
});
DefaultMetricsSystem.initialize("NodeManager");
// StatusUpdater should be added last so that it get started last
@ -200,10 +193,17 @@ public NodeHealthStatus getNodeHealthStatus() {
public static void main(String[] args) {
StringUtils.startupShutdownMessage(NodeManager.class, args, LOG);
try {
NodeManager nodeManager = new NodeManager();
Runtime.getRuntime().addShutdownHook(
new CompositeServiceShutdownHook(nodeManager));
YarnConfiguration conf = new YarnConfiguration();
nodeManager.init(conf);
nodeManager.start();
} catch (Throwable t) {
LOG.fatal("Error starting NodeManager", t);
System.exit(-1);
}
}
}

View File

@ -89,7 +89,7 @@ public Integer call() {
final Map<Path,String> localResources = container.getLocalizedResources();
String containerIdStr = ConverterUtils.toString(container.getContainerID());
final String user = launchContext.getUser();
final Map<String,String> env = launchContext.getEnv();
final Map<String,String> env = launchContext.getEnvironment();
final List<String> command = launchContext.getCommands();
int ret = -1;
@ -109,7 +109,7 @@ public Integer call() {
}
launchContext.setCommands(newCmds);
Map<String, String> envs = launchContext.getEnv();
Map<String, String> envs = launchContext.getEnvironment();
Map<String, String> newEnvs = new HashMap<String, String>(envs.size());
for (Entry<String, String> entry : envs.entrySet()) {
newEnvs.put(
@ -118,7 +118,7 @@ public Integer call() {
ApplicationConstants.LOG_DIR_EXPANSION_VAR,
containerLogDir.toUri().getPath()));
}
launchContext.setEnv(newEnvs);
launchContext.setEnvironment(newEnvs);
// /////////////////////////// End of variable expansion
FileContext lfs = FileContext.getLocalFSFileContext();

View File

@ -107,9 +107,10 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerSecurityInfo;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenSecretManager;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.util.ConverterUtils;
public class ResourceLocalizationService extends AbstractService
public class ResourceLocalizationService extends CompositeService
implements EventHandler<LocalizationEvent>, LocalizationProtocol {
private static final Log LOG = LogFactory.getLog(ResourceLocalizationService.class);
@ -201,9 +202,8 @@ public void init(Configuration conf) {
localizationServerAddress = NetUtils.createSocketAddr(
conf.get(YarnConfiguration.NM_LOCALIZER_ADDRESS, YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS));
localizerTracker = createLocalizerTracker(conf);
addService(localizerTracker);
dispatcher.register(LocalizerEventType.class, localizerTracker);
cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher),
cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS);
super.init(conf);
}
@ -214,6 +214,8 @@ public LocalizerHeartbeatResponse heartbeat(LocalizerStatus status) {
@Override
public void start() {
cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher),
cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS);
server = createServer();
LOG.info("Localizer started on port " + server.getPort());
server.start();
@ -247,9 +249,7 @@ public void stop() {
if (server != null) {
server.close();
}
if (localizerTracker != null) {
localizerTracker.stop();
}
cacheCleanup.shutdown();
super.stop();
}
@ -403,7 +403,7 @@ LocalResourcesTracker getLocalResourcesTracker(
/**
* Sub-component handling the spawning of {@link ContainerLocalizer}s
*/
class LocalizerTracker implements EventHandler<LocalizerEvent> {
class LocalizerTracker extends AbstractService implements EventHandler<LocalizerEvent> {
private final PublicLocalizer publicLocalizer;
private final Map<String,LocalizerRunner> privLocalizers;
@ -414,9 +414,15 @@ class LocalizerTracker implements EventHandler<LocalizerEvent> {
LocalizerTracker(Configuration conf,
Map<String,LocalizerRunner> privLocalizers) {
super(LocalizerTracker.class.getName());
this.publicLocalizer = new PublicLocalizer(conf);
this.privLocalizers = privLocalizers;
}
@Override
public synchronized void start() {
publicLocalizer.start();
super.start();
}
public LocalizerHeartbeatResponse processHeartbeat(LocalizerStatus status) {
@ -436,11 +442,13 @@ public LocalizerHeartbeatResponse processHeartbeat(LocalizerStatus status) {
}
}
@Override
public void stop() {
for (LocalizerRunner localizer : privLocalizers.values()) {
localizer.interrupt();
}
publicLocalizer.interrupt();
super.stop();
}
@Override

View File

@ -71,7 +71,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.service.AbstractService;
@ -90,7 +89,6 @@ public class ClientRMService extends AbstractService implements
final private AtomicInteger applicationCounter = new AtomicInteger(0);
final private YarnScheduler scheduler;
final private RMContext rmContext;
private final AMLivelinessMonitor amLivelinessMonitor;
private final RMAppManager rmAppManager;
private String clientServiceBindAddress;
@ -106,7 +104,6 @@ public ClientRMService(RMContext rmContext, YarnScheduler scheduler,
super(ClientRMService.class.getName());
this.scheduler = scheduler;
this.rmContext = rmContext;
this.amLivelinessMonitor = rmContext.getAMLivelinessMonitor();
this.rmAppManager = rmAppManager;
}
@ -195,15 +192,18 @@ public SubmitApplicationResponse submitApplication(
SubmitApplicationRequest request) throws YarnRemoteException {
ApplicationSubmissionContext submissionContext = request
.getApplicationSubmissionContext();
ApplicationId applicationId = null;
String user = null;
ApplicationId applicationId = submissionContext.getApplicationId();
String user = submissionContext.getUser();
try {
user = UserGroupInformation.getCurrentUser().getShortUserName();
applicationId = submissionContext.getApplicationId();
if (rmContext.getRMApps().get(applicationId) != null) {
throw new IOException("Application with id " + applicationId
+ " is already present! Cannot add a duplicate!");
}
// Safety
submissionContext.setUser(user);
// This needs to be synchronous as the client can query
// immediately following the submission to get the application status.
// So call handle directly and do not send an event.
@ -226,6 +226,7 @@ public SubmitApplicationResponse submitApplication(
return response;
}
@SuppressWarnings("unchecked")
@Override
public FinishApplicationResponse finishApplication(
FinishApplicationRequest request) throws YarnRemoteException {

View File

@ -210,7 +210,9 @@ protected synchronized void checkAppNumCompletedLimit() {
}
}
protected synchronized void submitApplication(ApplicationSubmissionContext submissionContext) {
@SuppressWarnings("unchecked")
protected synchronized void submitApplication(
ApplicationSubmissionContext submissionContext) {
ApplicationId applicationId = submissionContext.getApplicationId();
RMApp application = null;
try {
@ -224,27 +226,37 @@ protected synchronized void submitApplication(ApplicationSubmissionContext submi
clientTokenStr = clientToken.encodeToUrlString();
LOG.debug("Sending client token as " + clientTokenStr);
}
submissionContext.setQueue(submissionContext.getQueue() == null
? "default" : submissionContext.getQueue());
submissionContext.setApplicationName(submissionContext
.getApplicationName() == null ? "N/A" : submissionContext
.getApplicationName());
// Sanity checks
if (submissionContext.getQueue() == null) {
submissionContext.setQueue(YarnConfiguration.DEFAULT_QUEUE_NAME);
}
if (submissionContext.getApplicationName() == null) {
submissionContext.setApplicationName(
YarnConfiguration.DEFAULT_APPLICATION_NAME);
}
// Store application for recovery
ApplicationStore appStore = rmContext.getApplicationsStore()
.createApplicationStore(submissionContext.getApplicationId(),
submissionContext);
// Create RMApp
application = new RMAppImpl(applicationId, rmContext,
this.conf, submissionContext.getApplicationName(), user,
submissionContext.getQueue(), submissionContext, clientTokenStr,
appStore, rmContext.getAMLivelinessMonitor(), this.scheduler,
appStore, this.scheduler,
this.masterService);
if (rmContext.getRMApps().putIfAbsent(applicationId, application) != null) {
if (rmContext.getRMApps().putIfAbsent(applicationId, application) !=
null) {
LOG.info("Application with id " + applicationId +
" is already present! Cannot add a duplicate!");
// don't send event through dispatcher as it will be handled by app already
// present with this id.
// don't send event through dispatcher as it will be handled by app
// already present with this id.
application.handle(new RMAppRejectedEvent(applicationId,
"Application with this id is already present! Cannot add a duplicate!"));
"Application with this id is already present! " +
"Cannot add a duplicate!"));
} else {
this.rmContext.getDispatcher().getEventHandler().handle(
new RMAppEvent(applicationId, RMAppEventType.START));

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
public class RMAppManagerSubmitEvent extends RMAppManagerEvent {

View File

@ -22,7 +22,6 @@
import java.io.IOException;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -45,8 +44,8 @@
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@ -101,7 +100,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
private SchedulerEventDispatcher schedulerDispatcher;
protected RMAppManager rmAppManager;
private final AtomicBoolean shutdown = new AtomicBoolean(false);
private WebApp webApp;
private RMContext rmContext;
private final Store store;
@ -490,20 +488,19 @@ public void recover(RMState state) throws Exception {
public static void main(String argv[]) {
StringUtils.startupShutdownMessage(ResourceManager.class, argv, LOG);
ResourceManager resourceManager = null;
try {
Configuration conf = new YarnConfiguration();
Store store = StoreFactory.getStore(conf);
resourceManager = new ResourceManager(store);
ResourceManager resourceManager = new ResourceManager(store);
Runtime.getRuntime().addShutdownHook(
new CompositeServiceShutdownHook(resourceManager));
resourceManager.init(conf);
//resourceManager.recover(store.restore());
//store.doneWithRecovery();
resourceManager.start();
} catch (Throwable e) {
LOG.error("Error starting RM", e);
if (resourceManager != null) {
resourceManager.stop();
}
} catch (Throwable t) {
LOG.fatal("Error starting ResourceManager", t);
System.exit(-1);
}
}
}

View File

@ -23,7 +23,6 @@
import java.nio.ByteBuffer;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -120,7 +119,8 @@ private void launch() throws IOException {
+ " for AM " + application.getAppAttemptId());
ContainerLaunchContext launchContext =
createAMContainerLaunchContext(applicationContext, masterContainerID);
StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
StartContainerRequest request =
recordFactory.newRecordInstance(StartContainerRequest.class);
request.setContainerLaunchContext(launchContext);
containerMgrProxy.startContainer(request);
LOG.info("Done launching container " + application.getMasterContainer()
@ -130,7 +130,8 @@ private void launch() throws IOException {
private void cleanup() throws IOException {
connect();
ContainerId containerId = application.getMasterContainer().getId();
StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
StopContainerRequest stopRequest =
recordFactory.newRecordInstance(StopContainerRequest.class);
stopRequest.setContainerId(containerId);
containerMgrProxy.stopContainer(stopRequest);
}
@ -145,7 +146,7 @@ private ContainerManager getContainerMgrProxy(
final YarnRPC rpc = YarnRPC.create(conf); // TODO: Don't create again and again.
UserGroupInformation currentUser =
UserGroupInformation.createRemoteUser("TODO"); // TODO
UserGroupInformation.createRemoteUser("yarn"); // TODO
if (UserGroupInformation.isSecurityEnabled()) {
ContainerToken containerToken = container.getContainerToken();
Token<ContainerTokenIdentifier> token =
@ -170,8 +171,8 @@ private ContainerLaunchContext createAMContainerLaunchContext(
ContainerId containerID) throws IOException {
// Construct the actual Container
ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class);
container.setCommands(applicationMasterContext.getCommandList());
ContainerLaunchContext container =
applicationMasterContext.getAMContainerSpec();
StringBuilder mergedCommand = new StringBuilder();
String failCount = Integer.toString(application.getAppAttemptId()
.getAttemptId());
@ -189,34 +190,28 @@ private ContainerLaunchContext createAMContainerLaunchContext(
LOG.info("Command to launch container " +
containerID + " : " + mergedCommand);
Map<String, String> environment =
applicationMasterContext.getAllEnvironment();
environment.putAll(setupTokensInEnv(applicationMasterContext));
container.setEnv(environment);
// Construct the actual Container
// Finalize the container
container.setContainerId(containerID);
container.setUser(applicationMasterContext.getUser());
container.setResource(applicationMasterContext.getMasterCapability());
container.setLocalResources(applicationMasterContext.getAllResourcesTodo());
container.setContainerTokens(applicationMasterContext.getFsTokensTodo());
setupTokensAndEnv(container);
return container;
}
private Map<String, String> setupTokensInEnv(
ApplicationSubmissionContext asc)
private void setupTokensAndEnv(
ContainerLaunchContext container)
throws IOException {
Map<String, String> env =
new HashMap<String, String>();
Map<String, String> environment = container.getEnvironment();
if (UserGroupInformation.isSecurityEnabled()) {
// TODO: Security enabled/disabled info should come from RM.
Credentials credentials = new Credentials();
DataInputByteBuffer dibb = new DataInputByteBuffer();
if (asc.getFsTokensTodo() != null) {
if (container.getContainerTokens() != null) {
// TODO: Don't do this kind of checks everywhere.
dibb.reset(asc.getFsTokensTodo());
dibb.reset(container.getContainerTokens());
credentials.readTokenStorageStream(dibb);
}
@ -236,14 +231,16 @@ private Map<String, String> setupTokensInEnv(
token.setService(new Text(resolvedAddr));
String appMasterTokenEncoded = token.encodeToUrlString();
LOG.debug("Putting appMaster token in env : " + appMasterTokenEncoded);
env.put(ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME,
environment.put(
ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME,
appMasterTokenEncoded);
// Add the RM token
credentials.addToken(new Text(resolvedAddr), token);
DataOutputBuffer dob = new DataOutputBuffer();
credentials.writeTokenStorageToStream(dob);
asc.setFsTokensTodo(ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
container.setContainerTokens(
ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
ApplicationTokenIdentifier identifier = new ApplicationTokenIdentifier(
application.getAppAttemptId().getApplicationId());
@ -252,9 +249,10 @@ private Map<String, String> setupTokensInEnv(
String encoded =
Base64.encodeBase64URLSafeString(clientSecretKey.getEncoded());
LOG.debug("The encoded client secret-key to be put in env : " + encoded);
env.put(ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME, encoded);
environment.put(
ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME,
encoded);
}
return env;
}
@SuppressWarnings("unchecked")

View File

@ -86,7 +86,6 @@ public class RMAppImpl implements RMApp {
// Mutable fields
private long startTime;
private long finishTime;
private AMLivelinessMonitor amLivelinessMonitor;
private RMAppAttempt currentAttempt;
private static final FinalTransition FINAL_TRANSITION = new FinalTransition();
@ -163,7 +162,7 @@ RMAppEventType.KILL, new AppKilledTransition())
public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
Configuration config, String name, String user, String queue,
ApplicationSubmissionContext submissionContext, String clientTokenStr,
ApplicationStore appStore, AMLivelinessMonitor amLivelinessMonitor,
ApplicationStore appStore,
YarnScheduler scheduler, ApplicationMasterService masterService) {
this.applicationId = applicationId;
@ -176,7 +175,6 @@ public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
this.submissionContext = submissionContext;
this.clientTokenStr = clientTokenStr;
this.appStore = appStore;
this.amLivelinessMonitor = amLivelinessMonitor;
this.scheduler = scheduler;
this.masterService = masterService;
this.startTime = System.currentTimeMillis();
@ -380,6 +378,7 @@ public void handle(RMAppEvent event) {
}
}
@SuppressWarnings("unchecked")
private void createNewAttempt() {
ApplicationAttemptId appAttemptId = Records
.newRecord(ApplicationAttemptId.class);
@ -434,6 +433,7 @@ private Set<NodeId> getNodesOnWhichAttemptRan(RMAppImpl app) {
return nodes;
}
@SuppressWarnings("unchecked")
public void transition(RMAppImpl app, RMAppEvent event) {
Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
for (NodeId nodeId : nodes) {

View File

@ -84,6 +84,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
RMAppAttemptEvent> stateMachine;
private final RMContext rmContext;
@SuppressWarnings("rawtypes")
private final EventHandler eventHandler;
private final YarnScheduler scheduler;
private final ApplicationMasterService masterService;
@ -459,7 +460,7 @@ public void transition(RMAppAttemptImpl appAttempt,
// Request a container for the AM.
ResourceRequest request = BuilderUtils.newResourceRequest(
AM_CONTAINER_PRIORITY, "*", appAttempt.submissionContext
.getMasterCapability(), 1);
.getAMContainerSpec().getResource(), 1);
LOG.debug("About to request resources for AM of "
+ appAttempt.applicationAttemptId + " required " + request);

View File

@ -1023,21 +1023,17 @@ boolean canAssign(SchedulerApp application, Priority priority,
// Check if we need containers on this rack
ResourceRequest rackLocalRequest =
application.getResourceRequest(priority, node.getRackName());
if (type == NodeType.RACK_LOCAL) {
if (rackLocalRequest == null) {
if (rackLocalRequest == null || rackLocalRequest.getNumContainers() <= 0) {
return false;
} else {
return rackLocalRequest.getNumContainers() > 0;
}
// If we are here, we do need containers on this rack for RACK_LOCAL req
if (type == NodeType.RACK_LOCAL) {
return true;
}
// Check if we need containers on this host
if (type == NodeType.NODE_LOCAL) {
// First: Do we need containers on this rack?
if (rackLocalRequest != null && rackLocalRequest.getNumContainers() == 0) {
return false;
}
// Now check if we need containers on this host...
ResourceRequest nodeLocalRequest =
application.getResourceRequest(priority, node.getHostName());

View File

@ -289,6 +289,7 @@ private SchedulerNode getNode(NodeId nodeId) {
return nodes.get(nodeId);
}
@SuppressWarnings("unchecked")
private synchronized void addApplication(ApplicationAttemptId appAttemptId,
String queueName, String user) {
// TODO: Fix store
@ -440,6 +441,14 @@ private int assignNodeLocalContainers(SchedulerNode node,
ResourceRequest request =
application.getResourceRequest(priority, node.getRMNode().getNodeAddress());
if (request != null) {
// Don't allocate on this node if we don't need containers on this rack
ResourceRequest rackRequest =
application.getResourceRequest(priority,
node.getRMNode().getRackName());
if (rackRequest == null || rackRequest.getNumContainers() <= 0) {
return 0;
}
int assignableContainers =
Math.min(
getMaxAllocatableContainers(application, priority, node,
@ -458,6 +467,13 @@ private int assignRackLocalContainers(SchedulerNode node,
ResourceRequest request =
application.getResourceRequest(priority, node.getRMNode().getRackName());
if (request != null) {
// Don't allocate on this rack if the application doens't need containers
ResourceRequest offSwitchRequest =
application.getResourceRequest(priority, SchedulerNode.ANY);
if (offSwitchRequest.getNumContainers() <= 0) {
return 0;
}
int assignableContainers =
Math.min(
getMaxAllocatableContainers(application, priority, node,

View File

@ -23,7 +23,6 @@
import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.util.Apps;
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;
@ -59,7 +58,8 @@ class AppsBlock extends HtmlBlock {
String appId = app.getApplicationId().toString();
String trackingUrl = app.getTrackingUrl();
String ui = trackingUrl == null || trackingUrl.isEmpty() ? "UNASSIGNED" :
(app.getFinishTime() == 0 ? "ApplicationMaster" : "JobHistory");
(app.getFinishTime() == 0 ?
"ApplicationMaster URL" : "JobHistory URL");
String percent = String.format("%.1f", app.getProgress() * 100);
tbody.
tr().

View File

@ -29,6 +29,7 @@
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
@ -81,13 +82,17 @@ public RMApp submitApp(int masterMemory) throws Exception {
ApplicationId appId = resp.getApplicationId();
SubmitApplicationRequest req = Records.newRecord(SubmitApplicationRequest.class);
ApplicationSubmissionContext sub = Records.newRecord(ApplicationSubmissionContext.class);
ApplicationSubmissionContext sub =
Records.newRecord(ApplicationSubmissionContext.class);
sub.setApplicationId(appId);
sub.setApplicationName("");
sub.setUser("");
ContainerLaunchContext clc =
Records.newRecord(ContainerLaunchContext.class);
Resource capability = Records.newRecord(Resource.class);
capability.setMemory(masterMemory);
sub.setMasterCapability(capability);
clc.setResource(capability);
sub.setAMContainerSpec(clc);
req.setApplicationSubmissionContext(sub);
client.submitApplication(req);

View File

@ -18,19 +18,12 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import static org.mockito.Mockito.*;
import java.util.ArrayList;
import java.util.List;
import java.util.LinkedList;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
@ -44,7 +37,6 @@
import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManager;
@ -63,8 +55,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.service.Service;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import com.google.common.collect.Maps;
import com.google.common.collect.Lists;
@ -75,7 +65,6 @@
*/
public class TestAppManager{
private static final Log LOG = LogFactory.getLog(TestAppManager.class);
private static RMAppEventType appEventType = RMAppEventType.KILL;
public synchronized RMAppEventType getAppEventType() {
@ -117,10 +106,8 @@ public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
public class TestAppManagerDispatcher implements
EventHandler<RMAppManagerEvent> {
private final RMContext rmContext;
public TestAppManagerDispatcher(RMContext rmContext) {
this.rmContext = rmContext;
public TestAppManagerDispatcher() {
}
@Override
@ -132,15 +119,11 @@ public void handle(RMAppManagerEvent event) {
public class TestDispatcher implements
EventHandler<RMAppEvent> {
private final RMContext rmContext;
public TestDispatcher(RMContext rmContext) {
this.rmContext = rmContext;
public TestDispatcher() {
}
@Override
public void handle(RMAppEvent event) {
ApplicationId appID = event.getApplicationId();
//RMApp rmApp = this.rmContext.getRMApps().get(appID);
setAppEventType(event.getType());
System.out.println("in handle routine " + getAppEventType().toString());
@ -178,7 +161,8 @@ public int getCompletedAppsListSize() {
public void setCompletedAppsMax(int max) {
super.setCompletedAppsMax(max);
}
public void submitApplication(ApplicationSubmissionContext submissionContext) {
public void submitApplication(
ApplicationSubmissionContext submissionContext) {
super.submitApplication(submissionContext);
}
}
@ -336,8 +320,9 @@ public void testRMAppRetireZeroSetting() throws Exception {
}
protected void setupDispatcher(RMContext rmContext, Configuration conf) {
TestDispatcher testDispatcher = new TestDispatcher(rmContext);
TestAppManagerDispatcher testAppManagerDispatcher = new TestAppManagerDispatcher(rmContext);
TestDispatcher testDispatcher = new TestDispatcher();
TestAppManagerDispatcher testAppManagerDispatcher =
new TestAppManagerDispatcher();
rmContext.getDispatcher().register(RMAppEventType.class, testDispatcher);
rmContext.getDispatcher().register(RMAppManagerEventType.class, testAppManagerDispatcher);
((Service)rmContext.getDispatcher()).init(conf);
@ -359,7 +344,8 @@ public void testRMAppSubmit() throws Exception {
ApplicationId appID = MockApps.newAppID(1);
RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
ApplicationSubmissionContext context =
recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
context.setApplicationId(appID);
setupDispatcher(rmContext, conf);
@ -367,8 +353,12 @@ public void testRMAppSubmit() throws Exception {
RMApp app = rmContext.getRMApps().get(appID);
Assert.assertNotNull("app is null", app);
Assert.assertEquals("app id doesn't match", appID, app.getApplicationId());
Assert.assertEquals("app name doesn't match", "N/A", app.getName());
Assert.assertEquals("app queue doesn't match", "default", app.getQueue());
Assert.assertEquals("app name doesn't match",
YarnConfiguration.DEFAULT_APPLICATION_NAME,
app.getName());
Assert.assertEquals("app queue doesn't match",
YarnConfiguration.DEFAULT_QUEUE_NAME,
app.getQueue());
Assert.assertEquals("app state doesn't match", RMAppState.NEW, app.getState());
Assert.assertNotNull("app store is null", app.getApplicationStore());

View File

@ -128,7 +128,7 @@ protected RMApp createNewTestApp() {
RMApp application = new RMAppImpl(applicationId, rmContext,
conf, name, user,
queue, submissionContext, clientTokenStr,
appStore, rmContext.getAMLivelinessMonitor(), scheduler,
appStore, scheduler,
masterService);
testAppStartState(applicationId, user, name, queue, application);

View File

@ -625,7 +625,6 @@ public void testReservationExchange() throws Exception {
}
@Test
public void testLocalityScheduling() throws Exception {
@ -876,6 +875,107 @@ public void testApplicationPriorityScheduling() throws Exception {
}
@Test
public void testSchedulingConstraints() throws Exception {
// Manipulate queue 'a'
LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
// User
String user_0 = "user_0";
// Submit applications
final ApplicationAttemptId appAttemptId_0 =
TestUtils.getMockApplicationAttemptId(0, 0);
SchedulerApp app_0 =
spy(new SchedulerApp(appAttemptId_0, user_0, a, rmContext, null));
a.submitApplication(app_0, user_0, A);
// Setup some nodes and racks
String host_0_0 = "host_0_0";
String rack_0 = "rack_0";
SchedulerNode node_0_0 = TestUtils.getMockNode(host_0_0, rack_0, 0, 8*GB);
String host_0_1 = "host_0_1";
SchedulerNode node_0_1 = TestUtils.getMockNode(host_0_1, rack_0, 0, 8*GB);
String host_1_0 = "host_1_0";
String rack_1 = "rack_1";
SchedulerNode node_1_0 = TestUtils.getMockNode(host_1_0, rack_1, 0, 8*GB);
final int numNodes = 3;
Resource clusterResource = Resources.createResource(numNodes * (8*GB));
when(csContext.getNumClusterNodes()).thenReturn(numNodes);
// Setup resource-requests and submit
Priority priority = TestUtils.createMockPriority(1);
List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
app_0_requests_0.add(
TestUtils.createResourceRequest(host_0_0, 1*GB, 1,
priority, recordFactory));
app_0_requests_0.add(
TestUtils.createResourceRequest(host_0_1, 1*GB, 1,
priority, recordFactory));
app_0_requests_0.add(
TestUtils.createResourceRequest(rack_0, 1*GB, 1,
priority, recordFactory));
app_0_requests_0.add(
TestUtils.createResourceRequest(host_1_0, 1*GB, 1,
priority, recordFactory));
app_0_requests_0.add(
TestUtils.createResourceRequest(rack_1, 1*GB, 1,
priority, recordFactory));
app_0.updateResourceRequests(app_0_requests_0);
// Start testing...
// Add one request
app_0_requests_0.clear();
app_0_requests_0.add(
TestUtils.createResourceRequest(RMNodeImpl.ANY, 1*GB, 1, // only one
priority, recordFactory));
app_0.updateResourceRequests(app_0_requests_0);
// NODE_LOCAL - node_0_1
a.assignContainers(clusterResource, node_0_0);
verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
assertEquals(0, app_0.getTotalRequiredResources(priority));
// No allocation on node_1_0 even though it's node/rack local since
// required(ANY) == 0
a.assignContainers(clusterResource, node_1_0);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // Still zero
// since #req=0
assertEquals(0, app_0.getTotalRequiredResources(priority));
// Add one request
app_0_requests_0.clear();
app_0_requests_0.add(
TestUtils.createResourceRequest(RMNodeImpl.ANY, 1*GB, 1, // only one
priority, recordFactory));
app_0.updateResourceRequests(app_0_requests_0);
// No allocation on node_0_1 even though it's node/rack local since
// required(rack_1) == 0
a.assignContainers(clusterResource, node_0_1);
verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(1, app_0.getSchedulingOpportunities(priority));
assertEquals(1, app_0.getTotalRequiredResources(priority));
// NODE_LOCAL - node_1
a.assignContainers(clusterResource, node_1_0);
verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0),
any(Priority.class), any(ResourceRequest.class), any(Container.class));
assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
assertEquals(0, app_0.getTotalRequiredResources(priority));
}
@After
public void tearDown() throws Exception {
}

View File

@ -27,6 +27,8 @@
import java.net.InetSocketAddress;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import junit.framework.Assert;
@ -54,10 +56,10 @@
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerToken;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
@ -77,6 +79,7 @@
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.security.SchedulerSecurityInfo;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@ -137,15 +140,11 @@ public void test() throws IOException, InterruptedException {
ApplicationSubmissionContext appSubmissionContext =
recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
appSubmissionContext.setApplicationId(appID);
appSubmissionContext.setMasterCapability(recordFactory
.newRecordInstance(Resource.class));
appSubmissionContext.getMasterCapability().setMemory(1024);
// appSubmissionContext.resources = new HashMap<String, URL>();
ContainerLaunchContext amContainer =
recordFactory.newRecordInstance(ContainerLaunchContext.class);
amContainer.setResource(Resources.createResource(1024));
amContainer.setCommands(Arrays.asList("sleep", "100"));
appSubmissionContext.setUser("testUser");
// appSubmissionContext.environment = new HashMap<String, String>();
// appSubmissionContext.command = new ArrayList<String>();
appSubmissionContext.addCommand("sleep");
appSubmissionContext.addCommand("100");
// TODO: Use a resource to work around bugs. Today NM doesn't create local
// app-dirs if there are no file to download!!
@ -162,10 +161,11 @@ public void test() throws IOException, InterruptedException {
rsrc.setTimestamp(file.lastModified());
rsrc.setType(LocalResourceType.FILE);
rsrc.setVisibility(LocalResourceVisibility.PRIVATE);
appSubmissionContext.setResourceTodo("testFile", rsrc);
amContainer.setLocalResources(Collections.singletonMap("testFile", rsrc));
SubmitApplicationRequest submitRequest = recordFactory
.newRecordInstance(SubmitApplicationRequest.class);
submitRequest.setApplicationSubmissionContext(appSubmissionContext);
appSubmissionContext.setAMContainerSpec(amContainer);
resourceManager.getClientRMService().submitApplication(submitRequest);
// Wait till container gets allocated for AM

View File

@ -21,6 +21,7 @@
<%@ page
contentType="text/html; charset=UTF-8"
import="org.apache.hadoop.mapred.*"
import="org.apache.commons.lang.StringUtils"
import="javax.servlet.*"
import="javax.servlet.http.*"
import="java.io.*"
@ -51,7 +52,8 @@ import="java.util.*"
return "";
}
for (JobQueueInfo queueInfo : children) {
String variableName = queueInfo.getQueueName().replace(":", "_");
String variableName = StringUtils.replaceChars(queueInfo.getQueueName(),
":-*+#.^", "_______");
String label = queueInfo.getQueueName().split(":")[queueInfo
.getQueueName().split(":").length - 1];
str.append(String.format(