Merging trunk to HDFS-1623 branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1179484 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2011-10-06 01:16:48 +00:00
commit 60dcc5f237
217 changed files with 9037 additions and 2324 deletions

View File

@ -4,10 +4,6 @@ Trunk (unreleased changes)
INCOMPATIBLE CHANGES
HADOOP-7542. Change Configuration XML format to 1.1 to add support for
serializing additional characters. This requires XML1.1
support in the XML parser (Christopher Egner via harsh)
IMPROVEMENTS
HADOOP-7595. Upgrade dependency to Avro 1.5.3. (Alejandro Abdelnur via atm)
@ -25,6 +21,15 @@ Trunk (unreleased changes)
HADOOP-7687 Make getProtocolSignature public (sanjay)
HADOOP-7693. Enhance AvroRpcEngine to support the new #addProtocol
interface introduced in HADOOP-7524. (cutting)
HADOOP-7716. RPC protocol registration on SS does not log the protocol name
(only the class which may be different) (sanjay)
HADOOP-7717. Move handling of concurrent client fail-overs to
RetryInvocationHandler (atm)
BUGS
HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
@ -43,6 +48,11 @@ Trunk (unreleased changes)
HADOOP-6220. HttpServer wraps InterruptedExceptions by IOExceptions if interrupted
in startup (stevel)
HADOOP-7703. Improved excpetion handling of shutting down web server.
(Devaraj K via Eric Yang)
HADOOP-7704. Reduce number of object created by JMXJsonServlet.
(Devaraj K via Eric Yang)
Release 0.23.0 - Unreleased
@ -413,6 +423,18 @@ Release 0.23.0 - Unreleased
HADOOP-7575. Enhanced LocalDirAllocator to support fully-qualified
paths. (Jonathan Eagles via vinodkv)
HADOOP-7469 Add a standard handler for socket connection problems which
improves diagnostics (Uma Maheswara Rao G and stevel via stevel)
HADOOP-7710. Added hadoop-setup-application.sh for creating
application directory (Arpit Gupta via Eric Yang)
HADOOP-7707. Added toggle for dfs.support.append, webhdfs and hadoop proxy
user to setup config script. (Arpit Gupta via Eric Yang)
HADOOP-7720. Added parameter for HBase user to setup config script.
(Arpit Gupta via Eric Yang)
OPTIMIZATIONS
HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole
@ -634,6 +656,31 @@ Release 0.23.0 - Unreleased
HADOOP-7662. Fixed logs servlet to use the pathspec '/*' instead of '/'
for correct filtering. (Thomas Graves via vinodkv)
HADOOP-7691. Fixed conflict uid for install packages. (Eric Yang)
HADOOP-7603. Set hdfs, mapred uid, and hadoop uid to fixed numbers.
(Eric Yang)
HADOOP-7658. Fixed HADOOP_SECURE_DN_USER environment variable in
hadoop-evn.sh (Eric Yang)
HADOOP-7684. Added init.d script for jobhistory server and
secondary namenode. (Eric Yang)
HADOOP-7715. Removed unnecessary security logger configuration. (Eric Yang)
HADOOP-7685. Improved directory ownership check function in
hadoop-setup-conf.sh. (Eric Yang)
HADOOP-7711. Fixed recursive sourcing of HADOOP_OPTS environment
variables (Arpit Gupta via Eric Yang)
HADOOP-7681. Fixed security and hdfs audit log4j properties
(Arpit Gupta via Eric Yang)
HADOOP-7708. Fixed hadoop-setup-conf.sh to handle config files
consistently. (Eric Yang)
Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -1632,10 +1632,6 @@ private synchronized Document asXmlDocument() throws IOException {
try {
doc =
DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
// Allow a broader set of control characters to appear in job confs.
// cf https://issues.apache.org/jira/browse/MAPREDUCE-109
doc.setXmlVersion( "1.1" );
} catch (ParserConfigurationException pe) {
throw new IOException(pe);
}

View File

@ -210,7 +210,7 @@ public HttpServer(String name, String bindAddress, int port,
webServer.setHandler(contexts);
webAppContext = new WebAppContext();
webAppContext.setDisplayName("WepAppsContext");
webAppContext.setDisplayName(name);
webAppContext.setContextPath("/");
webAppContext.setWar(appDir + "/" + name);
webAppContext.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
@ -696,8 +696,44 @@ public void start() throws IOException {
* stop the server
*/
public void stop() throws Exception {
MultiException exception = null;
try {
listener.close();
} catch (Exception e) {
LOG.error("Error while stopping listener for webapp"
+ webAppContext.getDisplayName(), e);
exception = addMultiException(exception, e);
}
try {
// clear & stop webAppContext attributes to avoid memory leaks.
webAppContext.clearAttributes();
webAppContext.stop();
} catch (Exception e) {
LOG.error("Error while stopping web app context for webapp "
+ webAppContext.getDisplayName(), e);
exception = addMultiException(exception, e);
}
try {
webServer.stop();
} catch (Exception e) {
LOG.error("Error while stopping web server for webapp "
+ webAppContext.getDisplayName(), e);
exception = addMultiException(exception, e);
}
if (exception != null) {
exception.ifExceptionThrow();
}
}
private MultiException addMultiException(MultiException exception, Exception e) {
if(exception == null){
exception = new MultiException();
}
exception.add(e);
return exception;
}
public void join() throws InterruptedException {

View File

@ -24,6 +24,7 @@
import java.lang.reflect.Method;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -33,6 +34,11 @@ class RetryInvocationHandler implements InvocationHandler, Closeable {
public static final Log LOG = LogFactory.getLog(RetryInvocationHandler.class);
private FailoverProxyProvider proxyProvider;
/**
* The number of times the associated proxyProvider has ever been failed over.
*/
private long proxyProviderFailoverCount = 0;
private RetryPolicy defaultPolicy;
private Map<String,RetryPolicy> methodNameToPolicyMap;
private Object currentProxy;
@ -60,16 +66,24 @@ public Object invoke(Object proxy, Method method, Object[] args)
policy = defaultPolicy;
}
int failovers = 0;
// The number of times this method invocation has been failed over.
int invocationFailoverCount = 0;
int retries = 0;
while (true) {
// The number of times this invocation handler has ever been failed over,
// before this method invocation attempt. Used to prevent concurrent
// failed method invocations from triggering multiple failover attempts.
long invocationAttemptFailoverCount;
synchronized (proxyProvider) {
invocationAttemptFailoverCount = proxyProviderFailoverCount;
}
try {
return invokeMethod(method, args);
} catch (Exception e) {
boolean isMethodIdempotent = proxyProvider.getInterface()
.getMethod(method.getName(), method.getParameterTypes())
.isAnnotationPresent(Idempotent.class);
RetryAction action = policy.shouldRetry(e, retries++, failovers,
RetryAction action = policy.shouldRetry(e, retries++, invocationFailoverCount,
isMethodIdempotent);
if (action == RetryAction.FAIL) {
LOG.warn("Exception while invoking " + method.getName()
@ -81,10 +95,24 @@ public Object invoke(Object proxy, Method method, Object[] args)
} else if (action == RetryAction.FAILOVER_AND_RETRY) {
LOG.warn("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass()
+ ". Trying to fail over.", e);
failovers++;
+ " after " + invocationFailoverCount + " fail over attempts."
+ " Trying to fail over.", e);
// Make sure that concurrent failed method invocations only cause a
// single actual fail over.
synchronized (proxyProvider) {
if (invocationAttemptFailoverCount == proxyProviderFailoverCount) {
proxyProvider.performFailover(currentProxy);
proxyProviderFailoverCount++;
} else {
LOG.warn("A failover has occurred since the start of this method"
+ " invocation attempt.");
}
}
// The call to getProxy() could technically only be made in the event
// performFailover() is called, but it needs to be out here for the
// purpose of testing.
currentProxy = proxyProvider.getProxy();
invocationFailoverCount++;
}
if(LOG.isDebugEnabled()) {
LOG.debug("Exception while invoking " + method.getName()

View File

@ -29,6 +29,8 @@
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import javax.net.SocketFactory;
@ -54,7 +56,7 @@
public class AvroRpcEngine implements RpcEngine {
private static final Log LOG = LogFactory.getLog(RPC.class);
private static int VERSION = 0;
private static int VERSION = 1;
// the implementation we tunnel through
private static final RpcEngine ENGINE = new WritableRpcEngine();
@ -62,9 +64,10 @@ public class AvroRpcEngine implements RpcEngine {
/** Tunnel an Avro RPC request and response through Hadoop's RPC. */
private static interface TunnelProtocol extends VersionedProtocol {
//WritableRpcEngine expects a versionID in every protocol.
public static final long versionID = 0L;
public static final long versionID = VERSION;
/** All Avro methods and responses go through this. */
BufferListWritable call(BufferListWritable request) throws IOException;
BufferListWritable call(String protocol, BufferListWritable request)
throws IOException;
}
/** A Writable that holds a List<ByteBuffer>, The Avro RPC Transceiver's
@ -103,23 +106,25 @@ public void write(DataOutput out) throws IOException {
private static class ClientTransceiver extends Transceiver {
private TunnelProtocol tunnel;
private InetSocketAddress remote;
private String protocol;
public ClientTransceiver(InetSocketAddress addr,
UserGroupInformation ticket,
Configuration conf, SocketFactory factory,
int rpcTimeout)
int rpcTimeout, String protocol)
throws IOException {
this.tunnel = ENGINE.getProxy(TunnelProtocol.class, VERSION,
addr, ticket, conf, factory,
rpcTimeout).getProxy();
this.remote = addr;
this.protocol = protocol;
}
public String getRemoteName() { return remote.toString(); }
public List<ByteBuffer> transceive(List<ByteBuffer> request)
throws IOException {
return tunnel.call(new BufferListWritable(request)).buffers;
return tunnel.call(protocol, new BufferListWritable(request)).buffers;
}
public List<ByteBuffer> readBuffers() throws IOException {
@ -159,7 +164,8 @@ public Invoker(Class<?> protocol, InetSocketAddress addr,
UserGroupInformation ticket, Configuration conf,
SocketFactory factory,
int rpcTimeout) throws IOException {
this.tx = new ClientTransceiver(addr, ticket, conf, factory, rpcTimeout);
this.tx = new ClientTransceiver(addr, ticket, conf, factory, rpcTimeout,
protocol.getName());
this.requestor = createRequestor(protocol, tx);
}
@Override public Object invoke(Object proxy, Method method, Object[] args)
@ -182,9 +188,11 @@ protected Responder createResponder(Class<?> iface, Object impl) {
/** An Avro RPC Responder that can process requests passed via Hadoop RPC. */
private class TunnelResponder implements TunnelProtocol {
private Responder responder;
public TunnelResponder(Class<?> iface, Object impl) {
responder = createResponder(iface, impl);
private Map<String, Responder> responders =
new HashMap<String, Responder>();
public void addProtocol(Class<?> iface, Object impl) {
responders.put(iface.getName(), createResponder(iface, impl));
}
@Override
@ -197,13 +205,18 @@ public long getProtocolVersion(String protocol, long version)
public ProtocolSignature getProtocolSignature(
String protocol, long version, int clientMethodsHashCode)
throws IOException {
return new ProtocolSignature(VERSION, null);
return ProtocolSignature.getProtocolSignature
(clientMethodsHashCode, VERSION, TunnelProtocol.class);
}
public BufferListWritable call(final BufferListWritable request)
public BufferListWritable call(String protocol, BufferListWritable request)
throws IOException {
Responder responder = responders.get(protocol);
if (responder == null)
throw new IOException("No responder for: "+protocol);
return new BufferListWritable(responder.respond(request.buffers));
}
}
public Object[] call(Method method, Object[][] params,
@ -212,6 +225,32 @@ public Object[] call(Method method, Object[][] params,
throw new UnsupportedOperationException();
}
private class Server extends WritableRpcEngine.Server {
private TunnelResponder responder = new TunnelResponder();
public Server(Class<?> iface, Object impl, String bindAddress,
int port, int numHandlers, int numReaders,
int queueSizePerHandler, boolean verbose,
Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager
) throws IOException {
super((Class)null, new Object(), conf,
bindAddress, port, numHandlers, numReaders,
queueSizePerHandler, verbose, secretManager);
super.addProtocol(TunnelProtocol.class, responder);
responder.addProtocol(iface, impl);
}
@Override
public <PROTO, IMPL extends PROTO> Server
addProtocol(Class<PROTO> protocolClass, IMPL protocolImpl)
throws IOException {
responder.addProtocol(protocolClass, protocolImpl);
return this;
}
}
/** Construct a server for a protocol implementation instance listening on a
* port and address. */
public RPC.Server getServer(Class<?> iface, Object impl, String bindAddress,
@ -220,9 +259,8 @@ public RPC.Server getServer(Class<?> iface, Object impl, String bindAddress,
Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager
) throws IOException {
return ENGINE.getServer(TunnelProtocol.class,
new TunnelResponder(iface, impl),
bindAddress, port, numHandlers, numReaders,
return new Server
(iface, impl, bindAddress, port, numHandlers, numReaders,
queueSizePerHandler, verbose, conf, secretManager);
}

View File

@ -23,8 +23,6 @@
import java.net.InetSocketAddress;
import java.net.SocketTimeoutException;
import java.net.UnknownHostException;
import java.net.ConnectException;
import java.io.IOException;
import java.io.DataInputStream;
import java.io.DataOutputStream;
@ -235,8 +233,11 @@ public Connection(ConnectionId remoteId) throws IOException {
this.remoteId = remoteId;
this.server = remoteId.getAddress();
if (server.isUnresolved()) {
throw new UnknownHostException("unknown host: " +
remoteId.getAddress().getHostName());
throw NetUtils.wrapException(remoteId.getAddress().getHostName(),
remoteId.getAddress().getPort(),
null,
0,
new UnknownHostException());
}
this.rpcTimeout = remoteId.getRpcTimeout();
this.maxIdleTime = remoteId.getMaxIdleTime();
@ -1084,7 +1085,12 @@ public Writable call(Writable param, ConnectionId remoteId)
call.error.fillInStackTrace();
throw call.error;
} else { // local exception
throw wrapException(remoteId.getAddress(), call.error);
InetSocketAddress address = remoteId.getAddress();
throw NetUtils.wrapException(address.getHostName(),
address.getPort(),
NetUtils.getHostname(),
0,
call.error);
}
} else {
return call.value;
@ -1092,37 +1098,6 @@ public Writable call(Writable param, ConnectionId remoteId)
}
}
/**
* Take an IOException and the address we were trying to connect to
* and return an IOException with the input exception as the cause.
* The new exception provides the stack trace of the place where
* the exception is thrown and some extra diagnostics information.
* If the exception is ConnectException or SocketTimeoutException,
* return a new one of the same type; Otherwise return an IOException.
*
* @param addr target address
* @param exception the relevant exception
* @return an exception to throw
*/
private IOException wrapException(InetSocketAddress addr,
IOException exception) {
if (exception instanceof ConnectException) {
//connection refused; include the host:port in the error
return (ConnectException)new ConnectException(
"Call to " + addr + " failed on connection exception: " + exception)
.initCause(exception);
} else if (exception instanceof SocketTimeoutException) {
return (SocketTimeoutException)new SocketTimeoutException(
"Call to " + addr + " failed on socket timeout exception: "
+ exception).initCause(exception);
} else {
return (IOException)new IOException(
"Call to " + addr + " failed on local exception: " + exception)
.initCause(exception);
}
}
/**
* @deprecated Use {@link #call(Writable[], InetSocketAddress[],
* Class, UserGroupInformation, Configuration)} instead

View File

@ -605,7 +605,7 @@ protected Server(String bindAddress, int port,
* @param protocolImpl - the impl of the protocol that will be called
* @return the server (for convenience)
*/
public <PROTO extends VersionedProtocol, IMPL extends PROTO>
public <PROTO, IMPL extends PROTO>
Server addProtocol(Class<PROTO> protocolClass, IMPL protocolImpl
) throws IOException {
throw new IOException("addProtocol Not Implemented");

View File

@ -51,8 +51,6 @@
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import javax.security.sasl.Sasl;
import javax.security.sasl.SaslException;
@ -70,6 +68,7 @@
import org.apache.hadoop.ipc.RPC.VersionMismatch;
import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
import org.apache.hadoop.ipc.metrics.RpcMetrics;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.SaslRpcServer;
import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
@ -227,20 +226,11 @@ public static void bind(ServerSocket socket, InetSocketAddress address,
int backlog) throws IOException {
try {
socket.bind(address, backlog);
} catch (BindException e) {
BindException bindException = new BindException("Problem binding to " + address
+ " : " + e.getMessage());
bindException.initCause(e);
throw bindException;
} catch (SocketException e) {
// If they try to bind to a different host's address, give a better
// error message.
if ("Unresolved address".equals(e.getMessage())) {
throw new UnknownHostException("Invalid hostname for server: " +
address.getHostName());
} else {
throw e;
}
throw NetUtils.wrapException(null,
0,
address.getHostName(),
address.getPort(), e);
}
}

View File

@ -388,8 +388,9 @@ private void registerProtocolAndImpl(Class<?> protocolClass,
}
protocolImplMap.put(new ProtoNameVer(protocolName, version),
new ProtoClassProtoImpl(protocolClass, protocolImpl));
LOG.info("ProtocolImpl=" + protocolImpl.getClass().getName() +
" protocolClass=" + protocolClass.getName() + " version=" + version);
LOG.info("Protocol Name = " + protocolName + " version=" + version +
" ProtocolImpl=" + protocolImpl.getClass().getName() +
" protocolClass=" + protocolClass.getName());
}
private static class VerProtocolImpl {
@ -555,7 +556,7 @@ public Server(Class<?> protocolClass, Object protocolImpl,
@Override
public <PROTO extends VersionedProtocol, IMPL extends PROTO> Server
public <PROTO, IMPL extends PROTO> Server
addProtocol(
Class<PROTO> protocolClass, IMPL protocolImpl) throws IOException {
registerProtocolAndImpl(protocolClass, protocolImpl);

View File

@ -117,13 +117,15 @@ public class JMXJsonServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
// ----------------------------------------------------- Instance Variables
/**
* MBean server.
*/
protected transient MBeanServer mBeanServer = null;
protected transient MBeanServer mBeanServer;
// --------------------------------------------------------- Public Methods
/**
* Json Factory to create Json generators for write objects in json format
*/
protected transient JsonFactory jsonFactory;
/**
* Initialize this servlet.
*/
@ -131,6 +133,7 @@ public class JMXJsonServlet extends HttpServlet {
public void init() throws ServletException {
// Retrieve the MBean server
mBeanServer = ManagementFactory.getPlatformMBeanServer();
jsonFactory = new JsonFactory();
}
/**
@ -149,23 +152,14 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) {
response)) {
return;
}
JsonGenerator jg = null;
try {
response.setContentType("application/json; charset=utf8");
PrintWriter writer = response.getWriter();
JsonFactory jsonFactory = new JsonFactory();
JsonGenerator jg = jsonFactory.createJsonGenerator(writer);
jg = jsonFactory.createJsonGenerator(writer);
jg.useDefaultPrettyPrinter();
jg.writeStartObject();
if (mBeanServer == null) {
jg.writeStringField("result", "ERROR");
jg.writeStringField("message", "No MBeanServer could be found");
jg.close();
LOG.error("No MBeanServer could be found.");
response.setStatus(HttpServletResponse.SC_NOT_FOUND);
return;
}
// query per mbean attribute
String getmethod = request.getParameter("get");
@ -174,13 +168,11 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) {
if (splitStrings.length != 2) {
jg.writeStringField("result", "ERROR");
jg.writeStringField("message", "query format is not as expected.");
jg.close();
response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
return;
}
listBeans(jg, new ObjectName(splitStrings[0]), splitStrings[1],
response);
jg.close();
return;
}
@ -190,12 +182,15 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) {
qry = "*:*";
}
listBeans(jg, new ObjectName(qry), null, response);
} finally {
if (jg != null) {
jg.close();
} catch ( IOException e ) {
}
}
} catch (IOException e) {
LOG.error("Caught an exception while processing JMX request", e);
response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
} catch ( MalformedObjectNameException e ) {
} catch (MalformedObjectNameException e) {
LOG.error("Caught an exception while processing JMX request", e);
response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
}

View File

@ -20,12 +20,15 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.BindException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.NetworkInterface;
import java.net.NoRouteToHostException;
import java.net.Socket;
import java.net.SocketAddress;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.UnknownHostException;
@ -54,6 +57,13 @@ public class NetUtils {
private static Map<String, String> hostToResolved =
new HashMap<String, String>();
/** text to point users elsewhere: {@value} */
private static final String FOR_MORE_DETAILS_SEE
= " For more details see: ";
/** text included in wrapped exceptions if the host is null: {@value} */
public static final String UNKNOWN_HOST = "(unknown)";
/** Base URL of the Hadoop Wiki: {@value} */
public static final String HADOOP_WIKI = "http://wiki.apache.org/hadoop/";
/**
* Get the socket factory for the given class according to its
@ -537,4 +547,119 @@ public static boolean isLocalAddress(InetAddress addr) {
}
return local;
}
/**
* Take an IOException , the local host port and remote host port details and
* return an IOException with the input exception as the cause and also
* include the host details. The new exception provides the stack trace of the
* place where the exception is thrown and some extra diagnostics information.
* If the exception is BindException or ConnectException or
* UnknownHostException or SocketTimeoutException, return a new one of the
* same type; Otherwise return an IOException.
*
* @param destHost target host (nullable)
* @param destPort target port
* @param localHost local host (nullable)
* @param localPort local port
* @param exception the caught exception.
* @return an exception to throw
*/
public static IOException wrapException(final String destHost,
final int destPort,
final String localHost,
final int localPort,
final IOException exception) {
if (exception instanceof BindException) {
return new BindException(
"Problem binding to ["
+ localHost
+ ":"
+ localPort
+ "] "
+ exception
+ ";"
+ see("BindException"));
} else if (exception instanceof ConnectException) {
// connection refused; include the host:port in the error
return (ConnectException) new ConnectException(
"Call From "
+ localHost
+ " to "
+ destHost
+ ":"
+ destPort
+ " failed on connection exception: "
+ exception
+ ";"
+ see("ConnectionRefused"))
.initCause(exception);
} else if (exception instanceof UnknownHostException) {
return (UnknownHostException) new UnknownHostException(
"Invalid host name: "
+ getHostDetailsAsString(destHost, destPort, localHost)
+ exception
+ ";"
+ see("UnknownHost"))
.initCause(exception);
} else if (exception instanceof SocketTimeoutException) {
return (SocketTimeoutException) new SocketTimeoutException(
"Call From "
+ localHost + " to " + destHost + ":" + destPort
+ " failed on socket timeout exception: " + exception
+ ";"
+ see("SocketTimeout"))
.initCause(exception);
} else if (exception instanceof NoRouteToHostException) {
return (NoRouteToHostException) new NoRouteToHostException(
"No Route to Host from "
+ localHost + " to " + destHost + ":" + destPort
+ " failed on socket timeout exception: " + exception
+ ";"
+ see("NoRouteToHost"))
.initCause(exception);
}
else {
return (IOException) new IOException("Failed on local exception: "
+ exception
+ "; Host Details : "
+ getHostDetailsAsString(destHost, destPort, localHost))
.initCause(exception);
}
}
private static String see(final String entry) {
return FOR_MORE_DETAILS_SEE + HADOOP_WIKI + entry;
}
/**
* Get the host details as a string
* @param destHost destinatioon host (nullable)
* @param destPort destination port
* @param localHost local host (nullable)
* @return a string describing the destination host:port and the local host
*/
private static String getHostDetailsAsString(final String destHost,
final int destPort,
final String localHost) {
StringBuilder hostDetails = new StringBuilder(27);
hostDetails.append("local host is: ")
.append(quoteHost(localHost))
.append("; ");
hostDetails.append("destination host is: \"").append(quoteHost(destHost))
.append(":")
.append(destPort).append("; ");
return hostDetails.toString();
}
/**
* Quote a hostname if it is not null
* @param hostname the hostname; nullable
* @return a quoted hostname or {@link #UNKNOWN_HOST} if the hostname is null
*/
private static String quoteHost(final String hostname) {
return (hostname != null) ?
("\"" + hostname + "\"")
: UNKNOWN_HOST;
}
}

View File

@ -15,4 +15,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
getent group hadoop 2>/dev/null >/dev/null || /usr/sbin/groupadd -r hadoop
getent group hadoop 2>/dev/null >/dev/null || /usr/sbin/groupadd -g 123 -r hadoop

View File

@ -0,0 +1,140 @@
#!/usr/bin/env bash
# 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="${BASH_SOURCE-$0}"
bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P)
script="$(basename -- "$this")"
this="$bin/$script"
. "$bin"/../libexec/hadoop-config.sh
usage() {
echo "
usage: $0 <parameters>
Require parameter:
--config /etc/hadoop Location of Hadoop configuration file
--apps=<csl of apps:user hcat:hcat,hbase,hive:user> Apps you want to setup on hdfs
If user is not specified, app name
will be used as the user name as well
Optional parameters:
-h Display this message
--kerberos-realm=KERBEROS.EXAMPLE.COM Set Kerberos realm
--super-user=hdfs Set super user id
--super-user-keytab=/etc/security/keytabs/hdfs.keytab Set super user keytab location
"
exit 1
}
OPTS=$(getopt \
-n $0 \
-o '' \
-l 'kerberos-realm:' \
-l 'super-user:' \
-l 'super-user-keytab:' \
-l 'apps:' \
-o 'h' \
-- "$@")
if [ $? != 0 ] ; then
usage
exit 1
fi
function setup_apps
{
if [ -z $APPS ]
then
usage
break
fi
#if super user is not set default to hdfs
HADOOP_HDFS_USER=${HADOOP_HDFS_USER:-hdfs}
if [ ! "${KERBEROS_REALM}" = "" ]; then
# locate kinit cmd
if [ -e /etc/lsb-release ]; then
KINIT_CMD="/usr/bin/kinit -kt ${HDFS_USER_KEYTAB} ${HADOOP_HDFS_USER}"
else
KINIT_CMD="/usr/kerberos/bin/kinit -kt ${HDFS_USER_KEYTAB} ${HADOOP_HDFS_USER}"
fi
su -c "${KINIT_CMD}" ${HADOOP_HDFS_USER}
fi
#process each app
oldIFS=$IFS
IFS=','
for app in $APPS
do
IFS=":"
arr=($app)
app=${arr[0]}
user=${arr[1]}
IFS=','
#if user is empty, default it to app
if [ -z $user ]
then
user=$app
fi
path="/apps/${app}"
#create the dir
cmd="su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -mkdir ${path}' ${HADOOP_HDFS_USER}"
echo $cmd
eval $cmd
#make owner to be the app
cmd="su -c '${HADOOP_PREFIX}/bin/hadoop --config ${HADOOP_CONF_DIR} dfs -chown ${user} ${path}' ${HADOOP_HDFS_USER}"
echo $cmd
eval $cmd
if [ "$?" == "0" ]; then
echo "App directory has been setup: ${path}"
fi
done
IFS=$oldIFS
}
eval set -- "${OPTS}"
while true; do
case "$1" in
--apps)
APPS=$2; shift 2
;;
--kerberos-realm)
KERBEROS_REALM=$2; shift 2
;;
--super-user)
HADOOP_HDFS_USER=$2; shift 2
;;
--super-user-keytab)
HDFS_USER_KEYTAB=$2; shift 2
;;
-h)
usage
;;
--)
shift ; break
;;
*)
echo "Unknown option: $1"
usage
exit 1
;;
esac
done
setup_apps

View File

@ -51,6 +51,10 @@ usage: $0 <parameters>
--taskscheduler=org.apache.hadoop.mapred.JobQueueTaskScheduler Set task scheduler
--datanodes=hostname1,hostname2,... SET the datanodes
--tasktrackers=hostname1,hostname2,... SET the tasktrackers
--dfs-webhdfs-enabled=false|true Enable webhdfs
--dfs-support-append=false|true Enable append
--hadoop-proxy-users='user1:groups:hosts;user2:groups:hosts' Setup proxy users for hadoop
--hbase-user=hbase User which hbase is running as. Defaults to hbase
"
exit 1
}
@ -60,9 +64,11 @@ check_permission() {
OWNER="0"
RESULT=0
while [ "$TARGET" != "/" ]; do
PARENT=`dirname $TARGET`
NAME=`basename $TARGET`
OWNER=`ls -ln $PARENT | grep $NAME| awk '{print $3}'`
if [ "`uname`" = "Darwin" ]; then
OWNER=`stat -f %u $TARGET`
else
OWNER=`stat -c %u $TARGET`
fi
if [ "$OWNER" != "0" ]; then
RESULT=1
break
@ -74,6 +80,9 @@ check_permission() {
template_generator() {
REGEX='(\$\{[a-zA-Z_][a-zA-Z_0-9]*\})'
if [ -e $2 ]; then
mv -f $2 "$2.bak"
fi
cat $1 |
while read line ; do
while [[ "$line" =~ $REGEX ]] ; do
@ -85,6 +94,78 @@ template_generator() {
done
}
#########################################
# Function to modify a value of a field in an xml file
# Params: $1 is the file with full path; $2 is the property, $3 is the new value
#########################################
function addPropertyToXMLConf
{
#read the file name with full path
local file=$1
#get the property name
local property=$2
#get what value should be set for that
local propValue=$3
#get the description
local desc=$4
#get the value for the final tag
local finalVal=$5
#create the property text, make sure the / are escaped
propText="<property>\n<name>$property<\/name>\n<value>$propValue<\/value>"
#if description is not empty add it
if [ ! -z $desc ]
then
propText="${propText}<description>$desc<\/description>\n"
fi
#if final is not empty add it
if [ ! -z $finalVal ]
then
propText="${propText}final>$finalVal<\/final>\n"
fi
#add the ending tag
propText="${propText}<\/property>\n"
#add the property to the file
endText="<\/configuration>"
#add the text using sed at the end of the file
sed -i "s|$endText|$propText$endText|" $file
}
##########################################
# Function to setup up the proxy user settings
#########################################
function setupProxyUsers
{
#if hadoop proxy users are sent, setup hadoop proxy
if [ ! -z $HADOOP_PROXY_USERS ]
then
oldIFS=$IFS
IFS=';'
#process each proxy config
for proxy in $HADOOP_PROXY_USERS
do
#get the user, group and hosts information for each proxy
IFS=':'
arr=($proxy)
user="${arr[0]}"
groups="${arr[1]}"
hosts="${arr[2]}"
#determine the property names and values
proxy_groups_property="hadoop.proxyuser.${user}.groups"
proxy_groups_val="$groups"
addPropertyToXMLConf "${HADOOP_CONF_DIR}/hdfs-site.xml" "$proxy_groups_property" "$proxy_groups_val"
proxy_hosts_property="hadoop.proxyuser.${user}.hosts"
proxy_hosts_val="$hosts"
addPropertyToXMLConf "${HADOOP_CONF_DIR}/hdfs-site.xml" "$proxy_hosts_property" "$proxy_hosts_val"
IFS=';'
done
IFS=$oldIFS
fi
}
OPTS=$(getopt \
-n $0 \
-o '' \
@ -113,6 +194,10 @@ OPTS=$(getopt \
-l 'kinit-location:' \
-l 'datanodes:' \
-l 'tasktrackers:' \
-l 'dfs-webhdfs-enabled:' \
-l 'hadoop-proxy-users:' \
-l 'dfs-support-append:' \
-l 'hbase-user:' \
-o 'h' \
-- "$@")
@ -232,6 +317,22 @@ while true ; do
AUTOMATED=1
TASKTRACKERS=$(echo $TASKTRACKERS | tr ',' ' ')
;;
--dfs-webhdfs-enabled)
DFS_WEBHDFS_ENABLED=$2; shift 2
AUTOMATED=1
;;
--hadoop-proxy-users)
HADOOP_PROXY_USERS=$2; shift 2
AUTOMATED=1
;;
--dfs-support-append)
DFS_SUPPORT_APPEND=$2; shift 2
AUTOMATED=1
;;
--hbase-user)
HBASE_USER=$2; shift 2
AUTOMATED=1
;;
--)
shift ; break
;;
@ -247,6 +348,7 @@ AUTOSETUP=${AUTOSETUP:-1}
JAVA_HOME=${JAVA_HOME:-/usr/java/default}
HADOOP_GROUP=${HADOOP_GROUP:-hadoop}
HADOOP_NN_HOST=${HADOOP_NN_HOST:-`hostname`}
HADOOP_SNN_HOST=${HADOOP_SNN_HOST:-`hostname`}
HADOOP_NN_DIR=${HADOOP_NN_DIR:-/var/lib/hadoop/hdfs/namenode}
HADOOP_DN_DIR=${HADOOP_DN_DIR:-/var/lib/hadoop/hdfs/datanode}
HADOOP_JT_HOST=${HADOOP_JT_HOST:-`hostname`}
@ -259,9 +361,14 @@ HADOOP_REPLICATION=${HADOOP_RELICATION:-3}
HADOOP_TASK_SCHEDULER=${HADOOP_TASK_SCHEDULER:-org.apache.hadoop.mapred.JobQueueTaskScheduler}
HADOOP_HDFS_USER=${HADOOP_HDFS_USER:-hdfs}
HADOOP_MR_USER=${HADOOP_MR_USER:-mr}
DFS_WEBHDFS_ENABLED=${DFS_WEBHDFS_ENABLED:-false}
DFS_SUPPORT_APPEND=${DFS_SUPPORT_APPEND:-false}
HBASE_USER=${HBASE_USER:-hbase}
KEYTAB_DIR=${KEYTAB_DIR:-/etc/security/keytabs}
HDFS_KEYTAB=${HDFS_KEYTAB:-/home/hdfs/hdfs.keytab}
MR_KEYTAB=${MR_KEYTAB:-/home/mr/mr.keytab}
DFS_WEBHDFS_ENABLED=${DFS_WEBHDFS_ENABLED:-false}
DFS_SUPPORT_APPEND=${DFS_SUPPORT_APPEND:-false}
KERBEROS_REALM=${KERBEROS_REALM:-KERBEROS.EXAMPLE.COM}
SECURITY_TYPE=${SECURITY_TYPE:-simple}
KINIT=${KINIT:-/usr/kerberos/bin/kinit}
@ -270,13 +377,18 @@ if [ "${SECURITY_TYPE}" = "kerberos" ]; then
HADOOP_DN_ADDR="0.0.0.0:1019"
HADOOP_DN_HTTP_ADDR="0.0.0.0:1022"
SECURITY="true"
HADOOP_SECURE_DN_USER=${HADOOP_HDFS_USER}
else
TASK_CONTROLLER="org.apache.hadoop.mapred.DefaultTaskController"
HADDOP_DN_ADDR="0.0.0.0:50010"
HADOOP_DN_ADDR="0.0.0.0:50010"
HADOOP_DN_HTTP_ADDR="0.0.0.0:50075"
SECURITY="false"
HADOOP_SECURE_DN_USER=""
fi
#unset env vars
unset HADOOP_CLIENT_OPTS HADOOP_NAMENODE_OPTS HADOOP_JOBTRACKER_OPTS HADOOP_TASKTRACKER_OPTS HADOOP_DATANODE_OPTS HADOOP_SECONDARYNAMENODE_OPTS HADOOP_JAVA_PLATFORM_OPTS
if [ "${AUTOMATED}" != "1" ]; then
echo "Setup Hadoop Configuration"
echo
@ -383,46 +495,6 @@ if [ "${AUTOSETUP}" == "1" -o "${AUTOSETUP}" == "y" ]; then
chmod 755 ${HADOOP_LOG_DIR}/${HADOOP_MR_USER}
chown ${HADOOP_MR_USER}:${HADOOP_GROUP} ${HADOOP_LOG_DIR}/${HADOOP_MR_USER}
if [ -e ${HADOOP_CONF_DIR}/core-site.xml ]; then
mv -f ${HADOOP_CONF_DIR}/core-site.xml ${HADOOP_CONF_DIR}/core-site.xml.bak
fi
if [ -e ${HADOOP_CONF_DIR}/hdfs-site.xml ]; then
mv -f ${HADOOP_CONF_DIR}/hdfs-site.xml ${HADOOP_CONF_DIR}/hdfs-site.xml.bak
fi
if [ -e ${HADOOP_CONF_DIR}/mapred-site.xml ]; then
mv -f ${HADOOP_CONF_DIR}/mapred-site.xml ${HADOOP_CONF_DIR}/mapred-site.xml.bak
fi
if [ -e ${HADOOP_CONF_DIR}/hadoop-env.sh ]; then
mv -f ${HADOOP_CONF_DIR}/hadoop-env.sh ${HADOOP_CONF_DIR}/hadoop-env.sh.bak
fi
if [ -e ${HADOOP_CONF_DIR}/hadoop-policy.xml ]; then
mv -f ${HADOOP_CONF_DIR}/hadoop-policy.xml ${HADOOP_CONF_DIR}/hadoop-policy.xml.bak
fi
if [ -e ${HADOOP_CONF_DIR}/mapred-queue-acls.xml ]; then
mv -f ${HADOOP_CONF_DIR}/mapred-queue-acls.xml ${HADOOP_CONF_DIR}/mapred-queue-acls.xml.bak
fi
if [ -e ${HADOOP_CONF_DIR}/commons-logging.properties ]; then
mv -f ${HADOOP_CONF_DIR}/commons-logging.properties ${HADOOP_CONF_DIR}/commons-logging.properties.bak
fi
if [ -e ${HADOOP_CONF_DIR}/taskcontroller.cfg ]; then
mv -f ${HADOOP_CONF_DIR}/taskcontroller.cfg ${HADOOP_CONF_DIR}/taskcontroller.cfg.bak
fi
if [ -e ${HADOOP_CONF_DIR}/slaves ]; then
mv -f ${HADOOP_CONF_DIR}/slaves ${HADOOP_CONF_DIR}/slaves.bak
fi
if [ -e ${HADOOP_CONF_DIR}/dfs.include ]; then
mv -f ${HADOOP_CONF_DIR}/dfs.include ${HADOOP_CONF_DIR}/dfs.include.bak
fi
if [ -e ${HADOOP_CONF_DIR}/dfs.exclude ]; then
mv -f ${HADOOP_CONF_DIR}/dfs.exclude ${HADOOP_CONF_DIR}/dfs.exclude.bak
fi
if [ -e ${HADOOP_CONF_DIR}/mapred.include ]; then
mv -f ${HADOOP_CONF_DIR}/mapred.include ${HADOOP_CONF_DIR}/mapred.include.bak
fi
if [ -e ${HADOOP_CONF_DIR}/mapred.exclude ]; then
mv -f ${HADOOP_CONF_DIR}/mapred.exclude ${HADOOP_CONF_DIR}/mapred.exclude.bak
fi
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/core-site.xml ${HADOOP_CONF_DIR}/core-site.xml
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hdfs-site.xml ${HADOOP_CONF_DIR}/hdfs-site.xml
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/mapred-site.xml ${HADOOP_CONF_DIR}/mapred-site.xml
@ -431,6 +503,12 @@ if [ "${AUTOSETUP}" == "1" -o "${AUTOSETUP}" == "y" ]; then
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/commons-logging.properties ${HADOOP_CONF_DIR}/commons-logging.properties
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/mapred-queue-acls.xml ${HADOOP_CONF_DIR}/mapred-queue-acls.xml
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/taskcontroller.cfg ${HADOOP_CONF_DIR}/taskcontroller.cfg
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/capacity-scheduler.xml ${HADOOP_CONF_DIR}/capacity-scheduler.xml
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/log4j.properties ${HADOOP_CONF_DIR}/log4j.properties
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-metrics2.properties ${HADOOP_CONF_DIR}/hadoop-metrics2.properties
#setup up the proxy users
setupProxyUsers
#set the owner of the hadoop dir to root
chown root ${HADOOP_PREFIX}
@ -474,15 +552,12 @@ else
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/mapred-queue-acls.xml ${HADOOP_CONF_DIR}/mapred-queue-acls.xml
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/taskcontroller.cfg ${HADOOP_CONF_DIR}/taskcontroller.cfg
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-metrics2.properties ${HADOOP_CONF_DIR}/hadoop-metrics2.properties
if [ ! -e ${HADOOP_CONF_DIR}/capacity-scheduler.xml ]; then
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/capacity-scheduler.xml ${HADOOP_CONF_DIR}/capacity-scheduler.xml
fi
if [ ! -e ${HADOOP_CONF_DIR}/hadoop-metrics2.properties ]; then
cp ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-metrics2.properties ${HADOOP_CONF_DIR}/hadoop-metrics2.properties
fi
if [ ! -e ${HADOOP_CONF_DIR}/log4j.properties ]; then
cp ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/log4j.properties ${HADOOP_CONF_DIR}/log4j.properties
fi
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/log4j.properties ${HADOOP_CONF_DIR}/log4j.properties
template_generator ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/hadoop-metrics2.properties ${HADOOP_CONF_DIR}/hadoop-metrics2.properties
#setup up the proxy users
setupProxyUsers
chown root:${HADOOP_GROUP} ${HADOOP_CONF_DIR}/hadoop-env.sh
chmod 755 ${HADOOP_CONF_DIR}/hadoop-env.sh
@ -515,6 +590,12 @@ else
echo "${HADOOP_CONF_DIR}/hdfs-site.xml"
echo "${HADOOP_CONF_DIR}/mapred-site.xml"
echo "${HADOOP_CONF_DIR}/hadoop-env.sh"
echo "${HADOOP_CONF_DIR}/hadoop-policy.xml"
echo "${HADOOP_CONF_DIR}/commons-logging.properties"
echo "${HADOOP_CONF_DIR}/taskcontroller.cfg"
echo "${HADOOP_CONF_DIR}/capacity-scheduler.xml"
echo "${HADOOP_CONF_DIR}/log4j.properties"
echo "${HADOOP_CONF_DIR}/hadoop-metrics2.properties"
echo
echo " to ${HADOOP_CONF_DIR} on all nodes, and proceed to run hadoop-setup-hdfs.sh on namenode."
fi

View File

@ -132,7 +132,7 @@ mv ${RPM_BUILD_DIR}/%{_final_name}/share/* ${RPM_BUILD_DIR}%{_share_dir}
rm -rf ${RPM_BUILD_DIR}/%{_final_name}/etc
%pre
getent group hadoop 2>/dev/null >/dev/null || /usr/sbin/groupadd -r hadoop
getent group hadoop 2>/dev/null >/dev/null || /usr/sbin/groupadd -g 123 -r hadoop
%post
bash ${RPM_INSTALL_PREFIX0}/sbin/update-hadoop-env.sh \

View File

@ -58,6 +58,8 @@
RULE:[2:$1@$0](hdfs@.*${KERBEROS_REALM})s/.*/${HADOOP_HDFS_USER}/
RULE:[2:$1@$0](mapredqa@.*${KERBEROS_REALM})s/.*/${HADOOP_MR_USER}/
RULE:[2:$1@$0](hdfsqa@.*${KERBEROS_REALM})s/.*/${HADOOP_HDFS_USER}/
RULE:[2:$1@$0](hm@.*${KERBEROS_REALM})s/.*/${HBASE_USER}/
RULE:[2:$1@$0](rs@.*${KERBEROS_REALM})s/.*/${HBASE_USER}/
DEFAULT
</value>
<description></description>

View File

@ -41,22 +41,22 @@ done
#export HADOOP_NAMENODE_INIT_HEAPSIZE=""
# Extra Java runtime options. Empty by default.
export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true $HADOOP_OPTS"
export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true $HADOOP_CLIENT_OPTS"
# Command specific options appended to HADOOP_OPTS when specified
export HADOOP_NAMENODE_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dhdfs.audit.logger=INFO,DRFAAUDIT ${HADOOP_NAMENODE_OPTS}"
HADOOP_JOBTRACKER_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dmapred.audit.logger=INFO,MRAUDIT -Dmapred.jobsummary.logger=INFO,JSA ${HADOOP_JOBTRACKER_OPTS}"
HADOOP_TASKTRACKER_OPTS="-Dsecurity.audit.logger=ERROR,console -Dmapred.audit.logger=ERROR,console ${HADOOP_TASKTRACKER_OPTS}"
HADOOP_DATANODE_OPTS="-Dsecurity.audit.logger=ERROR,DRFAS ${HADOOP_DATANODE_OPTS}"
export HADOOP_NAMENODE_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dhdfs.audit.logger=INFO,DRFAAUDIT $HADOOP_NAMENODE_OPTS"
HADOOP_JOBTRACKER_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dmapred.audit.logger=INFO,MRAUDIT -Dmapred.jobsummary.logger=INFO,JSA $HADOOP_JOBTRACKER_OPTS"
HADOOP_TASKTRACKER_OPTS="-Dsecurity.audit.logger=ERROR,console -Dmapred.audit.logger=ERROR,console $HADOOP_TASKTRACKER_OPTS"
HADOOP_DATANODE_OPTS="-Dsecurity.audit.logger=ERROR,DRFAS $HADOOP_DATANODE_OPTS"
export HADOOP_SECONDARYNAMENODE_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dhdfs.audit.logger=INFO,DRFAAUDIT ${HADOOP_SECONDARYNAMENODE_OPTS}"
export HADOOP_SECONDARYNAMENODE_OPTS="-Dsecurity.audit.logger=INFO,DRFAS -Dhdfs.audit.logger=INFO,DRFAAUDIT $HADOOP_SECONDARYNAMENODE_OPTS"
# The following applies to multiple commands (fs, dfs, fsck, distcp etc)
export HADOOP_CLIENT_OPTS="-Xmx128m ${HADOOP_CLIENT_OPTS}"
#HADOOP_JAVA_PLATFORM_OPTS="-XX:-UsePerfData ${HADOOP_JAVA_PLATFORM_OPTS}"
export HADOOP_CLIENT_OPTS="-Xmx128m $HADOOP_CLIENT_OPTS"
#HADOOP_JAVA_PLATFORM_OPTS="-XX:-UsePerfData $HADOOP_JAVA_PLATFORM_OPTS"
# On secure datanodes, user to run the datanode as after dropping privileges
export HADOOP_SECURE_DN_USER=${HADOOP_HDFS_USER}
export HADOOP_SECURE_DN_USER=${HADOOP_SECURE_DN_USER}
# Where log files are stored. $HADOOP_HOME/logs by default.
export HADOOP_LOG_DIR=${HADOOP_LOG_DIR}/$USER

View File

@ -263,4 +263,14 @@
excluded.
</description>
</property>
<property>
<name>dfs.webhdfs.enabled</name>
<value>${DFS_WEBHDFS_ENABLED}</value>
<description>Enable or disable webhdfs. Defaults to false</description>
</property>
<property>
<name>dfs.support.append</name>
<value>${DFS_SUPPORT_APPEND}</value>
<description>Enable or disable append. Defaults to false</description>
</property>
</configuration>

View File

@ -21,15 +21,6 @@ hadoop.root.logger=INFO,console
hadoop.log.dir=.
hadoop.log.file=hadoop.log
#
# Job Summary Appender
#
# Use following logger to send summary to separate file defined by
# hadoop.mapreduce.jobsummary.log.file rolled daily:
# hadoop.mapreduce.jobsummary.logger=INFO,JSA
#
hadoop.mapreduce.jobsummary.logger=${hadoop.root.logger}
hadoop.mapreduce.jobsummary.log.file=hadoop-mapreduce.jobsummary.log
# Define the root logger to the system property "hadoop.root.logger".
log4j.rootLogger=${hadoop.root.logger}, EventCounter
@ -90,19 +81,21 @@ log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
#
#Security appender
#
security.audit.logger=INFO,console
hadoop.security.log.file=SecurityAuth.audit
log4j.appender.DRFAS=org.apache.log4j.DailyRollingFileAppender
log4j.appender.DRFAS.File=${hadoop.log.dir}/${hadoop.security.log.file}
log4j.appender.DRFAS.layout=org.apache.log4j.PatternLayout
log4j.appender.DRFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
log4j.appender.DRFAS.DatePattern=.yyyy-MM-dd
#new logger
# Define some default values that can be overridden by system properties
hadoop.security.logger=INFO,console
log4j.category.SecurityLogger=${hadoop.security.logger}
#
# hdfs audit logging
#
hdfs.audit.logger=INFO,console
log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=${hdfs.audit.logger}
log4j.additivity.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=false
@ -112,8 +105,9 @@ log4j.appender.DRFAAUDIT.layout=org.apache.log4j.PatternLayout
log4j.appender.DRFAAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
log4j.appender.DRFAAUDIT.DatePattern=.yyyy-MM-dd
#
# mapred audit logging
#
mapred.audit.logger=INFO,console
log4j.logger.org.apache.hadoop.mapred.AuditLogger=${mapred.audit.logger}
log4j.additivity.org.apache.hadoop.mapred.AuditLogger=false
@ -138,11 +132,6 @@ log4j.appender.MRAUDIT.DatePattern=.yyyy-MM-dd
#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} - %m%n
#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
#
# FSNamesystem Audit logging
# All audit events are logged at INFO level
#
log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=WARN
# Custom Logging levels
@ -162,6 +151,12 @@ log4j.appender.EventCounter=org.apache.hadoop.log.metrics.EventCounter
#
# Job Summary Appender
#
# Use following logger to send summary to separate file defined by
# hadoop.mapreduce.jobsummary.log.file rolled daily:
# hadoop.mapreduce.jobsummary.logger=INFO,JSA
#
hadoop.mapreduce.jobsummary.logger=${hadoop.root.logger}
hadoop.mapreduce.jobsummary.log.file=hadoop-mapreduce.jobsummary.log
log4j.appender.JSA=org.apache.log4j.DailyRollingFileAppender
log4j.appender.JSA.File=${hadoop.log.dir}/${hadoop.mapreduce.jobsummary.log.file}
log4j.appender.JSA.layout=org.apache.log4j.PatternLayout
@ -170,26 +165,6 @@ log4j.appender.JSA.DatePattern=.yyyy-MM-dd
log4j.logger.org.apache.hadoop.mapred.JobInProgress$JobSummary=${hadoop.mapreduce.jobsummary.logger}
log4j.additivity.org.apache.hadoop.mapred.JobInProgress$JobSummary=false
#
# MapReduce Audit Log Appender
#
# Set the MapReduce audit log filename
#hadoop.mapreduce.audit.log.file=hadoop-mapreduce.audit.log
# Appender for AuditLogger.
# Requires the following system properties to be set
# - hadoop.log.dir (Hadoop Log directory)
# - hadoop.mapreduce.audit.log.file (MapReduce audit log filename)
#log4j.logger.org.apache.hadoop.mapred.AuditLogger=INFO,MRAUDIT
#log4j.additivity.org.apache.hadoop.mapred.AuditLogger=false
#log4j.appender.MRAUDIT=org.apache.log4j.DailyRollingFileAppender
#log4j.appender.MRAUDIT.File=${hadoop.log.dir}/${hadoop.mapreduce.audit.log.file}
#log4j.appender.MRAUDIT.DatePattern=.yyyy-MM-dd
#log4j.appender.MRAUDIT.layout=org.apache.log4j.PatternLayout
#log4j.appender.MRAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
#
# Yarn ResourceManager Application Summary Log
#

View File

@ -58,7 +58,7 @@ protected void tearDown() throws Exception {
}
private void startConfig() throws IOException{
out.write("<?xml version=\"1.1\"?>\n");
out.write("<?xml version=\"1.0\"?>\n");
out.write("<configuration>\n");
}
@ -221,18 +221,6 @@ public void testCommentsInValue() throws IOException {
assertEquals("this contains a comment", conf.get("my.comment"));
}
public void testControlAInValue() throws IOException {
out = new BufferedWriter(new FileWriter(CONFIG));
startConfig();
appendProperty("my.char", "&#1;");
appendProperty("my.string", "some&#1;string");
endConfig();
Path fileResource = new Path(CONFIG);
conf.addResource(fileResource);
assertEquals("\u0001", conf.get("my.char"));
assertEquals("some\u0001string", conf.get("my.string"));
}
public void testTrim() throws IOException {
out=new BufferedWriter(new FileWriter(CONFIG));
startConfig();
@ -310,7 +298,7 @@ public void testWriteXml() throws IOException {
conf.writeXml(baos);
String result = baos.toString();
assertTrue("Result has proper header", result.startsWith(
"<?xml version=\"1.1\" encoding=\"UTF-8\" standalone=\"no\"?><configuration>"));
"<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?><configuration>"));
assertTrue("Result has proper footer", result.endsWith("</configuration>"));
}

View File

@ -56,17 +56,15 @@ private void assertNotLive(HttpServer server) {
*
* @throws Throwable on failure
*/
@Test public void testStartedServerIsAlive() throws Throwable {
@Test
public void testStartedServerIsAlive() throws Throwable {
HttpServer server = null;
try {
server = createTestServer();
assertNotLive(server);
server.start();
assertAlive(server);
} finally {
stop(server);
}
}
/**
* Assert that the result of {@link HttpServer#toString()} contains the specific text
@ -105,4 +103,24 @@ private void assertToStringContains(HttpServer server, String text) {
assertNotLive(server);
}
/**
* Test that the server is alive once started
*
* @throws Throwable
* on failure
*/
@Test
public void testWepAppContextAfterServerStop() throws Throwable {
HttpServer server = null;
String key = "test.attribute.key";
String value = "test.attribute.value";
server = createTestServer();
assertNotLive(server);
server.start();
server.setAttribute(key, value);
assertAlive(server);
assertEquals(value, server.getAttribute(key));
stop(server);
assertNull("Server context should have cleared", server.getAttribute(key));
}
}

View File

@ -20,6 +20,7 @@
import static org.junit.Assert.*;
import java.io.IOException;
import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.io.retry.UnreliableImplementation.TypeOfExceptionToFailWith;
import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
@ -35,22 +36,41 @@ public static class FlipFlopProxyProvider implements FailoverProxyProvider {
private Object impl1;
private Object impl2;
private boolean latchEnabled = false;
private CountDownLatch getProxyLatch;
private int failoversOccurred = 0;
public FlipFlopProxyProvider(Class<?> iface, Object activeImpl,
Object standbyImpl) {
Object standbyImpl, int getProxyCountDown) {
this.iface = iface;
this.impl1 = activeImpl;
this.impl2 = standbyImpl;
currentlyActive = impl1;
getProxyLatch = new CountDownLatch(getProxyCountDown);
}
public FlipFlopProxyProvider(Class<?> iface, Object activeImpl,
Object standbyImpl) {
this(iface, activeImpl, standbyImpl, 0);
}
@Override
public Object getProxy() {
if (latchEnabled) {
getProxyLatch.countDown();
try {
getProxyLatch.await();
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
return currentlyActive;
}
@Override
public void performFailover(Object currentProxy) {
public synchronized void performFailover(Object currentProxy) {
currentlyActive = impl1 == currentProxy ? impl2 : impl1;
failoversOccurred++;
}
@Override
@ -63,6 +83,13 @@ public void close() throws IOException {
// Nothing to do.
}
public void setLatchEnabled(boolean latchEnabled) {
this.latchEnabled = latchEnabled;
}
public int getFailoversOccurred() {
return failoversOccurred;
}
}
public static class FailOverOnceOnAnyExceptionPolicy implements RetryPolicy {
@ -186,4 +213,55 @@ public void testFailoverOnNetworkExceptionIdempotentOperation()
// IOException and this method is idempotent.
assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningStringIdempotent());
}
private static class ConcurrentMethodThread extends Thread {
private UnreliableInterface unreliable;
public String result;
public ConcurrentMethodThread(UnreliableInterface unreliable) {
this.unreliable = unreliable;
}
public void run() {
try {
result = unreliable.failsIfIdentifierDoesntMatch("impl2");
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}
/**
* Test that concurrent failed method invocations only result in a single
* failover.
*/
@Test
public void testConcurrentMethodFailures() throws InterruptedException {
FlipFlopProxyProvider proxyProvider = new FlipFlopProxyProvider(
UnreliableInterface.class,
new UnreliableImplementation("impl1",
TypeOfExceptionToFailWith.STANDBY_EXCEPTION),
new UnreliableImplementation("impl2",
TypeOfExceptionToFailWith.STANDBY_EXCEPTION),
2);
final UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
.create(UnreliableInterface.class, proxyProvider,
RetryPolicies.failoverOnNetworkException(10));
ConcurrentMethodThread t1 = new ConcurrentMethodThread(unreliable);
ConcurrentMethodThread t2 = new ConcurrentMethodThread(unreliable);
// Getting a proxy will now wait on a latch.
proxyProvider.setLatchEnabled(true);
t1.start();
t2.start();
t1.join();
t2.join();
assertEquals("impl2", t1.result);
assertEquals("impl2", t2.result);
assertEquals(1, proxyProvider.getFailoversOccurred());
}
}

View File

@ -141,4 +141,23 @@ public String succeedsOnceThenFailsReturningStringIdempotent()
}
}
@Override
public String failsIfIdentifierDoesntMatch(String identifier)
throws UnreliableException, StandbyException, IOException {
if (this.identifier.equals(identifier)) {
return identifier;
} else {
switch (exceptionToFailWith) {
case STANDBY_EXCEPTION:
throw new StandbyException(identifier);
case UNRELIABLE_EXCEPTION:
throw new UnreliableException(identifier);
case IO_EXCEPTION:
throw new IOException(identifier);
default:
throw new RuntimeException(identifier);
}
}
}
}

View File

@ -63,4 +63,8 @@ public String succeedsOnceThenFailsReturningStringIdempotent()
throws UnreliableException, StandbyException, IOException;
public String succeedsTenTimesThenFailsReturningString()
throws UnreliableException, StandbyException, IOException;
@Idempotent
public String failsIfIdentifierDoesntMatch(String identifier)
throws UnreliableException, StandbyException, IOException;
}

View File

@ -43,6 +43,7 @@
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.fs.CommonConfigurationKeys;
/** Unit tests for AvroRpc. */
public class TestAvroRpc extends TestCase {
@ -56,6 +57,9 @@ public class TestAvroRpc extends TestCase {
public TestAvroRpc(String name) { super(name); }
public static interface EmptyProtocol {}
public static class EmptyImpl implements EmptyProtocol {}
public static class TestImpl implements AvroTestProtocol {
public void ping() {}
@ -93,10 +97,12 @@ private void testReflect(boolean secure) throws Exception {
sm = new TestTokenSecretManager();
}
UserGroupInformation.setConfiguration(conf);
RPC.setProtocolEngine(conf, EmptyProtocol.class, AvroRpcEngine.class);
RPC.setProtocolEngine(conf, AvroTestProtocol.class, AvroRpcEngine.class);
Server server = RPC.getServer(AvroTestProtocol.class,
new TestImpl(), ADDRESS, 0, 5, true,
conf, sm);
RPC.Server server = RPC.getServer(EmptyProtocol.class, new EmptyImpl(),
ADDRESS, 0, 5, true, conf, sm);
server.addProtocol(AvroTestProtocol.class, new TestImpl());
try {
server.start();
InetSocketAddress addr = NetUtils.getConnectAddress(server);

View File

@ -23,7 +23,6 @@
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.net.NetUtils;
@ -270,7 +269,7 @@ public void testStandAloneClient() throws Exception {
fail("Expected an exception to have been thrown");
} catch (IOException e) {
String message = e.getMessage();
String addressText = address.toString();
String addressText = address.getHostName() + ":" + address.getPort();
assertTrue("Did not find "+addressText+" in "+message,
message.contains(addressText));
Throwable cause=e.getCause();

View File

@ -17,10 +17,15 @@
*/
package org.apache.hadoop.net;
import junit.framework.AssertionFailedError;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.junit.Test;
import static org.junit.Assert.*;
import java.io.IOException;
import java.net.BindException;
import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.Socket;
@ -34,6 +39,12 @@
public class TestNetUtils {
private static final Log LOG = LogFactory.getLog(TestNetUtils.class);
private static final int DEST_PORT = 4040;
private static final String DEST_PORT_NAME = Integer.toString(DEST_PORT);
private static final int LOCAL_PORT = 8080;
private static final String LOCAL_PORT_NAME = Integer.toString(LOCAL_PORT);
/**
* Test that we can't accidentally connect back to the connecting socket due
* to a quirk in the TCP spec.
@ -120,4 +131,100 @@ public void testIsLocalAddress() throws Exception {
}
assertFalse(NetUtils.isLocalAddress(InetAddress.getByName("8.8.8.8")));
}
@Test
public void testWrapConnectException() throws Throwable {
IOException e = new ConnectException("failed");
IOException wrapped = verifyExceptionClass(e, ConnectException.class);
assertInException(wrapped, "failed");
assertWikified(wrapped);
assertInException(wrapped, "localhost");
assertRemoteDetailsIncluded(wrapped);
assertInException(wrapped, "/ConnectionRefused");
}
@Test
public void testWrapBindException() throws Throwable {
IOException e = new BindException("failed");
IOException wrapped = verifyExceptionClass(e, BindException.class);
assertInException(wrapped, "failed");
assertLocalDetailsIncluded(wrapped);
assertNotInException(wrapped, DEST_PORT_NAME);
assertInException(wrapped, "/BindException");
}
@Test
public void testWrapUnknownHostException() throws Throwable {
IOException e = new UnknownHostException("failed");
IOException wrapped = verifyExceptionClass(e, UnknownHostException.class);
assertInException(wrapped, "failed");
assertWikified(wrapped);
assertInException(wrapped, "localhost");
assertRemoteDetailsIncluded(wrapped);
assertInException(wrapped, "/UnknownHost");
}
private void assertRemoteDetailsIncluded(IOException wrapped)
throws Throwable {
assertInException(wrapped, "desthost");
assertInException(wrapped, DEST_PORT_NAME);
}
private void assertLocalDetailsIncluded(IOException wrapped)
throws Throwable {
assertInException(wrapped, "localhost");
assertInException(wrapped, LOCAL_PORT_NAME);
}
private void assertWikified(Exception e) throws Throwable {
assertInException(e, NetUtils.HADOOP_WIKI);
}
private void assertInException(Exception e, String text) throws Throwable {
String message = extractExceptionMessage(e);
if (!(message.contains(text))) {
throw new AssertionFailedError("Wrong text in message "
+ "\"" + message + "\""
+ " expected \"" + text + "\"")
.initCause(e);
}
}
private String extractExceptionMessage(Exception e) throws Throwable {
assertNotNull("Null Exception", e);
String message = e.getMessage();
if (message == null) {
throw new AssertionFailedError("Empty text in exception " + e)
.initCause(e);
}
return message;
}
private void assertNotInException(Exception e, String text)
throws Throwable{
String message = extractExceptionMessage(e);
if (message.contains(text)) {
throw new AssertionFailedError("Wrong text in message "
+ "\"" + message + "\""
+ " did not expect \"" + text + "\"")
.initCause(e);
}
}
private IOException verifyExceptionClass(IOException e,
Class expectedClass)
throws Throwable {
assertNotNull("Null Exception", e);
IOException wrapped =
NetUtils.wrapException("desthost", DEST_PORT,
"localhost", LOCAL_PORT,
e);
LOG.info(wrapped.toString(), wrapped);
if(!(wrapped.getClass().equals(expectedClass))) {
throw new AssertionFailedError("Wrong exception class; expected "
+ expectedClass
+ " got " + wrapped.getClass() + ": " + wrapped).initCause(wrapped);
}
return wrapped;
}
}

View File

@ -19,9 +19,13 @@ Trunk (unreleased changes)
HDFS-2340. Support getFileBlockLocations and getDelegationToken in webhdfs.
(szetszwo)
HDFS-2348. Support getContentSummary and getFileChecksum in webhdfs.
(szetszwo)
IMPROVEMENTS
HADOOP-7524 Change RPC to allow multiple protocols including multuple versions of the same protocol (sanjay Radia)
HADOOP-7524 Change RPC to allow multiple protocols including multuple
versions of the same protocol (sanjay Radia)
HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
HdfsConstants. (Harsh J Chouraria via atm)
@ -50,6 +54,13 @@ Trunk (unreleased changes)
HDFS-2355. Federation: enable using the same configuration file across
all the nodes in the cluster. (suresh)
HDFS-2371. Refactor BlockSender.java for better readability. (suresh)
HDFS-2158. Add JournalSet to manage the set of journals. (jitendra)
HDFS-2395. Add a root element in the JSON responses of webhdfs.
(szetszwo)
BUG FIXES
HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
@ -83,6 +94,9 @@ Trunk (unreleased changes)
HDFS-2361. hftp is broken, fixed username checks in JspHelper. (jitendra)
HDFS-2298. Fix TestDfsOverAvroRpc by changing ClientProtocol to
not include multiple methods of the same name. (cutting)
Release 0.23.0 - Unreleased
INCOMPATIBLE CHANGES
@ -1145,6 +1159,39 @@ Release 0.23.0 - Unreleased
(todd)
HDFS-2027. Image inspector should return finalized logs before unfinalized
logs. (todd)
HDFS-2074. Determine edit log validity by truly reading and validating
transactions. (todd)
HDFS-2085. Finalize in-progress edit logs at startup. (todd)
HDFS-2026. SecondaryNameNode should properly handle the case where the
NameNode is reformatted. (todd)
HDFS-2077. Address checkpoint upload when one of the storage dirs is failed
(todd)
HDFS-2078. NameNode should not clear directory when restoring removed
storage. (todd)
HDFS-2088. Move edits log archiving logic into FSEditLog/JournalManager
(todd)
HDFS-2093. Handle case where an entirely empty log is left during NN crash
(todd)
HDFS-2102. Zero-pad edits filename to make them lexically sortable. (Ivan
Kelly via todd)
HDFS-2010. Fix NameNode to exit if all edit streams become inaccessible.
(atm via todd)
HDFS-2123. Checkpoint interval should be based on txn count, not size.
(todd)
HDFS-1979. Fix backupnode for new edits/image layout. (todd)
HDFS-2101. Fix remaining unit tests for new storage filenames. (todd)
HDFS-2133. Address remaining TODOs and pre-merge cleanup on HDFS-1073
branch. (todd)
HDFS-1780. Reduce need to rewrite FSImage on startup. (todd)
HDFS-2104. Add a flag to the 2NN to format its checkpoint dirs on startup.
(todd)
HDFS-2135. Fix regression of HDFS-1955 in HDFS-1073 branch. (todd)
HDFS-2160. Fix CreateEditsLog test tool in HDFS-1073 branch. (todd)
HDFS-2168. Reenable TestEditLog.testFailedOpen and fix exposed bug. (todd)
HDFS-2169. Clean up TestCheckpoint and remove TODOs (todd)
HDFS-2170. Address remaining TODOs in HDFS-1073 branch. (todd)
HDFS-2172. Address findbugs and javadoc warnings in HDFS-1073 branch.
(todd)
Release 0.22.0 - Unreleased

View File

@ -874,13 +874,13 @@ public void concat(String trg, String [] srcs) throws IOException {
}
/**
* Rename file or directory.
* @see ClientProtocol#rename(String, String, Options.Rename...)
* @see ClientProtocol#rename2(String, String, Options.Rename...)
*/
public void rename(String src, String dst, Options.Rename... options)
throws IOException {
checkOpen();
try {
namenode.rename(src, dst, options);
namenode.rename2(src, dst, options);
} catch(RemoteException re) {
throw re.unwrapRemoteException(AccessControlException.class,
DSQuotaExceededException.class,

View File

@ -67,9 +67,9 @@ public interface ClientProtocol extends VersionedProtocol {
* Compared to the previous version the following changes have been introduced:
* (Only the latest change is reflected.
* The log of historical changes can be retrieved from the svn).
* 68: Add Balancer Bandwidth Command protocol
* 69: Eliminate overloaded method names.
*/
public static final long versionID = 68L;
public static final long versionID = 69L;
///////////////////////////////////////
// File contents
@ -419,27 +419,12 @@ public void concat(String trg, String[] srcs)
* <code>dst</code> contains a symlink
* @throws IOException If an I/O error occurred
*/
public void rename(String src, String dst, Options.Rename... options)
public void rename2(String src, String dst, Options.Rename... options)
throws AccessControlException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException;
/**
* Delete the given file or directory from the file system.
* <p>
* Any blocks belonging to the deleted files will be garbage-collected.
*
* @param src existing name.
* @return true only if the existing file or directory was actually removed
* from the file system.
* @throws UnresolvedLinkException if <code>src</code> contains a symlink.
* @deprecated use {@link #delete(String, boolean)} istead.
*/
@Deprecated
public boolean delete(String src)
throws IOException, UnresolvedLinkException;
/**
* Delete the given file or directory from the file system.
* <p>

View File

@ -404,7 +404,7 @@ private void verifyBlock(ExtendedBlock block) {
adjustThrottler();
blockSender = new BlockSender(block, 0, -1, false, false, true,
datanode);
datanode, null);
DataOutputStream out =
new DataOutputStream(new IOUtils.NullOutputStream());

View File

@ -41,191 +41,230 @@
/**
* Reads a block from the disk and sends it to a recipient.
*
* Data sent from the BlockeSender in the following format:
* <br><b>Data format:</b> <pre>
* +--------------------------------------------------+
* | ChecksumHeader | Sequence of data PACKETS... |
* +--------------------------------------------------+
* </pre>
* <b>ChecksumHeader format:</b> <pre>
* +--------------------------------------------------+
* | 1 byte CHECKSUM_TYPE | 4 byte BYTES_PER_CHECKSUM |
* +--------------------------------------------------+
* </pre>
* An empty packet is sent to mark the end of block and read completion.
*
* PACKET Contains a packet header, checksum and data. Amount of data
* carried is set by BUFFER_SIZE.
* <pre>
* +-----------------------------------------------------+
* | 4 byte packet length (excluding packet header) |
* +-----------------------------------------------------+
* | 8 byte offset in the block | 8 byte sequence number |
* +-----------------------------------------------------+
* | 1 byte isLastPacketInBlock |
* +-----------------------------------------------------+
* | 4 byte Length of actual data |
* +-----------------------------------------------------+
* | x byte checksum data. x is defined below |
* +-----------------------------------------------------+
* | actual data ...... |
* +-----------------------------------------------------+
*
* Data is made of Chunks. Each chunk is of length <= BYTES_PER_CHECKSUM.
* A checksum is calculated for each chunk.
*
* x = (length of data + BYTE_PER_CHECKSUM - 1)/BYTES_PER_CHECKSUM *
* CHECKSUM_SIZE
*
* CHECKSUM_SIZE depends on CHECKSUM_TYPE (usually, 4 for CRC32)
* </pre>
*
* The client reads data until it receives a packet with
* "LastPacketInBlock" set to true or with a zero length. If there is
* no checksum error, it replies to DataNode with OP_STATUS_CHECKSUM_OK:
* <pre>
* +------------------------------+
* | 2 byte OP_STATUS_CHECKSUM_OK |
* +------------------------------+
* </pre>
*/
class BlockSender implements java.io.Closeable {
public static final Log LOG = DataNode.LOG;
static final Log LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog;
private ExtendedBlock block; // the block to read from
/** the replica to read from */
private final Replica replica;
/** The visible length of a replica. */
private final long replicaVisibleLength;
private static final boolean is32Bit = System.getProperty("sun.arch.data.model").equals("32");
private InputStream blockIn; // data stream
private long blockInPosition = -1; // updated while using transferTo().
private DataInputStream checksumIn; // checksum datastream
private DataChecksum checksum; // checksum stream
private long offset; // starting position to read
private long endOffset; // ending position
private int bytesPerChecksum; // chunk size
private int checksumSize; // checksum size
private boolean corruptChecksumOk; // if need to verify checksum
private boolean chunkOffsetOK; // if need to send chunk offset
private long seqno; // sequence number of packet
private boolean transferToAllowed = true;
// set once entire requested byte range has been sent to the client
private boolean sentEntireByteRange;
private boolean verifyChecksum; //if true, check is verified while reading
private DataTransferThrottler throttler;
private final String clientTraceFmt; // format of client trace log message
private static final boolean is32Bit =
System.getProperty("sun.arch.data.model").equals("32");
/**
* Minimum buffer used while sending data to clients. Used only if
* transferTo() is enabled. 64KB is not that large. It could be larger, but
* not sure if there will be much more improvement.
*/
private static final int MIN_BUFFER_WITH_TRANSFERTO = 64*1024;
private static final int TRANSFERTO_BUFFER_SIZE = Math.max(
HdfsConstants.IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
/** the block to read from */
private final ExtendedBlock block;
/** the replica to read from */
private final Replica replica;
/** The visible length of a replica. */
private final long replicaVisibleLength;
/** Stream to read block data from */
private InputStream blockIn;
/** updated while using transferTo() */
private long blockInPosition = -1;
/** Stream to read checksum */
private DataInputStream checksumIn;
/** Checksum utility */
private final DataChecksum checksum;
/** Starting position to read */
private long offset;
/** Position of last byte to read from block file */
private final long endOffset;
/** Number of bytes in chunk used for computing checksum */
private final int chunkSize;
/** Number bytes of checksum computed for a chunk */
private final int checksumSize;
/** If true, failure to read checksum is ignored */
private final boolean corruptChecksumOk;
/** true if chunk offset is needed to be sent in Checksum header */
private final boolean chunkOffsetOK;
/** Sequence number of packet being sent */
private long seqno;
/** Set to true if transferTo is allowed for sending data to the client */
private final boolean transferToAllowed;
/** Set to true once entire requested byte range has been sent to the client */
private boolean sentEntireByteRange;
/** When true, verify checksum while reading from checksum file */
private final boolean verifyChecksum;
/** Format used to print client trace log messages */
private final String clientTraceFmt;
private volatile ChunkChecksum lastChunkChecksum = null;
BlockSender(ExtendedBlock block, long startOffset, long length,
boolean corruptChecksumOk, boolean chunkOffsetOK,
boolean verifyChecksum, DataNode datanode) throws IOException {
this(block, startOffset, length, corruptChecksumOk, chunkOffsetOK,
verifyChecksum, datanode, null);
}
/**
* Constructor
*
* @param block Block that is being read
* @param startOffset starting offset to read from
* @param length length of data to read
* @param corruptChecksumOk
* @param chunkOffsetOK need to send check offset in checksum header
* @param verifyChecksum verify checksum while reading the data
* @param datanode datanode from which the block is being read
* @param clientTraceFmt format string used to print client trace logs
* @throws IOException
*/
BlockSender(ExtendedBlock block, long startOffset, long length,
boolean corruptChecksumOk, boolean chunkOffsetOK,
boolean verifyChecksum, DataNode datanode, String clientTraceFmt)
throws IOException {
try {
this.block = block;
this.chunkOffsetOK = chunkOffsetOK;
this.corruptChecksumOk = corruptChecksumOk;
this.verifyChecksum = verifyChecksum;
this.clientTraceFmt = clientTraceFmt;
synchronized(datanode.data) {
this.replica = datanode.data.getReplica(block.getBlockPoolId(),
block.getBlockId());
if (replica == null) {
throw new ReplicaNotFoundException(block);
}
this.replica = getReplica(block, datanode);
this.replicaVisibleLength = replica.getVisibleLength();
}
long minEndOffset = startOffset + length;
// if this is a write in progress
// if there is a write in progress
ChunkChecksum chunkChecksum = null;
if (replica instanceof ReplicaBeingWritten) {
for (int i = 0; i < 30 && replica.getBytesOnDisk() < minEndOffset; i++) {
try {
Thread.sleep(100);
} catch (InterruptedException ie) {
throw new IOException(ie);
}
}
long currentBytesOnDisk = replica.getBytesOnDisk();
if (currentBytesOnDisk < minEndOffset) {
throw new IOException(String.format(
"need %d bytes, but only %d bytes available",
minEndOffset,
currentBytesOnDisk
));
}
long minEndOffset = startOffset + length;
waitForMinLength((ReplicaBeingWritten)replica, minEndOffset);
ReplicaInPipeline rip = (ReplicaInPipeline) replica;
chunkChecksum = rip.getLastChecksumAndDataLen();
}
if (replica.getGenerationStamp() < block.getGenerationStamp()) {
throw new IOException(
"replica.getGenerationStamp() < block.getGenerationStamp(), block="
throw new IOException("Replica gen stamp < block genstamp, block="
+ block + ", replica=" + replica);
}
if (replicaVisibleLength < 0) {
throw new IOException("The replica is not readable, block="
throw new IOException("Replica is not readable, block="
+ block + ", replica=" + replica);
}
if (DataNode.LOG.isDebugEnabled()) {
DataNode.LOG.debug("block=" + block + ", replica=" + replica);
}
this.chunkOffsetOK = chunkOffsetOK;
this.corruptChecksumOk = corruptChecksumOk;
this.verifyChecksum = verifyChecksum;
// transferToFully() fails on 32 bit platforms for block sizes >= 2GB,
// use normal transfer in those cases
this.transferToAllowed = datanode.transferToAllowed &&
(!is32Bit || length < (long) Integer.MAX_VALUE);
this.clientTraceFmt = clientTraceFmt;
(!is32Bit || length <= Integer.MAX_VALUE);
if ( !corruptChecksumOk || datanode.data.metaFileExists(block) ) {
DataChecksum csum;
if (!corruptChecksumOk || datanode.data.metaFileExists(block)) {
checksumIn = new DataInputStream(new BufferedInputStream(datanode.data
.getMetaDataInputStream(block), HdfsConstants.IO_FILE_BUFFER_SIZE));
// read and handle the common header here. For now just a version
BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
short version = header.getVersion();
if (version != FSDataset.METADATA_VERSION) {
LOG.warn("Wrong version (" + version + ") for metadata file for "
+ block + " ignoring ...");
}
checksum = header.getChecksum();
csum = header.getChecksum();
} else {
LOG.warn("Could not find metadata file for " + block);
// This only decides the buffer size. Use BUFFER_SIZE?
checksum = DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_NULL,
csum = DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_NULL,
16 * 1024);
}
/* If bytesPerChecksum is very large, then the metadata file
* is mostly corrupted. For now just truncate bytesPerchecksum to
* blockLength.
/*
* If chunkSize is very large, then the metadata file is mostly
* corrupted. For now just truncate bytesPerchecksum to blockLength.
*/
bytesPerChecksum = checksum.getBytesPerChecksum();
if (bytesPerChecksum > 10*1024*1024 && bytesPerChecksum > replicaVisibleLength) {
checksum = DataChecksum.newDataChecksum(checksum.getChecksumType(),
int size = csum.getBytesPerChecksum();
if (size > 10*1024*1024 && size > replicaVisibleLength) {
csum = DataChecksum.newDataChecksum(csum.getChecksumType(),
Math.max((int)replicaVisibleLength, 10*1024*1024));
bytesPerChecksum = checksum.getBytesPerChecksum();
size = csum.getBytesPerChecksum();
}
chunkSize = size;
checksum = csum;
checksumSize = checksum.getChecksumSize();
if (length < 0) {
length = replicaVisibleLength;
}
length = length < 0 ? replicaVisibleLength : length;
// end is either last byte on disk or the length for which we have a
// checksum
if (chunkChecksum != null) {
endOffset = chunkChecksum.getDataLength();
} else {
endOffset = replica.getBytesOnDisk();
}
if (startOffset < 0 || startOffset > endOffset
|| (length + startOffset) > endOffset) {
long end = chunkChecksum != null ? chunkChecksum.getDataLength()
: replica.getBytesOnDisk();
if (startOffset < 0 || startOffset > end
|| (length + startOffset) > end) {
String msg = " Offset " + startOffset + " and length " + length
+ " don't match block " + block + " ( blockLen " + endOffset + " )";
+ " don't match block " + block + " ( blockLen " + end + " )";
LOG.warn(datanode.getDNRegistrationForBP(block.getBlockPoolId()) +
":sendBlock() : " + msg);
throw new IOException(msg);
}
offset = (startOffset - (startOffset % bytesPerChecksum));
// Ensure read offset is position at the beginning of chunk
offset = startOffset - (startOffset % chunkSize);
if (length >= 0) {
// Make sure endOffset points to end of a checksumed chunk.
// Ensure endOffset points to end of chunk.
long tmpLen = startOffset + length;
if (tmpLen % bytesPerChecksum != 0) {
tmpLen += (bytesPerChecksum - tmpLen % bytesPerChecksum);
if (tmpLen % chunkSize != 0) {
tmpLen += (chunkSize - tmpLen % chunkSize);
}
if (tmpLen < endOffset) {
if (tmpLen < end) {
// will use on-disk checksum here since the end is a stable chunk
endOffset = tmpLen;
end = tmpLen;
} else if (chunkChecksum != null) {
//in last chunk which is changing. flag that we need to use in-memory
// checksum
// last chunk is changing. flag that we need to use in-memory checksum
this.lastChunkChecksum = chunkChecksum;
}
}
endOffset = end;
// seek to the right offsets
if (offset > 0) {
long checksumSkip = (offset / bytesPerChecksum) * checksumSize;
long checksumSkip = (offset / chunkSize) * checksumSize;
// note blockInStream is seeked when created below
if (checksumSkip > 0) {
// Should we use seek() for checksum file as well?
@ -237,7 +276,6 @@ class BlockSender implements java.io.Closeable {
if (DataNode.LOG.isDebugEnabled()) {
DataNode.LOG.debug("replica=" + replica);
}
blockIn = datanode.data.getBlockInputStream(block, offset); // seek to offset
} catch (IOException ioe) {
IOUtils.closeStream(this);
@ -251,19 +289,17 @@ class BlockSender implements java.io.Closeable {
*/
public void close() throws IOException {
IOException ioe = null;
// close checksum file
if(checksumIn!=null) {
try {
checksumIn.close();
checksumIn.close(); // close checksum file
} catch (IOException e) {
ioe = e;
}
checksumIn = null;
}
// close data file
if(blockIn!=null) {
try {
blockIn.close();
blockIn.close(); // close data file
} catch (IOException e) {
ioe = e;
}
@ -275,6 +311,40 @@ public void close() throws IOException {
}
}
private static Replica getReplica(ExtendedBlock block, DataNode datanode)
throws ReplicaNotFoundException {
Replica replica = datanode.data.getReplica(block.getBlockPoolId(),
block.getBlockId());
if (replica == null) {
throw new ReplicaNotFoundException(block);
}
return replica;
}
/**
* Wait for rbw replica to reach the length
* @param rbw replica that is being written to
* @param len minimum length to reach
* @throws IOException on failing to reach the len in given wait time
*/
private static void waitForMinLength(ReplicaBeingWritten rbw, long len)
throws IOException {
// Wait for 3 seconds for rbw replica to reach the minimum length
for (int i = 0; i < 30 && rbw.getBytesOnDisk() < len; i++) {
try {
Thread.sleep(100);
} catch (InterruptedException ie) {
throw new IOException(ie);
}
}
long bytesOnDisk = rbw.getBytesOnDisk();
if (bytesOnDisk < len) {
throw new IOException(
String.format("Need %d bytes, but only %d bytes available", len,
bytesOnDisk));
}
}
/**
* Converts an IOExcpetion (not subclasses) to SocketException.
* This is typically done to indicate to upper layers that the error
@ -296,54 +366,43 @@ private static IOException ioeToSocketException(IOException ioe) {
}
/**
* Sends upto maxChunks chunks of data.
*
* When blockInPosition is >= 0, assumes 'out' is a
* {@link SocketOutputStream} and tries
* {@link SocketOutputStream#transferToFully(FileChannel, long, int)} to
* send data (and updates blockInPosition).
* @param datalen Length of data
* @return number of chunks for data of given size
*/
private int sendChunks(ByteBuffer pkt, int maxChunks, OutputStream out)
throws IOException {
// Sends multiple chunks in one packet with a single write().
private int numberOfChunks(long datalen) {
return (int) ((datalen + chunkSize - 1)/chunkSize);
}
int len = (int) Math.min(endOffset - offset,
(((long) bytesPerChecksum) * ((long) maxChunks)));
int numChunks = (len + bytesPerChecksum - 1)/bytesPerChecksum;
int packetLen = len + numChunks*checksumSize + 4;
boolean lastDataPacket = offset + len == endOffset && len > 0;
pkt.clear();
/**
* Sends a packet with up to maxChunks chunks of data.
*
* @param pkt buffer used for writing packet data
* @param maxChunks maximum number of chunks to send
* @param out stream to send data to
* @param transferTo use transferTo to send data
* @param throttler used for throttling data transfer bandwidth
*/
private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out,
boolean transferTo, DataTransferThrottler throttler) throws IOException {
int dataLen = (int) Math.min(endOffset - offset,
(chunkSize * (long) maxChunks));
int numChunks = numberOfChunks(dataLen); // Number of chunks be sent in the packet
int checksumDataLen = numChunks * checksumSize;
int packetLen = dataLen + checksumDataLen + 4;
boolean lastDataPacket = offset + dataLen == endOffset && dataLen > 0;
PacketHeader header = new PacketHeader(
packetLen, offset, seqno, (len == 0), len);
header.putInBuffer(pkt);
writePacketHeader(pkt, dataLen, packetLen);
int checksumOff = pkt.position();
int checksumLen = numChunks * checksumSize;
byte[] buf = pkt.array();
if (checksumSize > 0 && checksumIn != null) {
try {
checksumIn.readFully(buf, checksumOff, checksumLen);
} catch (IOException e) {
LOG.warn(" Could not read or failed to veirfy checksum for data"
+ " at offset " + offset + " for block " + block, e);
IOUtils.closeStream(checksumIn);
checksumIn = null;
if (corruptChecksumOk) {
if (checksumOff < checksumLen) {
// Just fill the array with zeros.
Arrays.fill(buf, checksumOff, checksumLen, (byte) 0);
}
} else {
throw e;
}
}
readChecksum(buf, checksumOff, checksumDataLen);
// write in progress that we need to use to get last checksum
if (lastDataPacket && lastChunkChecksum != null) {
int start = checksumOff + checksumLen - checksumSize;
int start = checksumOff + checksumDataLen - checksumSize;
byte[] updatedChecksum = lastChunkChecksum.getChecksum();
if (updatedChecksum != null) {
@ -352,52 +411,28 @@ private int sendChunks(ByteBuffer pkt, int maxChunks, OutputStream out)
}
}
int dataOff = checksumOff + checksumLen;
if (blockInPosition < 0) {
//normal transfer
IOUtils.readFully(blockIn, buf, dataOff, len);
int dataOff = checksumOff + checksumDataLen;
if (!transferTo) { // normal transfer
IOUtils.readFully(blockIn, buf, dataOff, dataLen);
if (verifyChecksum) {
int dOff = dataOff;
int cOff = checksumOff;
int dLeft = len;
for (int i=0; i<numChunks; i++) {
checksum.reset();
int dLen = Math.min(dLeft, bytesPerChecksum);
checksum.update(buf, dOff, dLen);
if (!checksum.compare(buf, cOff)) {
long failedPos = offset + len -dLeft;
throw new ChecksumException("Checksum failed at " +
failedPos, failedPos);
verifyChecksum(buf, dataOff, dataLen, numChunks, checksumOff);
}
dLeft -= dLen;
dOff += dLen;
cOff += checksumSize;
}
}
//writing is done below (mainly to handle IOException)
}
try {
if (blockInPosition >= 0) {
//use transferTo(). Checks on out and blockIn are already done.
if (transferTo) {
SocketOutputStream sockOut = (SocketOutputStream)out;
//first write the packet
sockOut.write(buf, 0, dataOff);
sockOut.write(buf, 0, dataOff); // First write checksum
// no need to flush. since we know out is not a buffered stream.
sockOut.transferToFully(((FileInputStream)blockIn).getChannel(),
blockInPosition, len);
blockInPosition += len;
blockInPosition, dataLen);
blockInPosition += dataLen;
} else {
// normal transfer
out.write(buf, 0, dataOff + len);
out.write(buf, 0, dataOff + dataLen);
}
} catch (IOException e) {
/* Exception while writing to the client. Connection closure from
* the other end is mostly the case and we do not care much about
@ -419,7 +454,70 @@ private int sendChunks(ByteBuffer pkt, int maxChunks, OutputStream out)
throttler.throttle(packetLen);
}
return len;
return dataLen;
}
/**
* Read checksum into given buffer
* @param buf buffer to read the checksum into
* @param checksumOffset offset at which to write the checksum into buf
* @param checksumLen length of checksum to write
* @throws IOException on error
*/
private void readChecksum(byte[] buf, final int checksumOffset,
final int checksumLen) throws IOException {
if (checksumSize <= 0 && checksumIn == null) {
return;
}
try {
checksumIn.readFully(buf, checksumOffset, checksumLen);
} catch (IOException e) {
LOG.warn(" Could not read or failed to veirfy checksum for data"
+ " at offset " + offset + " for block " + block, e);
IOUtils.closeStream(checksumIn);
checksumIn = null;
if (corruptChecksumOk) {
if (checksumOffset < checksumLen) {
// Just fill the array with zeros.
Arrays.fill(buf, checksumOffset, checksumLen, (byte) 0);
}
} else {
throw e;
}
}
}
/**
* Compute checksum for chunks and verify the checksum that is read from
* the metadata file is correct.
*
* @param buf buffer that has checksum and data
* @param dataOffset position where data is written in the buf
* @param datalen length of data
* @param numChunks number of chunks corresponding to data
* @param checksumOffset offset where checksum is written in the buf
* @throws ChecksumException on failed checksum verification
*/
public void verifyChecksum(final byte[] buf, final int dataOffset,
final int datalen, final int numChunks, final int checksumOffset)
throws ChecksumException {
int dOff = dataOffset;
int cOff = checksumOffset;
int dLeft = datalen;
for (int i = 0; i < numChunks; i++) {
checksum.reset();
int dLen = Math.min(dLeft, chunkSize);
checksum.update(buf, dOff, dLen);
if (!checksum.compare(buf, cOff)) {
long failedPos = offset + datalen - dLeft;
throw new ChecksumException("Checksum failed at " + failedPos,
failedPos);
}
dLeft -= dLen;
dOff += dLen;
cOff += checksumSize;
}
}
/**
@ -433,70 +531,54 @@ private int sendChunks(ByteBuffer pkt, int maxChunks, OutputStream out)
* {@link SocketOutputStream#transferToFully(FileChannel,
* long, int)}.
* @param throttler for sending data.
* @return total bytes reads, including crc.
* @return total bytes read, including checksum data.
*/
long sendBlock(DataOutputStream out, OutputStream baseStream,
DataTransferThrottler throttler) throws IOException {
if( out == null ) {
if (out == null) {
throw new IOException( "out stream is null" );
}
this.throttler = throttler;
long initialOffset = offset;
final long initialOffset = offset;
long totalRead = 0;
OutputStream streamForSendChunks = out;
final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
try {
try {
checksum.writeHeader(out);
if ( chunkOffsetOK ) {
out.writeLong( offset );
}
out.flush();
} catch (IOException e) { //socket error
throw ioeToSocketException(e);
}
writeChecksumHeader(out);
int maxChunksPerPacket;
int pktSize = PacketHeader.PKT_HEADER_LEN;
if (transferToAllowed && !verifyChecksum &&
baseStream instanceof SocketOutputStream &&
blockIn instanceof FileInputStream) {
boolean transferTo = transferToAllowed && !verifyChecksum
&& baseStream instanceof SocketOutputStream
&& blockIn instanceof FileInputStream;
if (transferTo) {
FileChannel fileChannel = ((FileInputStream)blockIn).getChannel();
// blockInPosition also indicates sendChunks() uses transferTo.
blockInPosition = fileChannel.position();
streamForSendChunks = baseStream;
maxChunksPerPacket = numberOfChunks(TRANSFERTO_BUFFER_SIZE);
// assure a mininum buffer size.
maxChunksPerPacket = (Math.max(HdfsConstants.IO_FILE_BUFFER_SIZE,
MIN_BUFFER_WITH_TRANSFERTO)
+ bytesPerChecksum - 1)/bytesPerChecksum;
// allocate smaller buffer while using transferTo().
// Smaller packet size to only hold checksum when doing transferTo
pktSize += checksumSize * maxChunksPerPacket;
} else {
maxChunksPerPacket = Math.max(1, (HdfsConstants.IO_FILE_BUFFER_SIZE
+ bytesPerChecksum - 1) / bytesPerChecksum);
pktSize += (bytesPerChecksum + checksumSize) * maxChunksPerPacket;
maxChunksPerPacket = Math.max(1,
numberOfChunks(HdfsConstants.IO_FILE_BUFFER_SIZE));
// Packet size includes both checksum and data
pktSize += (chunkSize + checksumSize) * maxChunksPerPacket;
}
ByteBuffer pktBuf = ByteBuffer.allocate(pktSize);
while (endOffset > offset) {
long len = sendChunks(pktBuf, maxChunksPerPacket,
streamForSendChunks);
long len = sendPacket(pktBuf, maxChunksPerPacket, streamForSendChunks,
transferTo, throttler);
offset += len;
totalRead += len + ((len + bytesPerChecksum - 1)/bytesPerChecksum*
checksumSize);
totalRead += len + (numberOfChunks(len) * checksumSize);
seqno++;
}
try {
// send an empty packet to mark the end of the block
sendChunks(pktBuf, maxChunksPerPacket, streamForSendChunks);
sendPacket(pktBuf, maxChunksPerPacket, streamForSendChunks, transferTo,
throttler);
out.flush();
} catch (IOException e) { //socket error
throw ioeToSocketException(e);
@ -506,14 +588,39 @@ long sendBlock(DataOutputStream out, OutputStream baseStream,
} finally {
if (clientTraceFmt != null) {
final long endTime = System.nanoTime();
ClientTraceLog.info(String.format(clientTraceFmt, totalRead, initialOffset, endTime - startTime));
ClientTraceLog.info(String.format(clientTraceFmt, totalRead,
initialOffset, endTime - startTime));
}
close();
}
return totalRead;
}
/**
* Write checksum header to the output stream
*/
private void writeChecksumHeader(DataOutputStream out) throws IOException {
try {
checksum.writeHeader(out);
if (chunkOffsetOK) {
out.writeLong(offset);
}
out.flush();
} catch (IOException e) { //socket error
throw ioeToSocketException(e);
}
}
/**
* Write packet header into {@code pkt}
*/
private void writePacketHeader(ByteBuffer pkt, int dataLen, int packetLen) {
pkt.clear();
PacketHeader header = new PacketHeader(packetLen, offset, seqno,
(dataLen == 0), dataLen);
header.putInBuffer(pkt);
}
boolean didSendEntireByteRange() {
return sentEntireByteRange;
}

View File

@ -2058,7 +2058,7 @@ public void run() {
out = new DataOutputStream(new BufferedOutputStream(baseStream,
HdfsConstants.SMALL_BUFFER_SIZE));
blockSender = new BlockSender(b, 0, b.getNumBytes(),
false, false, false, DataNode.this);
false, false, false, DataNode.this, null);
DatanodeInfo srcNode = new DatanodeInfo(bpReg);
//

View File

@ -597,7 +597,7 @@ public void copyBlock(final ExtendedBlock block,
try {
// check if the block exists or not
blockSender = new BlockSender(block, 0, -1, false, false, false,
datanode);
datanode, null);
// set up response stream
OutputStream baseStream = NetUtils.getOutputStream(

View File

@ -46,10 +46,12 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.web.JsonUtil;
import org.apache.hadoop.hdfs.web.ParamFilter;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
@ -219,13 +221,13 @@ public Response run() throws IOException {
final String fullpath = path.getAbsolutePath();
final DataNode datanode = (DataNode)context.getAttribute("datanode");
final Configuration conf = new Configuration(datanode.getConf());
final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
switch(op.getValue()) {
case OPEN:
{
final Configuration conf = new Configuration(datanode.getConf());
final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
final int b = bufferSize.getValue(conf);
final DFSDataInputStream in = new DFSClient.DFSDataInputStream(
dfsclient.open(fullpath, b, true));
@ -244,6 +246,12 @@ public void write(final OutputStream out) throws IOException {
};
return Response.ok(streaming).type(MediaType.APPLICATION_OCTET_STREAM).build();
}
case GETFILECHECKSUM:
{
final MD5MD5CRC32FileChecksum checksum = dfsclient.getFileChecksum(fullpath);
final String js = JsonUtil.toJsonString(checksum);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}
default:
throw new UnsupportedOperationException(op + " is not supported");
}

View File

@ -54,7 +54,6 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
this.nnRegistration = nnReg;
InetSocketAddress bnAddress =
NetUtils.createSocketAddr(bnRegistration.getAddress());
Storage.LOG.info("EditLogBackupOutputStream connects to: " + bnAddress);
try {
this.backupNode =
RPC.getProxy(JournalProtocol.class,
@ -67,16 +66,6 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
this.out = new DataOutputBuffer(DEFAULT_BUFFER_SIZE);
}
@Override // JournalStream
public String getName() {
return bnRegistration.getAddress();
}
@Override // JournalStream
public JournalType getType() {
return JournalType.BACKUP;
}
@Override // EditLogOutputStream
void write(FSEditLogOp op) throws IOException {
doubleBuf.writeOp(op);
@ -141,16 +130,6 @@ protected void flushAndSync() throws IOException {
}
}
/**
* There is no persistent storage. Therefore length is 0.<p>
* Length is used to check when it is large enough to start a checkpoint.
* This criteria should not be used for backup streams.
*/
@Override // EditLogOutputStream
long length() throws IOException {
return 0;
}
/**
* Get backup node registration.
*/

View File

@ -37,9 +37,7 @@
* stores edits in a local file.
*/
class EditLogFileOutputStream extends EditLogOutputStream {
private static Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);;
private static int EDITS_FILE_HEADER_SIZE_BYTES = Integer.SIZE / Byte.SIZE;
private static Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
private File file;
private FileOutputStream fp; // file stream for storing edit logs
@ -73,16 +71,6 @@ class EditLogFileOutputStream extends EditLogOutputStream {
fc.position(fc.size());
}
@Override // JournalStream
public String getName() {
return file.getPath();
}
@Override // JournalStream
public JournalType getType() {
return JournalType.FILE;
}
/** {@inheritDoc} */
@Override
void write(FSEditLogOp op) throws IOException {
@ -176,7 +164,10 @@ protected void flushAndSync() throws IOException {
if (fp == null) {
throw new IOException("Trying to use aborted output stream");
}
if (doubleBuf.isFlushed()) {
LOG.info("Nothing to flush");
return;
}
preallocate(); // preallocate file if necessary
doubleBuf.flushTo(fp);
fc.force(false); // metadata updates not needed because of preallocation
@ -191,16 +182,6 @@ public boolean shouldForceSync() {
return doubleBuf.shouldForceSync();
}
/**
* Return the size of the current edit log including buffered data.
*/
@Override
long length() throws IOException {
// file size - header size + size of both buffers
return fc.size() - EDITS_FILE_HEADER_SIZE_BYTES +
doubleBuf.countBufferedBytes();
}
// allocate a big chunk of data
private void preallocate() throws IOException {
long position = fc.position();

View File

@ -18,23 +18,20 @@
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.util.zip.Checksum;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Writable;
/**
* A generic abstract class to support journaling of edits logs into
* a persistent storage.
*/
abstract class EditLogOutputStream implements JournalStream {
abstract class EditLogOutputStream {
// these are statistics counters
private long numSync; // number of sync(s) to disk
private long totalTimeSync; // total time to sync
EditLogOutputStream() throws IOException {
EditLogOutputStream() {
numSync = totalTimeSync = 0;
}
@ -105,12 +102,6 @@ public void flush() throws IOException {
totalTimeSync += (end - start);
}
/**
* Return the size of the current edits log.
* Length is used to check when it is large enough to start a checkpoint.
*/
abstract long length() throws IOException;
/**
* Implement the policy when to automatically sync the buffered edits log
* The buffered edits can be flushed when the buffer becomes full or
@ -132,12 +123,7 @@ long getTotalSyncTime() {
/**
* Return number of calls to {@link #flushAndSync()}
*/
long getNumSync() {
protected long getNumSync() {
return numSync;
}
@Override // Object
public String toString() {
return getName();
}
}

View File

@ -17,12 +17,12 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.SortedSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -34,25 +34,17 @@
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.JournalManager.CorruptionException;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableListMultimap;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimaps;
import com.google.common.collect.Sets;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
/**
* FSEditLog maintains a log of the namespace modifications.
@ -62,9 +54,6 @@
@InterfaceStability.Evolving
public class FSEditLog {
static final String NO_JOURNAL_STREAMS_WARNING = "!!! WARNING !!!" +
" File system changes are not persistent. No journal streams.";
static final Log LOG = LogFactory.getLog(FSEditLog.class);
/**
@ -83,8 +72,9 @@ private enum State {
}
private State state = State.UNINITIALIZED;
private List<JournalAndStream> journals = Lists.newArrayList();
//initialize
final private JournalSet journalSet;
private EditLogOutputStream editLogStream = null;
// a monotonically increasing counter that represents transactionIds.
private long txid = 0;
@ -138,14 +128,14 @@ protected synchronized TransactionId initialValue() {
metrics = NameNode.getNameNodeMetrics();
lastPrintTime = now();
this.journalSet = new JournalSet();
for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.EDITS)) {
journals.add(new JournalAndStream(new FileJournalManager(sd)));
journalSet.add(new FileJournalManager(sd));
}
if (journals.isEmpty()) {
if (journalSet.isEmpty()) {
LOG.error("No edits directories configured!");
}
state = State.BETWEEN_LOG_SEGMENTS;
}
@ -172,9 +162,8 @@ synchronized void close() {
LOG.warn("Closing log when already closed", new Exception());
return;
}
if (state == State.IN_SEGMENT) {
assert !journals.isEmpty();
assert editLogStream != null;
waitForSyncToFinish();
endCurrentLogSegment(true);
}
@ -193,20 +182,14 @@ void logEdit(final FSEditLogOp op) {
// wait if an automatic sync is scheduled
waitIfAutoSyncScheduled();
if (journals.isEmpty()) {
throw new java.lang.IllegalStateException(NO_JOURNAL_STREAMS_WARNING);
}
long start = beginTransaction();
op.setTransactionId(txid);
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (!jas.isActive()) return;
jas.stream.write(op);
try {
editLogStream.write(op);
} catch (IOException ex) {
// All journals failed, it is handled in logSync.
}
}, "logging edit");
endTransaction(start);
@ -251,14 +234,7 @@ synchronized void doneWithAutoSyncScheduling() {
* @return true if any of the edit stream says that it should sync
*/
private boolean shouldForceSync() {
for (JournalAndStream jas : journals) {
if (!jas.isActive()) continue;
if (jas.getCurrentStream().shouldForceSync()) {
return true;
}
}
return false;
return editLogStream.shouldForceSync();
}
private long beginTransaction() {
@ -322,7 +298,7 @@ synchronized void setNextTxId(long nextTxId) {
* NOTE: this should be done while holding the FSNamesystem lock, or
* else more operations can start writing while this is in progress.
*/
void logSyncAll() throws IOException {
void logSyncAll() {
// Record the most recent transaction ID as our own id
synchronized (this) {
TransactionId id = myTransactionId.get();
@ -366,12 +342,9 @@ public void logSync() {
// Fetch the transactionId of this thread.
long mytxid = myTransactionId.get().txid;
List<JournalAndStream> candidateJournals =
Lists.newArrayListWithCapacity(journals.size());
List<JournalAndStream> badJournals = Lists.newArrayList();
boolean sync = false;
try {
EditLogOutputStream logStream = null;
synchronized (this) {
try {
printStatistics(false);
@ -389,8 +362,10 @@ public void logSync() {
//
if (mytxid <= synctxid) {
numTransactionsBatchedInSync++;
if (metrics != null) // Metrics is non-null only when used inside name node
if (metrics != null) {
// Metrics is non-null only when used inside name node
metrics.incrTransactionsBatchedInSync();
}
return;
}
@ -400,40 +375,40 @@ public void logSync() {
sync = true;
// swap buffers
assert !journals.isEmpty() : "no editlog streams";
for (JournalAndStream jas : journals) {
if (!jas.isActive()) continue;
try {
jas.getCurrentStream().setReadyToFlush();
candidateJournals.add(jas);
} catch (IOException ie) {
LOG.error("Unable to get ready to flush.", ie);
badJournals.add(jas);
if (journalSet.isEmpty()) {
throw new IOException("No journals available to flush");
}
editLogStream.setReadyToFlush();
} catch (IOException e) {
LOG.fatal("Could not sync any journal to persistent storage. "
+ "Unsynced transactions: " + (txid - synctxid),
new Exception());
runtime.exit(1);
}
} finally {
// Prevent RuntimeException from blocking other log edit write
doneWithAutoSyncScheduling();
}
//editLogStream may become null,
//so store a local variable for flush.
logStream = editLogStream;
}
// do the sync
long start = now();
for (JournalAndStream jas : candidateJournals) {
if (!jas.isActive()) continue;
try {
jas.getCurrentStream().flush();
} catch (IOException ie) {
LOG.error("Unable to sync edit log.", ie);
//
// remember the streams that encountered an error.
//
badJournals.add(jas);
if (logStream != null) {
logStream.flush();
}
} catch (IOException ex) {
synchronized (this) {
LOG.fatal("Could not sync any journal to persistent storage. "
+ "Unsynced transactions: " + (txid - synctxid), new Exception());
runtime.exit(1);
}
}
long elapsed = now() - start;
disableAndReportErrorOnJournals(badJournals);
if (metrics != null) { // Metrics non-null only when used inside name node
metrics.addSync(elapsed);
@ -443,13 +418,6 @@ public void logSync() {
// Prevent RuntimeException from blocking other log edit sync
synchronized (this) {
if (sync) {
if (badJournals.size() >= journals.size()) {
LOG.fatal("Could not sync any journal to persistent storage. " +
"Unsynced transactions: " + (txid - synctxid),
new Exception());
runtime.exit(1);
}
synctxid = syncStart;
isSyncRunning = false;
}
@ -466,9 +434,6 @@ private void printStatistics(boolean force) {
if (lastPrintTime + 60000 > now && !force) {
return;
}
if (journals.isEmpty()) {
return;
}
lastPrintTime = now;
StringBuilder buf = new StringBuilder();
buf.append("Number of transactions: ");
@ -478,20 +443,9 @@ private void printStatistics(boolean force) {
buf.append("Number of transactions batched in Syncs: ");
buf.append(numTransactionsBatchedInSync);
buf.append(" Number of syncs: ");
for (JournalAndStream jas : journals) {
if (!jas.isActive()) continue;
buf.append(jas.getCurrentStream().getNumSync());
break;
}
buf.append(editLogStream.getNumSync());
buf.append(" SyncTimes(ms): ");
for (JournalAndStream jas : journals) {
if (!jas.isActive()) continue;
EditLogOutputStream eStream = jas.getCurrentStream();
buf.append(eStream.getTotalSyncTime());
buf.append(" ");
}
buf.append(journalSet.getSyncTimes());
LOG.info(buf);
}
@ -664,7 +618,6 @@ void logSymlink(String path, String value, long mtime,
* log delegation token to edit log
* @param id DelegationTokenIdentifier
* @param expiryTime of the token
* @return
*/
void logGetDelegationToken(DelegationTokenIdentifier id,
long expiryTime) {
@ -702,25 +655,12 @@ void logReassignLease(String leaseHolder, String src, String newHolder) {
logEdit(op);
}
/**
* @return the number of active (non-failed) journals
*/
private int countActiveJournals() {
int count = 0;
for (JournalAndStream jas : journals) {
if (jas.isActive()) {
count++;
}
}
return count;
}
/**
* Used only by unit tests.
*/
@VisibleForTesting
List<JournalAndStream> getJournals() {
return journals;
return journalSet.getAllJournalStreams();
}
/**
@ -734,62 +674,9 @@ synchronized void setRuntimeForTesting(Runtime runtime) {
/**
* Return a manifest of what finalized edit logs are available
*/
public synchronized RemoteEditLogManifest getEditLogManifest(
long fromTxId) throws IOException {
// Collect RemoteEditLogs available from each FileJournalManager
List<RemoteEditLog> allLogs = Lists.newArrayList();
for (JournalAndStream j : journals) {
if (j.getManager() instanceof FileJournalManager) {
FileJournalManager fjm = (FileJournalManager)j.getManager();
try {
allLogs.addAll(fjm.getRemoteEditLogs(fromTxId));
} catch (Throwable t) {
LOG.warn("Cannot list edit logs in " + fjm, t);
}
}
}
// Group logs by their starting txid
ImmutableListMultimap<Long, RemoteEditLog> logsByStartTxId =
Multimaps.index(allLogs, RemoteEditLog.GET_START_TXID);
long curStartTxId = fromTxId;
List<RemoteEditLog> logs = Lists.newArrayList();
while (true) {
ImmutableList<RemoteEditLog> logGroup = logsByStartTxId.get(curStartTxId);
if (logGroup.isEmpty()) {
// we have a gap in logs - for example because we recovered some old
// storage directory with ancient logs. Clear out any logs we've
// accumulated so far, and then skip to the next segment of logs
// after the gap.
SortedSet<Long> startTxIds = Sets.newTreeSet(logsByStartTxId.keySet());
startTxIds = startTxIds.tailSet(curStartTxId);
if (startTxIds.isEmpty()) {
break;
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Found gap in logs at " + curStartTxId + ": " +
"not returning previous logs in manifest.");
}
logs.clear();
curStartTxId = startTxIds.first();
continue;
}
}
// Find the one that extends the farthest forward
RemoteEditLog bestLog = Collections.max(logGroup);
logs.add(bestLog);
// And then start looking from after that point
curStartTxId = bestLog.getEndTxId() + 1;
}
RemoteEditLogManifest ret = new RemoteEditLogManifest(logs);
if (LOG.isDebugEnabled()) {
LOG.debug("Generated manifest for logs since " + fromTxId + ":"
+ ret);
}
return ret;
public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId)
throws IOException {
return journalSet.getEditLogManifest(fromTxId);
}
/**
@ -832,14 +719,9 @@ synchronized void startLogSegment(final long segmentTxId,
// See HDFS-2174.
storage.attemptRestoreRemovedStorage();
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.startLogSegment(segmentTxId);
}
}, "starting log segment " + segmentTxId);
if (countActiveJournals() == 0) {
try {
editLogStream = journalSet.startLogSegment(segmentTxId);
} catch (IOException ex) {
throw new IOException("Unable to start log segment " +
segmentTxId + ": no journals successfully started.");
}
@ -873,14 +755,12 @@ synchronized void endCurrentLogSegment(boolean writeEndTxn) {
final long lastTxId = getLastWrittenTxId();
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.close(lastTxId);
try {
journalSet.finalizeLogSegment(curSegmentTxId, lastTxId);
editLogStream = null;
} catch (IOException e) {
//All journals have failed, it will be handled in logSync.
}
}
}, "ending log segment");
state = State.BETWEEN_LOG_SEGMENTS;
}
@ -889,14 +769,15 @@ public void apply(JournalAndStream jas) throws IOException {
* Abort all current logs. Called from the backup node.
*/
synchronized void abortCurrentLogSegment() {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.abort();
try {
//Check for null, as abort can be called any time.
if (editLogStream != null) {
editLogStream.abort();
editLogStream = null;
}
} catch (IOException e) {
LOG.warn("All journals failed to abort", e);
}
}, "aborting all streams");
state = State.BETWEEN_LOG_SEGMENTS;
}
/**
@ -913,12 +794,11 @@ public void purgeLogsOlderThan(final long minTxIdToKeep) {
" when current segment starts at " + curSegmentTxId;
}
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.manager.purgeLogsOlderThan(minTxIdToKeep);
try {
journalSet.purgeLogsOlderThan(minTxIdToKeep);
} catch (IOException ex) {
//All journals have failed, it will be handled in logSync.
}
}, "purging logs older than " + minTxIdToKeep);
}
@ -946,9 +826,7 @@ synchronized long getSyncTxId() {
// sets the initial capacity of the flush buffer.
public void setOutputBufferCapacity(int size) {
for (JournalAndStream jas : journals) {
jas.manager.setOutputBufferCapacity(size);
}
journalSet.setOutputBufferCapacity(size);
}
/**
@ -969,7 +847,7 @@ synchronized void registerBackupNode(
if(bnReg.isRole(NamenodeRole.CHECKPOINT))
return; // checkpoint node does not stream edits
JournalAndStream jas = findBackupJournalAndStream(bnReg);
JournalManager jas = findBackupJournal(bnReg);
if (jas != null) {
// already registered
LOG.info("Backup node " + bnReg + " re-registers");
@ -978,35 +856,29 @@ synchronized void registerBackupNode(
LOG.info("Registering new backup node: " + bnReg);
BackupJournalManager bjm = new BackupJournalManager(bnReg, nnReg);
journals.add(new JournalAndStream(bjm));
journalSet.add(bjm);
}
synchronized void releaseBackupStream(NamenodeRegistration registration) {
for (Iterator<JournalAndStream> iter = journals.iterator();
iter.hasNext();) {
JournalAndStream jas = iter.next();
if (jas.manager instanceof BackupJournalManager &&
((BackupJournalManager)jas.manager).matchesRegistration(
registration)) {
jas.abort();
LOG.info("Removing backup journal " + jas);
iter.remove();
}
synchronized void releaseBackupStream(NamenodeRegistration registration)
throws IOException {
BackupJournalManager bjm = this.findBackupJournal(registration);
if (bjm != null) {
LOG.info("Removing backup journal " + bjm);
journalSet.remove(bjm);
}
}
/**
* Find the JournalAndStream associated with this BackupNode.
*
* @return null if it cannot be found
*/
private synchronized JournalAndStream findBackupJournalAndStream(
private synchronized BackupJournalManager findBackupJournal(
NamenodeRegistration bnReg) {
for (JournalAndStream jas : journals) {
if (jas.manager instanceof BackupJournalManager) {
BackupJournalManager bjm = (BackupJournalManager)jas.manager;
if (bjm.matchesRegistration(bnReg)) {
return jas;
}
for (JournalManager bjm : journalSet.getJournalManagers()) {
if ((bjm instanceof BackupJournalManager)
&& ((BackupJournalManager) bjm).matchesRegistration(bnReg)) {
return (BackupJournalManager) bjm;
}
}
return null;
@ -1019,123 +891,23 @@ private synchronized JournalAndStream findBackupJournalAndStream(
synchronized void logEdit(final int length, final byte[] data) {
long start = beginTransaction();
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.getCurrentStream().writeRaw(data, 0, length); // TODO writeRaw
try {
editLogStream.writeRaw(data, 0, length);
} catch (IOException ex) {
// All journals have failed, it will be handled in logSync.
}
}
}, "Logging edit");
endTransaction(start);
}
//// Iteration across journals
private interface JournalClosure {
public void apply(JournalAndStream jas) throws IOException;
}
/**
* Apply the given function across all of the journal managers, disabling
* any for which the closure throws an IOException.
* @param status message used for logging errors (e.g. "opening journal")
*/
private void mapJournalsAndReportErrors(
JournalClosure closure, String status) {
List<JournalAndStream> badJAS = Lists.newLinkedList();
for (JournalAndStream jas : journals) {
try {
closure.apply(jas);
} catch (Throwable t) {
LOG.error("Error " + status + " (journal " + jas + ")", t);
badJAS.add(jas);
}
}
disableAndReportErrorOnJournals(badJAS);
}
/**
* Called when some journals experience an error in some operation.
* This propagates errors to the storage level.
*/
private void disableAndReportErrorOnJournals(List<JournalAndStream> badJournals) {
if (badJournals == null || badJournals.isEmpty()) {
return; // nothing to do
}
for (JournalAndStream j : badJournals) {
LOG.error("Disabling journal " + j);
j.abort();
}
}
/**
* Find the best editlog input stream to read from txid. In this case
* best means the editlog which has the largest continuous range of
* transactions starting from the transaction id, fromTxId.
*
* If a journal throws an CorruptionException while reading from a txn id,
* it means that it has more transactions, but can't find any from fromTxId.
* If this is the case and no other journal has transactions, we should throw
* an exception as it means more transactions exist, we just can't load them.
*
* @param fromTxId Transaction id to start from.
* @return a edit log input stream with tranactions fromTxId
* or null if no more exist
*/
private EditLogInputStream selectStream(long fromTxId)
throws IOException {
JournalManager bestjm = null;
long bestjmNumTxns = 0;
CorruptionException corruption = null;
for (JournalAndStream jas : journals) {
JournalManager candidate = jas.getManager();
long candidateNumTxns = 0;
try {
candidateNumTxns = candidate.getNumberOfTransactions(fromTxId);
} catch (CorruptionException ce) {
corruption = ce;
} catch (IOException ioe) {
LOG.warn("Error reading number of transactions from " + candidate);
continue; // error reading disk, just skip
}
if (candidateNumTxns > bestjmNumTxns) {
bestjm = candidate;
bestjmNumTxns = candidateNumTxns;
}
}
if (bestjm == null) {
/**
* If all candidates either threw a CorruptionException or
* found 0 transactions, then a gap exists.
*/
if (corruption != null) {
throw new IOException("Gap exists in logs from "
+ fromTxId, corruption);
} else {
return null;
}
}
return bestjm.getInputStream(fromTxId);
}
/**
* Run recovery on all journals to recover any unclosed segments
*/
void recoverUnclosedStreams() {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.manager.recoverUnfinalizedSegments();
try {
journalSet.recoverUnfinalizedSegments();
} catch (IOException ex) {
// All journals have failed, it is handled in logSync.
}
}, "recovering unclosed streams");
}
/**
@ -1143,23 +915,16 @@ public void apply(JournalAndStream jas) throws IOException {
* @param fromTxId first transaction in the selected streams
* @param toAtLeast the selected streams must contain this transaction
*/
Collection<EditLogInputStream> selectInputStreams(long fromTxId, long toAtLeastTxId)
throws IOException {
List<EditLogInputStream> streams = Lists.newArrayList();
boolean gapFound = false;
EditLogInputStream stream = selectStream(fromTxId);
Collection<EditLogInputStream> selectInputStreams(long fromTxId,
long toAtLeastTxId) throws IOException {
List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
EditLogInputStream stream = journalSet.getInputStream(fromTxId);
while (stream != null) {
fromTxId = stream.getLastTxId() + 1;
streams.add(stream);
try {
stream = selectStream(fromTxId);
} catch (IOException ioe) {
gapFound = true;
break;
stream = journalSet.getInputStream(fromTxId);
}
}
if (fromTxId <= toAtLeastTxId || gapFound) {
if (fromTxId <= toAtLeastTxId) {
closeAllStreams(streams);
throw new IOException("No non-corrupt logs for txid "
+ fromTxId);
@ -1176,75 +941,4 @@ static void closeAllStreams(Iterable<EditLogInputStream> streams) {
IOUtils.closeStream(s);
}
}
/**
* Container for a JournalManager paired with its currently
* active stream.
*
* If a Journal gets disabled due to an error writing to its
* stream, then the stream will be aborted and set to null.
*/
static class JournalAndStream {
private final JournalManager manager;
private EditLogOutputStream stream;
private long segmentStartsAtTxId = HdfsConstants.INVALID_TXID;
private JournalAndStream(JournalManager manager) {
this.manager = manager;
}
private void startLogSegment(long txId) throws IOException {
Preconditions.checkState(stream == null);
stream = manager.startLogSegment(txId);
segmentStartsAtTxId = txId;
}
private void close(long lastTxId) throws IOException {
Preconditions.checkArgument(lastTxId >= segmentStartsAtTxId,
"invalid segment: lastTxId %s >= " +
"segment starting txid %s", lastTxId, segmentStartsAtTxId);
if (stream == null) return;
stream.close();
manager.finalizeLogSegment(segmentStartsAtTxId, lastTxId);
stream = null;
}
@VisibleForTesting
void abort() {
if (stream == null) return;
try {
stream.abort();
} catch (IOException ioe) {
LOG.error("Unable to abort stream " + stream, ioe);
}
stream = null;
segmentStartsAtTxId = HdfsConstants.INVALID_TXID;
}
private boolean isActive() {
return stream != null;
}
@VisibleForTesting
EditLogOutputStream getCurrentStream() {
return stream;
}
@Override
public String toString() {
return "JournalAndStream(mgr=" + manager +
", " + "stream=" + stream + ")";
}
@VisibleForTesting
void setCurrentStreamForTests(EditLogOutputStream stream) {
this.stream = stream;
}
@VisibleForTesting
JournalManager getManager() {
return manager;
}
}
}

View File

@ -23,7 +23,6 @@
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.HashMap;
import java.util.Comparator;
import java.util.Collections;
import java.util.regex.Matcher;

View File

@ -19,7 +19,6 @@
import java.io.IOException;
import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
/**
* A JournalManager is responsible for managing a single place of storing

View File

@ -0,0 +1,549 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.SortedSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableListMultimap;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimaps;
import com.google.common.collect.Sets;
/**
* Manages a collection of Journals. None of the methods are synchronized, it is
* assumed that FSEditLog methods, that use this class, use proper
* synchronization.
*/
public class JournalSet implements JournalManager {
static final Log LOG = LogFactory.getLog(FSEditLog.class);
/**
* Container for a JournalManager paired with its currently
* active stream.
*
* If a Journal gets disabled due to an error writing to its
* stream, then the stream will be aborted and set to null.
*
* This should be used outside JournalSet only for testing.
*/
@VisibleForTesting
static class JournalAndStream {
private final JournalManager journal;
private boolean disabled = false;
private EditLogOutputStream stream;
public JournalAndStream(JournalManager manager) {
this.journal = manager;
}
public void startLogSegment(long txId) throws IOException {
Preconditions.checkState(stream == null);
disabled = false;
stream = journal.startLogSegment(txId);
}
/**
* Closes the stream, also sets it to null.
*/
public void close() throws IOException {
if (stream == null) return;
stream.close();
stream = null;
}
/**
* Aborts the stream, also sets it to null.
*/
public void abort() {
if (stream == null) return;
try {
stream.abort();
} catch (IOException ioe) {
LOG.error("Unable to abort stream " + stream, ioe);
}
stream = null;
}
boolean isActive() {
return stream != null;
}
/**
* Should be used outside JournalSet only for testing.
*/
EditLogOutputStream getCurrentStream() {
return stream;
}
@Override
public String toString() {
return "JournalAndStream(mgr=" + journal +
", " + "stream=" + stream + ")";
}
void setCurrentStreamForTests(EditLogOutputStream stream) {
this.stream = stream;
}
JournalManager getManager() {
return journal;
}
private boolean isDisabled() {
return disabled;
}
private void setDisabled(boolean disabled) {
this.disabled = disabled;
}
}
private List<JournalAndStream> journals = Lists.newArrayList();
@Override
public EditLogOutputStream startLogSegment(final long txId) throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.startLogSegment(txId);
}
}, "starting log segment " + txId);
return new JournalSetOutputStream();
}
@Override
public void finalizeLogSegment(final long firstTxId, final long lastTxId)
throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.close();
jas.getManager().finalizeLogSegment(firstTxId, lastTxId);
}
}
}, "finalize log segment " + firstTxId + ", " + lastTxId);
}
/**
* Find the best editlog input stream to read from txid.
* If a journal throws an CorruptionException while reading from a txn id,
* it means that it has more transactions, but can't find any from fromTxId.
* If this is the case and no other journal has transactions, we should throw
* an exception as it means more transactions exist, we just can't load them.
*
* @param fromTxnId Transaction id to start from.
* @return A edit log input stream with tranactions fromTxId
* or null if no more exist
*/
@Override
public EditLogInputStream getInputStream(long fromTxnId) throws IOException {
JournalManager bestjm = null;
long bestjmNumTxns = 0;
CorruptionException corruption = null;
for (JournalAndStream jas : journals) {
JournalManager candidate = jas.getManager();
long candidateNumTxns = 0;
try {
candidateNumTxns = candidate.getNumberOfTransactions(fromTxnId);
} catch (CorruptionException ce) {
corruption = ce;
} catch (IOException ioe) {
continue; // error reading disk, just skip
}
if (candidateNumTxns > bestjmNumTxns) {
bestjm = candidate;
bestjmNumTxns = candidateNumTxns;
}
}
if (bestjm == null) {
if (corruption != null) {
throw new IOException("No non-corrupt logs for txid "
+ fromTxnId, corruption);
} else {
return null;
}
}
return bestjm.getInputStream(fromTxnId);
}
@Override
public long getNumberOfTransactions(long fromTxnId) throws IOException {
long num = 0;
for (JournalAndStream jas: journals) {
if (jas.isActive()) {
long newNum = jas.getManager().getNumberOfTransactions(fromTxnId);
if (newNum > num) {
num = newNum;
}
}
}
return num;
}
/**
* Returns true if there are no journals or all are disabled.
* @return True if no journals or all are disabled.
*/
public boolean isEmpty() {
for (JournalAndStream jas : journals) {
if (!jas.isDisabled()) {
return false;
}
}
return true;
}
/**
* Called when some journals experience an error in some operation.
*/
private void disableAndReportErrorOnJournals(List<JournalAndStream> badJournals) {
if (badJournals == null || badJournals.isEmpty()) {
return; // nothing to do
}
for (JournalAndStream j : badJournals) {
LOG.error("Disabling journal " + j);
j.abort();
j.setDisabled(true);
}
}
/**
* Implementations of this interface encapsulate operations that can be
* iteratively applied on all the journals. For example see
* {@link JournalSet#mapJournalsAndReportErrors}.
*/
private interface JournalClosure {
/**
* The operation on JournalAndStream.
* @param jas Object on which operations are performed.
* @throws IOException
*/
public void apply(JournalAndStream jas) throws IOException;
}
/**
* Apply the given operation across all of the journal managers, disabling
* any for which the closure throws an IOException.
* @param closure {@link JournalClosure} object encapsulating the operation.
* @param status message used for logging errors (e.g. "opening journal")
* @throws IOException If the operation fails on all the journals.
*/
private void mapJournalsAndReportErrors(
JournalClosure closure, String status) throws IOException{
List<JournalAndStream> badJAS = Lists.newLinkedList();
for (JournalAndStream jas : journals) {
try {
closure.apply(jas);
} catch (Throwable t) {
LOG.error("Error: " + status + " failed for (journal " + jas + ")", t);
badJAS.add(jas);
}
}
disableAndReportErrorOnJournals(badJAS);
if (badJAS.size() >= journals.size()) {
LOG.error("Error: "+status+" failed for all journals");
throw new IOException(status+" failed on all the journals");
}
}
/**
* An implementation of EditLogOutputStream that applies a requested method on
* all the journals that are currently active.
*/
private class JournalSetOutputStream extends EditLogOutputStream {
JournalSetOutputStream() throws IOException {
super();
}
@Override
void write(final FSEditLogOp op)
throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.getCurrentStream().write(op);
}
}
}, "write op");
}
@Override
void writeRaw(final byte[] data, final int offset, final int length)
throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.getCurrentStream().writeRaw(data, offset, length);
}
}
}, "write bytes");
}
@Override
void create() throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.getCurrentStream().create();
}
}
}, "create");
}
@Override
public void close() throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.close();
}
}, "close");
}
@Override
public void abort() throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.abort();
}
}, "abort");
}
@Override
void setReadyToFlush() throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.getCurrentStream().setReadyToFlush();
}
}
}, "setReadyToFlush");
}
@Override
protected void flushAndSync() throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.getCurrentStream().flushAndSync();
}
}
}, "flushAndSync");
}
@Override
public void flush() throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
if (jas.isActive()) {
jas.getCurrentStream().flush();
}
}
}, "flush");
}
@Override
public boolean shouldForceSync() {
for (JournalAndStream js : journals) {
if (js.isActive() && js.getCurrentStream().shouldForceSync()) {
return true;
}
}
return false;
}
@Override
protected long getNumSync() {
for (JournalAndStream jas : journals) {
if (jas.isActive()) {
return jas.getCurrentStream().getNumSync();
}
}
return 0;
}
}
@Override
public void setOutputBufferCapacity(final int size) {
try {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.getManager().setOutputBufferCapacity(size);
}
}, "setOutputBufferCapacity");
} catch (IOException e) {
LOG.error("Error in setting outputbuffer capacity");
}
}
@VisibleForTesting
List<JournalAndStream> getAllJournalStreams() {
return journals;
}
List<JournalManager> getJournalManagers() {
List<JournalManager> jList = new ArrayList<JournalManager>();
for (JournalAndStream j : journals) {
jList.add(j.getManager());
}
return jList;
}
void add(JournalManager j) {
journals.add(new JournalAndStream(j));
}
void remove(JournalManager j) {
JournalAndStream jasToRemove = null;
for (JournalAndStream jas: journals) {
if (jas.getManager().equals(j)) {
jasToRemove = jas;
break;
}
}
if (jasToRemove != null) {
jasToRemove.abort();
journals.remove(jasToRemove);
}
}
@Override
public void purgeLogsOlderThan(final long minTxIdToKeep) throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.getManager().purgeLogsOlderThan(minTxIdToKeep);
}
}, "purgeLogsOlderThan " + minTxIdToKeep);
}
@Override
public void recoverUnfinalizedSegments() throws IOException {
mapJournalsAndReportErrors(new JournalClosure() {
@Override
public void apply(JournalAndStream jas) throws IOException {
jas.getManager().recoverUnfinalizedSegments();
}
}, "recoverUnfinalizedSegments");
}
/**
* Return a manifest of what finalized edit logs are available. All available
* edit logs are returned starting from the transaction id passed.
*
* @param fromTxId Starting transaction id to read the logs.
* @return RemoteEditLogManifest object.
*/
public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId) {
// Collect RemoteEditLogs available from each FileJournalManager
List<RemoteEditLog> allLogs = Lists.newArrayList();
for (JournalAndStream j : journals) {
if (j.getManager() instanceof FileJournalManager) {
FileJournalManager fjm = (FileJournalManager)j.getManager();
try {
allLogs.addAll(fjm.getRemoteEditLogs(fromTxId));
} catch (Throwable t) {
LOG.warn("Cannot list edit logs in " + fjm, t);
}
}
}
// Group logs by their starting txid
ImmutableListMultimap<Long, RemoteEditLog> logsByStartTxId =
Multimaps.index(allLogs, RemoteEditLog.GET_START_TXID);
long curStartTxId = fromTxId;
List<RemoteEditLog> logs = Lists.newArrayList();
while (true) {
ImmutableList<RemoteEditLog> logGroup = logsByStartTxId.get(curStartTxId);
if (logGroup.isEmpty()) {
// we have a gap in logs - for example because we recovered some old
// storage directory with ancient logs. Clear out any logs we've
// accumulated so far, and then skip to the next segment of logs
// after the gap.
SortedSet<Long> startTxIds = Sets.newTreeSet(logsByStartTxId.keySet());
startTxIds = startTxIds.tailSet(curStartTxId);
if (startTxIds.isEmpty()) {
break;
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Found gap in logs at " + curStartTxId + ": " +
"not returning previous logs in manifest.");
}
logs.clear();
curStartTxId = startTxIds.first();
continue;
}
}
// Find the one that extends the farthest forward
RemoteEditLog bestLog = Collections.max(logGroup);
logs.add(bestLog);
// And then start looking from after that point
curStartTxId = bestLog.getEndTxId() + 1;
}
RemoteEditLogManifest ret = new RemoteEditLogManifest(logs);
if (LOG.isDebugEnabled()) {
LOG.debug("Generated manifest for logs since " + fromTxId + ":"
+ ret);
}
return ret;
}
/**
* Add sync times to the buffer.
*/
String getSyncTimes() {
StringBuilder buf = new StringBuilder();
for (JournalAndStream jas : journals) {
if (jas.isActive()) {
buf.append(jas.getCurrentStream().getTotalSyncTime());
buf.append(" ");
}
}
return buf.toString();
}
}

View File

@ -547,7 +547,7 @@ public void concat(String trg, String[] src) throws IOException {
}
@Override // ClientProtocol
public void rename(String src, String dst, Options.Rename... options)
public void rename2(String src, String dst, Options.Rename... options)
throws IOException {
nn.checkOperation(OperationCategory.WRITE);
if(stateChangeLog.isDebugEnabled()) {
@ -561,13 +561,6 @@ public void rename(String src, String dst, Options.Rename... options)
metrics.incrFilesRenamed();
}
@Deprecated
@Override // ClientProtocol
public boolean delete(String src) throws IOException {
nn.checkOperation(OperationCategory.WRITE);
return delete(src, true);
}
@Override // ClientProtocol
public boolean delete(String src, boolean recursive) throws IOException {
nn.checkOperation(OperationCategory.WRITE);

View File

@ -46,6 +46,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@ -112,7 +113,9 @@ public static String getRemoteAddress() {
private static DatanodeInfo chooseDatanode(final NameNode namenode,
final String path, final HttpOpParam.Op op, final long openOffset
) throws IOException {
if (op == GetOpParam.Op.OPEN || op == PostOpParam.Op.APPEND) {
if (op == GetOpParam.Op.OPEN
|| op == GetOpParam.Op.GETFILECHECKSUM
|| op == PostOpParam.Op.APPEND) {
final NamenodeProtocols np = namenode.getRpcServer();
final HdfsFileStatus status = np.getFileInfo(path);
final long len = status.getLen();
@ -245,7 +248,7 @@ public Response run() throws IOException, URISyntaxException {
case MKDIRS:
{
final boolean b = np.mkdirs(fullpath, permission.getFsPermission(), true);
final String js = JsonUtil.toJsonString(PutOpParam.Op.MKDIRS, b);
final String js = JsonUtil.toJsonString("boolean", b);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}
case RENAME:
@ -254,10 +257,10 @@ public Response run() throws IOException, URISyntaxException {
if (s.isEmpty()) {
@SuppressWarnings("deprecation")
final boolean b = np.rename(fullpath, dstPath.getValue());
final String js = JsonUtil.toJsonString(PutOpParam.Op.RENAME, b);
final String js = JsonUtil.toJsonString("boolean", b);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
} else {
np.rename(fullpath, dstPath.getValue(),
np.rename2(fullpath, dstPath.getValue(),
s.toArray(new Options.Rename[s.size()]));
return Response.ok().type(MediaType.APPLICATION_JSON).build();
}
@ -265,7 +268,7 @@ public Response run() throws IOException, URISyntaxException {
case SETREPLICATION:
{
final boolean b = np.setReplication(fullpath, replication.getValue());
final String js = JsonUtil.toJsonString(PutOpParam.Op.SETREPLICATION, b);
final String js = JsonUtil.toJsonString("boolean", b);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}
case SETOWNER:
@ -431,6 +434,18 @@ public Response run() throws IOException, URISyntaxException {
final StreamingOutput streaming = getListingStream(np, fullpath);
return Response.ok(streaming).type(MediaType.APPLICATION_JSON).build();
}
case GETCONTENTSUMMARY:
{
final ContentSummary contentsummary = np.getContentSummary(fullpath);
final String js = JsonUtil.toJsonString(contentsummary);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}
case GETFILECHECKSUM:
{
final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
op.getValue(), -1L);
return Response.temporaryRedirect(uri).build();
}
case GETDELEGATIONTOKEN:
{
final Token<? extends TokenIdentifier> token = generateDelegationToken(
@ -467,7 +482,7 @@ private static StreamingOutput getListingStream(final NamenodeProtocols np,
@Override
public void write(final OutputStream outstream) throws IOException {
final PrintStream out = new PrintStream(outstream);
out.print('[');
out.println("{\"" + HdfsFileStatus[].class.getSimpleName() + "\":[");
final HdfsFileStatus[] partial = first.getPartialListing();
if (partial.length > 0) {
@ -486,7 +501,7 @@ public void write(final OutputStream outstream) throws IOException {
}
}
out.println(']');
out.println("]}");
}
};
}
@ -522,7 +537,7 @@ public Response run() throws IOException {
case DELETE:
{
final boolean b = namenode.getRpcServer().delete(fullpath, recursive.getValue());
final String js = JsonUtil.toJsonString(DeleteOpParam.Op.DELETE, b);
final String js = JsonUtil.toJsonString("boolean", b);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}
default:

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hdfs.web;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
@ -24,6 +26,8 @@
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -34,45 +38,33 @@
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.StringUtils;
import org.mortbay.util.ajax.JSON;
/** JSON Utilities */
public class JsonUtil {
private static class ThreadLocalMap extends ThreadLocal<Map<String, Object>> {
@Override
protected Map<String, Object> initialValue() {
return new TreeMap<String, Object>();
}
@Override
public Map<String, Object> get() {
final Map<String, Object> m = super.get();
m.clear();
return m;
}
}
private static final ThreadLocalMap jsonMap = new ThreadLocalMap();
private static final ThreadLocalMap tokenMap = new ThreadLocalMap();
private static final ThreadLocalMap datanodeInfoMap = new ThreadLocalMap();
private static final ThreadLocalMap extendedBlockMap = new ThreadLocalMap();
private static final ThreadLocalMap locatedBlockMap = new ThreadLocalMap();
private static final Object[] EMPTY_OBJECT_ARRAY = {};
private static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
/** Convert a token object to a Json string. */
public static String toJsonString(final Token<? extends TokenIdentifier> token
) throws IOException {
return toJsonString(Token.class, toJsonMap(token));
}
private static Map<String, Object> toJsonMap(
final Token<? extends TokenIdentifier> token) throws IOException {
if (token == null) {
return null;
}
final Map<String, Object> m = tokenMap.get();
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("urlString", token.encodeToUrlString());
return JSON.toString(m);
return m;
}
/** Convert a Json map to a Token. */
@ -91,46 +83,52 @@ public static Token<? extends TokenIdentifier> toToken(
/** Convert a Json map to a Token of DelegationTokenIdentifier. */
@SuppressWarnings("unchecked")
public static Token<DelegationTokenIdentifier> toDelegationToken(
final Map<?, ?> m) throws IOException {
final Map<?, ?> json) throws IOException {
final Map<?, ?> m = (Map<?, ?>)json.get(Token.class.getSimpleName());
return (Token<DelegationTokenIdentifier>)toToken(m);
}
/** Convert a Json map to a Token of BlockTokenIdentifier. */
@SuppressWarnings("unchecked")
public static Token<BlockTokenIdentifier> toBlockToken(
private static Token<BlockTokenIdentifier> toBlockToken(
final Map<?, ?> m) throws IOException {
return (Token<BlockTokenIdentifier>)toToken(m);
}
/** Convert an exception object to a Json string. */
public static String toJsonString(final Exception e) {
final Map<String, Object> m = jsonMap.get();
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("className", e.getClass().getName());
m.put("message", e.getMessage());
return JSON.toString(m);
return toJsonString(RemoteException.class, m);
}
/** Convert a Json map to a RemoteException. */
public static RemoteException toRemoteException(final Map<String, Object> m) {
public static RemoteException toRemoteException(final Map<?, ?> json) {
final Map<?, ?> m = (Map<?, ?>)json.get(RemoteException.class.getSimpleName());
final String className = (String)m.get("className");
final String message = (String)m.get("message");
return new RemoteException(className, message);
}
private static String toJsonString(final Class<?> clazz, final Object value) {
return toJsonString(clazz.getSimpleName(), value);
}
/** Convert a key-value pair to a Json string. */
public static String toJsonString(final Object key, final Object value) {
final Map<String, Object> m = jsonMap.get();
m.put(key instanceof String ? (String) key : key.toString(), value);
public static String toJsonString(final String key, final Object value) {
final Map<String, Object> m = new TreeMap<String, Object>();
m.put(key, value);
return JSON.toString(m);
}
/** Convert a FsPermission object to a string. */
public static String toString(final FsPermission permission) {
private static String toString(final FsPermission permission) {
return String.format("%o", permission.toShort());
}
/** Convert a string to a FsPermission object. */
public static FsPermission toFsPermission(final String s) {
private static FsPermission toFsPermission(final String s) {
return new FsPermission(Short.parseShort(s, 8));
}
@ -139,7 +137,7 @@ public static String toJsonString(final HdfsFileStatus status) {
if (status == null) {
return null;
} else {
final Map<String, Object> m = jsonMap.get();
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("localName", status.getLocalName());
m.put("isDir", status.isDir());
m.put("isSymlink", status.isSymlink());
@ -155,21 +153,17 @@ public static String toJsonString(final HdfsFileStatus status) {
m.put("modificationTime", status.getModificationTime());
m.put("blockSize", status.getBlockSize());
m.put("replication", status.getReplication());
return JSON.toString(m);
return toJsonString(HdfsFileStatus.class, m);
}
}
@SuppressWarnings("unchecked")
static Map<String, Object> parse(String jsonString) {
return (Map<String, Object>) JSON.parse(jsonString);
}
/** Convert a Json map to a HdfsFileStatus object. */
public static HdfsFileStatus toFileStatus(final Map<String, Object> m) {
if (m == null) {
public static HdfsFileStatus toFileStatus(final Map<?, ?> json) {
if (json == null) {
return null;
}
final Map<?, ?> m = (Map<?, ?>)json.get(HdfsFileStatus.class.getSimpleName());
final String localName = (String) m.get("localName");
final boolean isDir = (Boolean) m.get("isDir");
final boolean isSymlink = (Boolean) m.get("isSymlink");
@ -189,22 +183,22 @@ public static HdfsFileStatus toFileStatus(final Map<String, Object> m) {
symlink, DFSUtil.string2Bytes(localName));
}
/** Convert a LocatedBlock to a Json string. */
public static String toJsonString(final ExtendedBlock extendedblock) {
/** Convert an ExtendedBlock to a Json map. */
private static Map<String, Object> toJsonMap(final ExtendedBlock extendedblock) {
if (extendedblock == null) {
return null;
}
final Map<String, Object> m = extendedBlockMap.get();
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("blockPoolId", extendedblock.getBlockPoolId());
m.put("blockId", extendedblock.getBlockId());
m.put("numBytes", extendedblock.getNumBytes());
m.put("generationStamp", extendedblock.getGenerationStamp());
return JSON.toString(m);
return m;
}
/** Convert a Json map to an ExtendedBlock object. */
public static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
private static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
if (m == null) {
return null;
}
@ -216,13 +210,13 @@ public static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
return new ExtendedBlock(blockPoolId, blockId, numBytes, generationStamp);
}
/** Convert a DatanodeInfo to a Json string. */
public static String toJsonString(final DatanodeInfo datanodeinfo) {
/** Convert a DatanodeInfo to a Json map. */
private static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
if (datanodeinfo == null) {
return null;
}
final Map<String, Object> m = datanodeInfoMap.get();
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("name", datanodeinfo.getName());
m.put("storageID", datanodeinfo.getStorageID());
m.put("infoPort", datanodeinfo.getInfoPort());
@ -238,11 +232,11 @@ public static String toJsonString(final DatanodeInfo datanodeinfo) {
m.put("networkLocation", datanodeinfo.getNetworkLocation());
m.put("hostName", datanodeinfo.getHostName());
m.put("adminState", datanodeinfo.getAdminState().name());
return JSON.toString(m);
return m;
}
/** Convert a Json map to an DatanodeInfo object. */
public static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
private static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
if (m == null) {
return null;
}
@ -264,25 +258,23 @@ public static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
AdminStates.valueOf((String)m.get("adminState")));
}
/** Convert a DatanodeInfo[] to a Json string. */
public static String toJsonString(final DatanodeInfo[] array
) throws IOException {
/** Convert a DatanodeInfo[] to a Json array. */
private static Object[] toJsonArray(final DatanodeInfo[] array) {
if (array == null) {
return null;
} else if (array.length == 0) {
return "[]";
return EMPTY_OBJECT_ARRAY;
} else {
final StringBuilder b = new StringBuilder().append('[').append(
toJsonString(array[0]));
for(int i = 1; i < array.length; i++) {
b.append(", ").append(toJsonString(array[i]));
final Object[] a = new Object[array.length];
for(int i = 0; i < array.length; i++) {
a[i] = toJsonMap(array[i]);
}
return b.append(']').toString();
return a;
}
}
/** Convert an Object[] to a DatanodeInfo[]. */
public static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) {
private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) {
if (objects == null) {
return null;
} else if (objects.length == 0) {
@ -290,66 +282,63 @@ public static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) {
} else {
final DatanodeInfo[] array = new DatanodeInfo[objects.length];
for(int i = 0; i < array.length; i++) {
array[i] = (DatanodeInfo)toDatanodeInfo((Map<?, ?>) objects[i]);
array[i] = toDatanodeInfo((Map<?, ?>) objects[i]);
}
return array;
}
}
/** Convert a LocatedBlock to a Json string. */
public static String toJsonString(final LocatedBlock locatedblock
/** Convert a LocatedBlock to a Json map. */
private static Map<String, Object> toJsonMap(final LocatedBlock locatedblock
) throws IOException {
if (locatedblock == null) {
return null;
}
final Map<String, Object> m = locatedBlockMap.get();
m.put("blockToken", toJsonString(locatedblock.getBlockToken()));
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("blockToken", toJsonMap(locatedblock.getBlockToken()));
m.put("isCorrupt", locatedblock.isCorrupt());
m.put("startOffset", locatedblock.getStartOffset());
m.put("block", toJsonString(locatedblock.getBlock()));
m.put("locations", toJsonString(locatedblock.getLocations()));
return JSON.toString(m);
m.put("block", toJsonMap(locatedblock.getBlock()));
m.put("locations", toJsonArray(locatedblock.getLocations()));
return m;
}
/** Convert a Json map to LocatedBlock. */
public static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException {
private static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException {
if (m == null) {
return null;
}
final ExtendedBlock b = toExtendedBlock((Map<?, ?>)JSON.parse((String)m.get("block")));
final ExtendedBlock b = toExtendedBlock((Map<?, ?>)m.get("block"));
final DatanodeInfo[] locations = toDatanodeInfoArray(
(Object[])JSON.parse((String)m.get("locations")));
(Object[])m.get("locations"));
final long startOffset = (Long)m.get("startOffset");
final boolean isCorrupt = (Boolean)m.get("isCorrupt");
final LocatedBlock locatedblock = new LocatedBlock(b, locations, startOffset, isCorrupt);
locatedblock.setBlockToken(toBlockToken((Map<?, ?>)JSON.parse((String)m.get("blockToken"))));
locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
return locatedblock;
}
/** Convert a LocatedBlock[] to a Json string. */
public static String toJsonString(final List<LocatedBlock> array
/** Convert a LocatedBlock[] to a Json array. */
private static Object[] toJsonArray(final List<LocatedBlock> array
) throws IOException {
if (array == null) {
return null;
} else if (array.size() == 0) {
return "[]";
return EMPTY_OBJECT_ARRAY;
} else {
final StringBuilder b = new StringBuilder().append('[').append(
toJsonString(array.get(0)));
for(int i = 1; i < array.size(); i++) {
b.append(",\n ").append(toJsonString(array.get(i)));
final Object[] a = new Object[array.size()];
for(int i = 0; i < array.size(); i++) {
a[i] = toJsonMap(array.get(0));
}
return b.append(']').toString();
return a;
}
}
/** Convert an Object[] to a List of LocatedBlock.
* @throws IOException */
public static List<LocatedBlock> toLocatedBlockList(final Object[] objects
/** Convert an Object[] to a List of LocatedBlock. */
private static List<LocatedBlock> toLocatedBlockList(final Object[] objects
) throws IOException {
if (objects == null) {
return null;
@ -358,7 +347,7 @@ public static List<LocatedBlock> toLocatedBlockList(final Object[] objects
} else {
final List<LocatedBlock> list = new ArrayList<LocatedBlock>(objects.length);
for(int i = 0; i < objects.length; i++) {
list.add((LocatedBlock)toLocatedBlock((Map<?, ?>)objects[i]));
list.add(toLocatedBlock((Map<?, ?>)objects[i]));
}
return list;
}
@ -371,31 +360,115 @@ public static String toJsonString(final LocatedBlocks locatedblocks
return null;
}
final Map<String, Object> m = jsonMap.get();
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("fileLength", locatedblocks.getFileLength());
m.put("isUnderConstruction", locatedblocks.isUnderConstruction());
m.put("locatedBlocks", toJsonString(locatedblocks.getLocatedBlocks()));
m.put("lastLocatedBlock", toJsonString(locatedblocks.getLastLocatedBlock()));
m.put("locatedBlocks", toJsonArray(locatedblocks.getLocatedBlocks()));
m.put("lastLocatedBlock", toJsonMap(locatedblocks.getLastLocatedBlock()));
m.put("isLastBlockComplete", locatedblocks.isLastBlockComplete());
return JSON.toString(m);
return toJsonString(LocatedBlocks.class, m);
}
/** Convert a Json map to LocatedBlock. */
public static LocatedBlocks toLocatedBlocks(final Map<String, Object> m
public static LocatedBlocks toLocatedBlocks(final Map<?, ?> json
) throws IOException {
if (m == null) {
if (json == null) {
return null;
}
final Map<?, ?> m = (Map<?, ?>)json.get(LocatedBlocks.class.getSimpleName());
final long fileLength = (Long)m.get("fileLength");
final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
final List<LocatedBlock> locatedBlocks = toLocatedBlockList(
(Object[])JSON.parse((String) m.get("locatedBlocks")));
(Object[])m.get("locatedBlocks"));
final LocatedBlock lastLocatedBlock = toLocatedBlock(
(Map<?, ?>)JSON.parse((String)m.get("lastLocatedBlock")));
(Map<?, ?>)m.get("lastLocatedBlock"));
final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
lastLocatedBlock, isLastBlockComplete);
}
/** Convert a ContentSummary to a Json string. */
public static String toJsonString(final ContentSummary contentsummary) {
if (contentsummary == null) {
return null;
}
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("length", contentsummary.getLength());
m.put("fileCount", contentsummary.getFileCount());
m.put("directoryCount", contentsummary.getDirectoryCount());
m.put("quota", contentsummary.getQuota());
m.put("spaceConsumed", contentsummary.getSpaceConsumed());
m.put("spaceQuota", contentsummary.getSpaceQuota());
return toJsonString(ContentSummary.class, m);
}
/** Convert a Json map to a ContentSummary. */
public static ContentSummary toContentSummary(final Map<?, ?> json) {
if (json == null) {
return null;
}
final Map<?, ?> m = (Map<?, ?>)json.get(ContentSummary.class.getSimpleName());
final long length = (Long)m.get("length");
final long fileCount = (Long)m.get("fileCount");
final long directoryCount = (Long)m.get("directoryCount");
final long quota = (Long)m.get("quota");
final long spaceConsumed = (Long)m.get("spaceConsumed");
final long spaceQuota = (Long)m.get("spaceQuota");
return new ContentSummary(length, fileCount, directoryCount,
quota, spaceConsumed, spaceQuota);
}
/** Convert a MD5MD5CRC32FileChecksum to a Json string. */
public static String toJsonString(final MD5MD5CRC32FileChecksum checksum) {
if (checksum == null) {
return null;
}
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("algorithm", checksum.getAlgorithmName());
m.put("length", checksum.getLength());
m.put("bytes", StringUtils.byteToHexString(checksum.getBytes()));
return toJsonString(MD5MD5CRC32FileChecksum.class, m);
}
/** Convert a Json map to a MD5MD5CRC32FileChecksum. */
public static MD5MD5CRC32FileChecksum toMD5MD5CRC32FileChecksum(
final Map<?, ?> json) throws IOException {
if (json == null) {
return null;
}
final Map<?, ?> m = (Map<?, ?>)json.get(
MD5MD5CRC32FileChecksum.class.getSimpleName());
final String algorithm = (String)m.get("algorithm");
final int length = (int)(long)(Long)m.get("length");
final byte[] bytes = StringUtils.hexStringToByte((String)m.get("bytes"));
final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
final int bytesPerCRC = in.readInt();
final long crcPerBlock = in.readLong();
final MD5Hash md5 = MD5Hash.read(in);
final MD5MD5CRC32FileChecksum checksum = new MD5MD5CRC32FileChecksum(
bytesPerCRC, crcPerBlock, md5);
//check algorithm name
final String alg = "MD5-of-" + crcPerBlock + "MD5-of-" + bytesPerCRC + "CRC32";
if (!alg.equals(algorithm)) {
throw new IOException("Algorithm not matched: algorithm=" + algorithm
+ ", crcPerBlock=" + crcPerBlock
+ ", bytesPerCRC=" + bytesPerCRC);
}
//check length
if (length != checksum.getLength()) {
throw new IOException("Length not matched: length=" + length
+ ", checksum.getLength()=" + checksum.getLength());
}
return checksum;
}
}

View File

@ -33,10 +33,12 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
@ -278,7 +280,7 @@ public boolean mkdirs(Path f, FsPermission permission) throws IOException {
final HttpOpParam.Op op = PutOpParam.Op.MKDIRS;
final Map<String, Object> json = run(op, f,
new PermissionParam(applyUMask(permission)));
return (Boolean)json.get(op.toString());
return (Boolean)json.get("boolean");
}
@Override
@ -287,7 +289,7 @@ public boolean rename(final Path src, final Path dst) throws IOException {
final HttpOpParam.Op op = PutOpParam.Op.RENAME;
final Map<String, Object> json = run(op, src,
new DstPathParam(makeQualified(dst).toUri().getPath()));
return (Boolean)json.get(op.toString());
return (Boolean)json.get("boolean");
}
@SuppressWarnings("deprecation")
@ -327,7 +329,7 @@ public boolean setReplication(final Path p, final short replication
final HttpOpParam.Op op = PutOpParam.Op.SETREPLICATION;
final Map<String, Object> json = run(op, p,
new ReplicationParam(replication));
return (Boolean)json.get(op.toString());
return (Boolean)json.get("boolean");
}
@Override
@ -384,7 +386,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize,
public boolean delete(Path f, boolean recursive) throws IOException {
final HttpOpParam.Op op = DeleteOpParam.Op.DELETE;
final Map<String, Object> json = run(op, f, new RecursiveParam(recursive));
return (Boolean)json.get(op.toString());
return (Boolean)json.get("boolean");
}
@Override
@ -401,7 +403,9 @@ public FileStatus[] listStatus(final Path f) throws IOException {
statistics.incrementReadOps(1);
final HttpOpParam.Op op = GetOpParam.Op.LISTSTATUS;
final Object[] array = run(op, f);
final Map<?, ?> json = run(op, f);
final Object[] array = (Object[])json.get(
HdfsFileStatus[].class.getSimpleName());
//convert FileStatus
final FileStatus[] statuses = new FileStatus[array.length];
@ -449,4 +453,23 @@ public BlockLocation[] getFileBlockLocations(final Path p,
new LengthParam(length));
return DFSUtil.locatedBlocks2Locations(JsonUtil.toLocatedBlocks(m));
}
@Override
public ContentSummary getContentSummary(final Path p) throws IOException {
statistics.incrementReadOps(1);
final HttpOpParam.Op op = GetOpParam.Op.GETCONTENTSUMMARY;
final Map<String, Object> m = run(op, p);
return JsonUtil.toContentSummary(m);
}
@Override
public MD5MD5CRC32FileChecksum getFileChecksum(final Path p
) throws IOException {
statistics.incrementReadOps(1);
final HttpOpParam.Op op = GetOpParam.Op.GETFILECHECKSUM;
final Map<String, Object> m = run(op, p);
return JsonUtil.toMD5MD5CRC32FileChecksum(m);
}
}

View File

@ -28,6 +28,8 @@ public static enum Op implements HttpOpParam.Op {
GETFILESTATUS(HttpURLConnection.HTTP_OK),
LISTSTATUS(HttpURLConnection.HTTP_OK),
GETCONTENTSUMMARY(HttpURLConnection.HTTP_OK),
GETFILECHECKSUM(HttpURLConnection.HTTP_OK),
GETDELEGATIONTOKEN(HttpURLConnection.HTTP_OK),

View File

@ -15,4 +15,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
/usr/sbin/useradd --comment "Hadoop HDFS" --shell /bin/bash -M -r --groups hadoop --home /var/lib/hadoop/hdfs hdfs 2> /dev/null || :
/usr/sbin/useradd --comment "Hadoop HDFS" -u 201 --shell /bin/bash -M -r --groups hadoop --home /var/lib/hadoop/hdfs hdfs 2> /dev/null || :

View File

@ -0,0 +1,142 @@
#! /bin/sh
# 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.
### BEGIN INIT INFO
# Provides: hadoop-secondarynamenode
# Required-Start: $remote_fs $syslog
# Required-Stop: $remote_fs $syslog
# Default-Start: 2 3 4 5
# Default-Stop:
# Short-Description: Apache Hadoop Name Node server
### END INIT INFO
set -e
# /etc/init.d/hadoop-secondarynamenode: start and stop the Apache Hadoop Secondary Name Node daemon
test -x /usr/bin/hadoop || exit 0
( /usr/bin/hadoop 2>&1 | grep -q hadoop ) 2>/dev/null || exit 0
umask 022
if test -f /etc/default/hadoop-env.sh; then
. /etc/default/hadoop-env.sh
fi
. /lib/lsb/init-functions
# Are we running from init?
run_by_init() {
([ "$previous" ] && [ "$runlevel" ]) || [ "$runlevel" = S ]
}
check_for_no_start() {
# forget it if we're trying to start, and /etc/hadoop/hadoop-secondarynamenode_not_to_be_run exists
if [ -e /etc/hadoop/hadoop-secondarynamenode_not_to_be_run ]; then
if [ "$1" = log_end_msg ]; then
log_end_msg 0
fi
if ! run_by_init; then
log_action_msg "Apache Hadoop Name Node server not in use (/etc/hadoop/hadoop-secondarynamenode_not_to_be_run)"
fi
exit 0
fi
}
check_privsep_dir() {
# Create the PrivSep empty dir if necessary
if [ ! -d ${HADOOP_PID_DIR} ]; then
mkdir -p ${HADOOP_PID_DIR}
chown root:hadoop ${HADOOP_PID_DIR}
chmod 0775 ${HADOOP_PID_DIR}
fi
}
export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin"
export HADOOP_PREFIX="/usr"
case "$1" in
start)
check_privsep_dir
check_for_no_start
log_daemon_msg "Starting Apache Hadoop Secondary Name Node server" "hadoop-secondarynamenode"
if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-secondarynamenode.pid -c hdfs -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start secondarynamenode; then
log_end_msg 0
else
log_end_msg 1
fi
;;
stop)
log_daemon_msg "Stopping Apache Hadoop Secondary Name Node server" "hadoop-secondarynamenode"
if start-stop-daemon --stop --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-secondarynamenode.pid; then
log_end_msg 0
else
log_end_msg 1
fi
;;
restart)
check_privsep_dir
log_daemon_msg "Restarting Apache Hadoop Secondary Name Node server" "hadoop-secondarynamenode"
start-stop-daemon --stop --quiet --oknodo --retry 30 --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-secondarynamenode.pid
check_for_no_start log_end_msg
if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-secondarynamenode.pid -c hdfs -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start secondarynamenode; then
log_end_msg 0
else
log_end_msg 1
fi
;;
try-restart)
check_privsep_dir
log_daemon_msg "Restarting Apache Hadoop Secondary Name Node server" "hadoop-secondarynamenode"
set +e
start-stop-daemon --stop --quiet --retry 30 --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-secondarynamenode.pid
RET="$?"
set -e
case $RET in
0)
# old daemon stopped
check_for_no_start log_end_msg
if start-stop-daemon --start --quiet --oknodo --pidfile ${HADOOP_PID_DIR}/hadoop-hdfs-secondarynamenode.pid -c hdfs -x ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh -- --config ${HADOOP_CONF_DIR} start secondarynamenode; then
log_end_msg 0
else
log_end_msg 1
fi
;;
1)
# daemon not running
log_progress_msg "(not running)"
log_end_msg 0
;;
*)
# failed to stop
log_progress_msg "(failed to stop)"
log_end_msg 1
;;
esac
;;
status)
status_of_proc -p ${HADOOP_PID_DIR}/hadoop-hdfs-secondarynamenode.pid ${JAVA_HOME}/bin/java hadoop-secondarynamenode && exit 0 || exit $?
;;
*)
log_action_msg "Usage: /etc/init.d/hadoop-secondarynamenode {start|stop|restart|try-restart|status}"
exit 1
esac
exit 0

View File

@ -0,0 +1,92 @@
#!/bin/bash
# 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.
#
# Starts a Hadoop secondarynamenode
#
# chkconfig: 2345 90 10
# description: Hadoop secondarynamenode
source /etc/rc.d/init.d/functions
source /etc/default/hadoop-env.sh
RETVAL=0
PIDFILE="${HADOOP_PID_DIR}/hadoop-hdfs-secondarynamenode.pid"
desc="Hadoop secondary namenode daemon"
export HADOOP_PREFIX="/usr"
start() {
echo -n $"Starting $desc (hadoop-secondarynamenode): "
daemon --user hdfs ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh --config "${HADOOP_CONF_DIR}" start secondarynamenode $1
RETVAL=$?
echo
[ $RETVAL -eq 0 ] && touch /var/lock/subsys/hadoop-secondarynamenode
return $RETVAL
}
upgrade() {
start -upgrade
}
stop() {
echo -n $"Stopping $desc (hadoop-secondarynamenode): "
daemon --user hdfs ${HADOOP_PREFIX}/sbin/hadoop-daemon.sh --config "${HADOOP_CONF_DIR}" stop secondarynamenode
RETVAL=$?
sleep 5
echo
[ $RETVAL -eq 0 ] && rm -f /var/lock/subsys/hadoop-secondarynamenode $PIDFILE
}
checkstatus(){
status -p $PIDFILE ${JAVA_HOME}/bin/java
RETVAL=$?
}
restart() {
stop
start
}
condrestart(){
[ -e /var/lock/subsys/hadoop-secondarynamenode ] && restart || :
}
case "$1" in
start)
start
;;
upgrade)
upgrade
;;
stop)
stop
;;
status)
checkstatus
;;
restart)
restart
;;
condrestart|try-restart)
condrestart
;;
*)
echo $"Usage: $0 {start|stop|status|restart|try-restart|upgrade}"
exit 1
esac
exit $RETVAL

View File

@ -144,8 +144,8 @@ mv ${RPM_BUILD_DIR}/%{_final_name}/share/* ${RPM_BUILD_DIR}%{_share_dir}
rm -rf ${RPM_BUILD_DIR}/%{_final_name}/etc
%pre
getent group hadoop 2>/dev/null >/dev/null || /usr/sbin/groupadd -r hadoop
/usr/sbin/useradd --comment "Hadoop HDFS" --shell /bin/bash -M -r --groups hadoop --home %{_var_dir}/hdfs hdfs 2> /dev/null || :
getent group hadoop 2>/dev/null >/dev/null || /usr/sbin/groupadd -g 123 -r hadoop
/usr/sbin/useradd --comment "Hadoop HDFS" -u 201 --shell /bin/bash -M -r --groups hadoop --home %{_var_dir}/hdfs hdfs 2> /dev/null || :
%post
bash ${RPM_INSTALL_PREFIX0}/sbin/update-hdfs-env.sh \

View File

@ -26,6 +26,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.Random;
import org.apache.commons.logging.impl.Log4JLogger;
@ -37,6 +38,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.log4j.Level;
import org.junit.Test;
@ -399,15 +401,40 @@ public void testFileChecksum() throws Exception {
RAN.setSeed(seed);
final Configuration conf = getTestConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "localhost");
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
final FileSystem hdfs = cluster.getFileSystem();
final String hftpuri = "hftp://" + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
System.out.println("hftpuri=" + hftpuri);
final FileSystem hftp = new Path(hftpuri).getFileSystem(conf);
final String dir = "/filechecksum";
final String nnAddr = conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
final UserGroupInformation current = UserGroupInformation.getCurrentUser();
final UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
current.getShortUserName() + "x", new String[]{"user"});
//hftp
final String hftpuri = "hftp://" + nnAddr;
System.out.println("hftpuri=" + hftpuri);
final FileSystem hftp = ugi.doAs(
new PrivilegedExceptionAction<FileSystem>() {
@Override
public FileSystem run() throws Exception {
return new Path(hftpuri).getFileSystem(conf);
}
});
//webhdfs
final String webhdfsuri = WebHdfsFileSystem.SCHEME + "://" + nnAddr;
System.out.println("webhdfsuri=" + webhdfsuri);
final FileSystem webhdfs = ugi.doAs(
new PrivilegedExceptionAction<FileSystem>() {
@Override
public FileSystem run() throws Exception {
return new Path(webhdfsuri).getFileSystem(conf);
}
});
final Path dir = new Path("/filechecksum");
final int block_size = 1024;
final int buffer_size = conf.getInt("io.file.buffer.size", 4096);
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
@ -432,6 +459,7 @@ public void testFileChecksum() throws Exception {
final FileChecksum hdfsfoocs = hdfs.getFileChecksum(foo);
System.out.println("hdfsfoocs=" + hdfsfoocs);
//hftp
final FileChecksum hftpfoocs = hftp.getFileChecksum(foo);
System.out.println("hftpfoocs=" + hftpfoocs);
@ -439,6 +467,14 @@ public void testFileChecksum() throws Exception {
final FileChecksum qfoocs = hftp.getFileChecksum(qualified);
System.out.println("qfoocs=" + qfoocs);
//webhdfs
final FileChecksum webhdfsfoocs = webhdfs.getFileChecksum(foo);
System.out.println("webhdfsfoocs=" + webhdfsfoocs);
final Path webhdfsqualified = new Path(webhdfsuri + dir, "foo" + n);
final FileChecksum webhdfs_qfoocs = webhdfs.getFileChecksum(webhdfsqualified);
System.out.println("webhdfs_qfoocs=" + webhdfs_qfoocs);
//write another file
final Path bar = new Path(dir, "bar" + n);
{
@ -454,24 +490,40 @@ public void testFileChecksum() throws Exception {
assertEquals(hdfsfoocs.hashCode(), barhashcode);
assertEquals(hdfsfoocs, barcs);
//hftp
assertEquals(hftpfoocs.hashCode(), barhashcode);
assertEquals(hftpfoocs, barcs);
assertEquals(qfoocs.hashCode(), barhashcode);
assertEquals(qfoocs, barcs);
//webhdfs
assertEquals(webhdfsfoocs.hashCode(), barhashcode);
assertEquals(webhdfsfoocs, barcs);
assertEquals(webhdfs_qfoocs.hashCode(), barhashcode);
assertEquals(webhdfs_qfoocs, barcs);
}
hdfs.setPermission(dir, new FsPermission((short)0));
{ //test permission error on hftp
hdfs.setPermission(new Path(dir), new FsPermission((short)0));
try {
final String username = UserGroupInformation.getCurrentUser().getShortUserName() + "1";
final HftpFileSystem hftp2 = cluster.getHftpFileSystemAs(username, conf, 0, "somegroup");
hftp2.getFileChecksum(qualified);
hftp.getFileChecksum(qualified);
fail();
} catch(IOException ioe) {
FileSystem.LOG.info("GOOD: getting an exception", ioe);
}
}
{ //test permission error on webhdfs
try {
webhdfs.getFileChecksum(webhdfsqualified);
fail();
} catch(IOException ioe) {
FileSystem.LOG.info("GOOD: getting an exception", ioe);
}
}
hdfs.setPermission(dir, new FsPermission((short)0777));
}
cluster.shutdown();
}

View File

@ -34,6 +34,7 @@
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.junit.Test;
@ -770,6 +771,11 @@ public void testSpaceCommands() throws Exception {
}
}
private static void checkContentSummary(final ContentSummary expected,
final ContentSummary computed) {
assertEquals(expected.toString(), computed.toString());
}
/**
* Violate a space quota using files of size < 1 block. Test that block
* allocation conservatively assumes that for quota checking the entire
@ -781,12 +787,18 @@ public void testBlockAllocationAdjustsUsageConservatively()
Configuration conf = new HdfsConfiguration();
final int BLOCK_SIZE = 6 * 1024;
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
MiniDFSCluster cluster =
new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
DFSAdmin admin = new DFSAdmin(conf);
final String nnAddr = conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
final String webhdfsuri = WebHdfsFileSystem.SCHEME + "://" + nnAddr;
System.out.println("webhdfsuri=" + webhdfsuri);
final FileSystem webhdfs = new Path(webhdfsuri).getFileSystem(conf);
try {
Path dir = new Path("/test");
Path file1 = new Path("/test/test1");
@ -806,6 +818,7 @@ public void testBlockAllocationAdjustsUsageConservatively()
DFSTestUtil.createFile(fs, file1, FILE_SIZE, (short) 3, 1L);
DFSTestUtil.waitReplication(fs, file1, (short) 3);
c = fs.getContentSummary(dir);
checkContentSummary(c, webhdfs.getContentSummary(dir));
assertEquals("Quota is half consumed", QUOTA_SIZE / 2,
c.getSpaceConsumed());
@ -836,12 +849,18 @@ public void testMultipleFilesSmallerThanOneBlock() throws Exception {
Configuration conf = new HdfsConfiguration();
final int BLOCK_SIZE = 6 * 1024;
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
MiniDFSCluster cluster =
new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
DFSAdmin admin = new DFSAdmin(conf);
final String nnAddr = conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
final String webhdfsuri = WebHdfsFileSystem.SCHEME + "://" + nnAddr;
System.out.println("webhdfsuri=" + webhdfsuri);
final FileSystem webhdfs = new Path(webhdfsuri).getFileSystem(conf);
try {
//Test for deafult NameSpace Quota
@ -882,6 +901,7 @@ public void testMultipleFilesSmallerThanOneBlock() throws Exception {
// Should account for all 59 files (almost QUOTA_SIZE)
c = fs.getContentSummary(dir);
checkContentSummary(c, webhdfs.getContentSummary(dir));
assertEquals("Invalid space consumed", 59 * FILE_SIZE * 3,
c.getSpaceConsumed());
assertEquals("Invalid space consumed", QUOTA_SIZE - (59 * FILE_SIZE * 3),

View File

@ -20,6 +20,7 @@
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy;
@ -33,6 +34,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -73,7 +75,7 @@ public void shutDownMiniCluster() throws IOException {
public void testSingleFailedEditsDirOnFlush() throws IOException {
assertTrue(doAnEdit());
// Invalidate one edits journal.
invalidateEditsDirAtIndex(0, true);
invalidateEditsDirAtIndex(0, true, false);
// Make sure runtime.exit(...) hasn't been called at all yet.
assertExitInvocations(0);
assertTrue(doAnEdit());
@ -86,8 +88,22 @@ public void testSingleFailedEditsDirOnFlush() throws IOException {
public void testAllEditsDirsFailOnFlush() throws IOException {
assertTrue(doAnEdit());
// Invalidate both edits journals.
invalidateEditsDirAtIndex(0, true);
invalidateEditsDirAtIndex(1, true);
invalidateEditsDirAtIndex(0, true, false);
invalidateEditsDirAtIndex(1, true, false);
// Make sure runtime.exit(...) hasn't been called at all yet.
assertExitInvocations(0);
assertTrue(doAnEdit());
// The previous edit could not be synced to any persistent storage, should
// have halted the NN.
assertExitInvocations(1);
}
@Test
public void testAllEditsDirFailOnWrite() throws IOException {
assertTrue(doAnEdit());
// Invalidate both edits journals.
invalidateEditsDirAtIndex(0, true, true);
invalidateEditsDirAtIndex(1, true, true);
// Make sure runtime.exit(...) hasn't been called at all yet.
assertExitInvocations(0);
assertTrue(doAnEdit());
@ -100,7 +116,7 @@ public void testAllEditsDirsFailOnFlush() throws IOException {
public void testSingleFailedEditsDirOnSetReadyToFlush() throws IOException {
assertTrue(doAnEdit());
// Invalidate one edits journal.
invalidateEditsDirAtIndex(0, false);
invalidateEditsDirAtIndex(0, false, false);
// Make sure runtime.exit(...) hasn't been called at all yet.
assertExitInvocations(0);
assertTrue(doAnEdit());
@ -117,16 +133,18 @@ public void testSingleFailedEditsDirOnSetReadyToFlush() throws IOException {
* @return the original <code>EditLogOutputStream</code> of the journal.
*/
private EditLogOutputStream invalidateEditsDirAtIndex(int index,
boolean failOnFlush) throws IOException {
boolean failOnFlush, boolean failOnWrite) throws IOException {
FSImage fsimage = cluster.getNamesystem().getFSImage();
FSEditLog editLog = fsimage.getEditLog();
FSEditLog.JournalAndStream jas = editLog.getJournals().get(index);
JournalAndStream jas = editLog.getJournals().get(index);
EditLogFileOutputStream elos =
(EditLogFileOutputStream) jas.getCurrentStream();
EditLogFileOutputStream spyElos = spy(elos);
if (failOnWrite) {
doThrow(new IOException("fail on write()")).when(spyElos).write(
(FSEditLogOp) any());
}
if (failOnFlush) {
doThrow(new IOException("fail on flush()")).when(spyElos).flush();
} else {
@ -151,7 +169,7 @@ private void restoreEditsDirAtIndex(int index, EditLogOutputStream elos) {
FSImage fsimage = cluster.getNamesystem().getFSImage();
FSEditLog editLog = fsimage.getEditLog();
FSEditLog.JournalAndStream jas = editLog.getJournals().get(index);
JournalAndStream jas = editLog.getJournals().get(index);
jas.setCurrentStreamForTests(elos);
}

View File

@ -40,6 +40,7 @@
import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.log4j.Level;
@ -356,7 +357,7 @@ public void testSaveImageWhileSyncInProgress() throws Exception {
FSImage fsimage = namesystem.getFSImage();
FSEditLog editLog = fsimage.getEditLog();
FSEditLog.JournalAndStream jas = editLog.getJournals().get(0);
JournalAndStream jas = editLog.getJournals().get(0);
EditLogFileOutputStream spyElos =
spy((EditLogFileOutputStream)jas.getCurrentStream());
jas.setCurrentStreamForTests(spyElos);

View File

@ -28,7 +28,6 @@
import java.util.Iterator;
import java.util.Set;
import static org.mockito.Matchers.anyByte;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy;
@ -45,7 +44,7 @@
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
@ -123,7 +122,7 @@ public void invalidateStorage(FSImage fi, Set<File> filesToInvalidate) throws IO
// simulate an error
fi.getStorage().reportErrorsOnDirectories(al);
for (FSEditLog.JournalAndStream j : fi.getEditLog().getJournals()) {
for (JournalAndStream j : fi.getEditLog().getJournals()) {
if (j.getManager() instanceof FileJournalManager) {
FileJournalManager fm = (FileJournalManager)j.getManager();
if (fm.getStorageDirectory().getRoot().equals(path2)

View File

@ -17,14 +17,16 @@
*/
package org.apache.hadoop.hdfs.web;
import java.util.Map;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.web.JsonUtil;
import org.junit.Assert;
import org.junit.Test;
import org.mortbay.util.ajax.JSON;
public class TestJsonUtil {
static FileStatus toFileStatus(HdfsFileStatus f, String parent) {
@ -46,7 +48,7 @@ public void testHdfsFileStatus() {
System.out.println("fstatus = " + fstatus);
final String json = JsonUtil.toJsonString(status);
System.out.println("json = " + json.replace(",", ",\n "));
final HdfsFileStatus s2 = JsonUtil.toFileStatus(JsonUtil.parse(json));
final HdfsFileStatus s2 = JsonUtil.toFileStatus((Map<?, ?>)JSON.parse(json));
final FileStatus fs2 = toFileStatus(s2, parent);
System.out.println("s2 = " + s2);
System.out.println("fs2 = " + fs2);

View File

@ -22,6 +22,9 @@ Trunk (unreleased changes)
BUG FIXES
MAPREDUCE-2950. [Gridmix] TestUserResolve fails in trunk.
(Ravi Gummadi via amarrk)
MAPREDUCE-2784. [Gridmix] Bug fixes in ExecutionSummarizer and
ResourceUsageMatcher. (amarrk)
@ -75,6 +78,9 @@ Release 0.23.0 - Unreleased
MAPREDUCE-2930. Added the ability to be able to generate graphs from the
state-machine definitions. (Binglin Chang via vinodkv)
MAPREDUCE-2719. Add a simple, DistributedShell, application to illustrate
alternate frameworks on YARN. (Hitesh Shah via acmurthy)
IMPROVEMENTS
MAPREDUCE-2187. Reporter sends progress during sort/merge. (Anupam Seth via
@ -321,6 +327,33 @@ Release 0.23.0 - Unreleased
MAPREDUCE-3099. Add docs for setting up a single node MRv2 cluster.
(mahadev)
MAPREDUCE-3001. Added task-specific counters to AppMaster and JobHistory
web-UIs. (Robert Joseph Evans via vinodkv)
MAPREDUCE-3098. Fixed RM and MR AM to report YarnApplicationState and
application's FinalStatus separately. (Hitesh Shah via vinodkv)
MAPREDUCE-2889. Added documentation for writing new YARN applications.
(Hitesh Shah via acmurthy)
MAPREDUCE-3134. Added documentation the CapacityScheduler. (acmurthy)
MAPREDUCE-3138. Add a utility to help applications bridge changes in
Context Objects APIs due to MAPREDUCE-954. (omalley via acmurthy)
MAPREDUCE-3013. Removed YarnConfiguration.YARN_SECURITY_INFO and its usage
as it doesn't affect security any more. (vinodkv)
MAPREDUCE-2907. Changed log level for various messages in ResourceManager
from INFO to DEBUG. (Ravi Prakash via vinodkv)
MAPREDUCE-2702. Added a new API in OutputCommitter for recovering
the outputs of tasks from a crashed job so as to support MR Application
Master recovery. (Sharad Agarwal and Arun C Murthy via vinodkv)
MAPREDUCE-2738. Added the missing cluster level statisticss on the RM web
UI. (Robert Joseph Evans via vinodkv)
OPTIMIZATIONS
MAPREDUCE-2026. Make JobTracker.getJobCounters() and
@ -343,9 +376,6 @@ Release 0.23.0 - Unreleased
MAPREDUCE-2539. Fixed NPE in getMapTaskReports in JobClient. (Robert Evans via
acmurthy)
MAPREDUCE-2531. Fixed jobcontrol to downgrade JobID. (Robert Evans via
acmurthy)
MAPREDUCE-1978. Rumen TraceBuilder should provide recursive
input folder scanning.
@ -1472,6 +1502,33 @@ Release 0.23.0 - Unreleased
MAPREDUCE-3078. Ensure MapReduce AM reports progress correctly for
displaying on the RM Web-UI. (vinodkv via acmurthy)
MAPREDUCE-3114. Fixed invalid ApplicationURL on RM WebUI. (Subroto Sanyal
via vinodkv)
MAPREDUCE-2791. Added missing info on 'job -status' output. (Devaraj K via
acmurthy)
MAPREDUCE-2996. Add uber-ness information to JobHistory. (Jonathan Eagles
via acmurthy)
MAPREDUCE-3050. Add ability to get resource usage information for
applications and nodes. (Robert Evans via acmurthy)
MAPREDUCE-3113. Ensure bin/yarn and bin/yarn-daemon.sh identify the root
of the install properly. (Xie Xianshan via acmurthy)
MAPREDUCE-2792. Replace usage of node ip-addresses with hostnames.
(vinodkv via acmurthy)
MAPREDUCE-3112. Fixed recursive sourcing of HADOOP_OPTS environment
variable. (Eric Yang)
MAPREDUCE-3056. Changed the default staging directory to not include
user.name to prevent issues with non-secure mode. (Devaraj K via vinodkv)
MAPREDUCE-2913. Fixed TestMRJobs.testFailingMapper to assert the correct
TaskCompletionEventStatus. (Jonathan Eagles via vinodkv)
Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES
@ -2057,6 +2114,12 @@ Release 0.22.0 - Unreleased
MAPREDUCE-2991. queueinfo.jsp fails to show queue status for Capacity
scheduler if queue names contain special symbols. (Priyo Mustafi via shv)
MAPREDUCE-2779. JobSplitWriter.java can't handle large job.split file.
(Ming Ma via shv)
MAPREDUCE-2531. Fixed jobcontrol to downgrade JobID. (Robert Evans via
acmurthy)
Release 0.21.1 - Unreleased
NEW FEATURES

View File

@ -22,6 +22,7 @@
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.security.PrivilegedExceptionAction;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
@ -84,7 +85,6 @@
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service;
@ -657,14 +657,29 @@ public static void main(String[] args) {
new CompositeServiceShutdownHook(appMaster));
YarnConfiguration conf = new YarnConfiguration(new JobConf());
conf.addResource(new Path(MRJobConfig.JOB_CONF_FILE));
conf.set(MRJobConfig.USER_NAME,
System.getProperty("user.name"));
UserGroupInformation.setConfiguration(conf);
appMaster.init(conf);
appMaster.start();
String jobUserName = System
.getenv(ApplicationConstants.Environment.USER.name());
conf.set(MRJobConfig.USER_NAME, jobUserName);
initAndStartAppMaster(appMaster, conf, jobUserName);
} catch (Throwable t) {
LOG.fatal("Error starting MRAppMaster", t);
System.exit(1);
}
}
protected static void initAndStartAppMaster(final MRAppMaster appMaster,
final YarnConfiguration conf, String jobUserName) throws IOException,
InterruptedException {
UserGroupInformation.setConfiguration(conf);
UserGroupInformation appMasterUgi = UserGroupInformation
.createRemoteUser(jobUserName);
appMasterUgi.doAs(new PrivilegedExceptionAction<Object>() {
@Override
public Object run() throws Exception {
appMaster.init(conf);
appMaster.start();
return null;
}
});
}
}

View File

@ -114,7 +114,7 @@ public MRClientService(AppContext appContext) {
}
public void start() {
Configuration conf = new Configuration(getConfig()); // Just for not messing up sec-info class config
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
InetSocketAddress address = NetUtils.createSocketAddr("0.0.0.0:0");
InetAddress hostNameResolved = null;
@ -134,9 +134,6 @@ public void start() {
ApplicationTokenIdentifier identifier =
new ApplicationTokenIdentifier(this.appContext.getApplicationID());
secretManager.setMasterKey(identifier, bytes);
conf.setClass(
YarnConfiguration.YARN_SECURITY_INFO,
SchedulerSecurityInfo.class, SecurityInfo.class); // Same for now.
}
server =
rpc.getServer(MRClientProtocol.class, protocolHandler, address,

View File

@ -590,12 +590,12 @@ public JobReport getReport() {
if (getState() == JobState.NEW) {
return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
startTime, finishTime, setupProgress, 0.0f,
0.0f, cleanupProgress);
0.0f, cleanupProgress, remoteJobConfFile.toString());
}
return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
startTime, finishTime, setupProgress, computeProgress(mapTasks),
computeProgress(reduceTasks), cleanupProgress);
computeProgress(reduceTasks), cleanupProgress, remoteJobConfFile.toString());
} finally {
readLock.unlock();
}
@ -991,10 +991,9 @@ protected void setup(JobImpl job) throws IOException {
String user =
UserGroupInformation.getCurrentUser().getShortUserName();
Path path = MRApps.getStagingAreaDir(job.conf, user);
LOG.info("DEBUG --- startJobs:"
+ " parent="
+ path + " child="
+ oldJobIDString);
if(LOG.isDebugEnabled()) {
LOG.debug("startJobs: parent=" + path + " child=" + oldJobIDString);
}
job.remoteJobSubmitDir =
FileSystem.get(job.conf).makeQualified(
@ -1135,7 +1134,8 @@ public void transition(JobImpl job, JobEvent event) {
new JobInitedEvent(job.oldJobId,
job.startTime,
job.numMapTasks, job.numReduceTasks,
job.getState().toString()); //Will transition to state running. Currently in INITED
job.getState().toString(),
job.isUber()); //Will transition to state running. Currently in INITED
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jie));
JobInfoChangeEvent jice = new JobInfoChangeEvent(job.oldJobId,
job.submitTime, job.startTime);

View File

@ -91,17 +91,11 @@ public ContainerLauncherImpl(AppContext context) {
@Override
public synchronized void init(Configuration conf) {
// Clone configuration for this component so that the SecurityInfo setting
// doesn't affect the original configuration
Configuration myLocalConfig = new Configuration(conf);
myLocalConfig.setClass(
YarnConfiguration.YARN_SECURITY_INFO,
ContainerManagerSecurityInfo.class, SecurityInfo.class);
this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
this.limitOnPoolSize = conf.getInt(
MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT,
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
super.init(myLocalConfig);
super.init(conf);
}
public void start() {

View File

@ -47,6 +47,7 @@
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
@ -146,8 +147,8 @@ protected float getApplicationProgress() {
protected void register() {
//Register
String host =
clientService.getBindAddress().getAddress().getHostAddress();
String host = clientService.getBindAddress().getAddress()
.getCanonicalHostName();
try {
RegisterApplicationMasterRequest request =
recordFactory.newRecordInstance(RegisterApplicationMasterRequest.class);
@ -169,14 +170,14 @@ protected void register() {
protected void unregister() {
try {
String finalState = "RUNNING";
FinalApplicationStatus finishState = FinalApplicationStatus.UNDEFINED;
if (job.getState() == JobState.SUCCEEDED) {
finalState = "SUCCEEDED";
finishState = FinalApplicationStatus.SUCCEEDED;
} else if (job.getState() == JobState.KILLED) {
finalState = "KILLED";
finishState = FinalApplicationStatus.KILLED;
} else if (job.getState() == JobState.FAILED
|| job.getState() == JobState.ERROR) {
finalState = "FAILED";
finishState = FinalApplicationStatus.FAILED;
}
StringBuffer sb = new StringBuffer();
for (String s : job.getDiagnostics()) {
@ -191,7 +192,7 @@ protected void unregister() {
FinishApplicationMasterRequest request =
recordFactory.newRecordInstance(FinishApplicationMasterRequest.class);
request.setAppAttemptId(this.applicationAttemptId);
request.setFinalState(finalState.toString());
request.setFinishApplicationStatus(finishState);
request.setDiagnostics(sb.toString());
request.setTrackingUrl(historyUrl);
scheduler.finishApplicationMaster(request);
@ -246,7 +247,7 @@ public void run() {
protected AMRMProtocol createSchedulerProxy() {
final YarnRPC rpc = YarnRPC.create(getConfig());
final Configuration conf = new Configuration(getConfig());
final Configuration conf = getConfig();
final String serviceAddr = conf.get(
YarnConfiguration.RM_SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS);
@ -259,9 +260,6 @@ protected AMRMProtocol createSchedulerProxy() {
}
if (UserGroupInformation.isSecurityEnabled()) {
conf.setClass(YarnConfiguration.YARN_SECURITY_INFO,
SchedulerSecurityInfo.class, SecurityInfo.class);
String tokenURLEncodedStr = System.getenv().get(
ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
LOG.debug("AppMasterToken is " + tokenURLEncodedStr);

View File

@ -28,4 +28,6 @@ public interface AMParams {
static final String TASK_ID = "task.id";
static final String TASK_TYPE = "task.type";
static final String ATTEMPT_STATE = "attempt.state";
static final String COUNTER_GROUP = "counter.group";
static final String COUNTER_NAME = "counter.name";
}

View File

@ -34,9 +34,14 @@ public void setup() {
route(pajoin("/job", JOB_ID), AppController.class, "job");
route(pajoin("/conf", JOB_ID), AppController.class, "conf");
route(pajoin("/jobcounters", JOB_ID), AppController.class, "jobCounters");
route(pajoin("/singlejobcounter",JOB_ID, COUNTER_GROUP, COUNTER_NAME),
AppController.class, "singleJobCounter");
route(pajoin("/tasks", JOB_ID, TASK_TYPE), AppController.class, "tasks");
route(pajoin("/attempts", JOB_ID, TASK_TYPE, ATTEMPT_STATE),
AppController.class, "attempts");
route(pajoin("/task", TASK_ID), AppController.class, "task");
route(pajoin("/taskcounters", TASK_ID), AppController.class, "taskCounters");
route(pajoin("/singletaskcounter",TASK_ID, COUNTER_GROUP, COUNTER_NAME),
AppController.class, "singleTaskCounter");
}
}

View File

@ -20,6 +20,8 @@
import static org.apache.hadoop.yarn.util.StringHelper.join;
import java.io.IOException;
import java.net.URLDecoder;
import java.util.Locale;
import javax.servlet.http.HttpServletResponse;
@ -30,7 +32,7 @@
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.util.Apps;
import org.apache.hadoop.yarn.util.StringHelper;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.webapp.Controller;
import org.apache.hadoop.yarn.webapp.View;
@ -41,7 +43,7 @@
* This class renders the various pages that the web app supports.
*/
public class AppController extends Controller implements AMParams {
final App app;
protected final App app;
protected AppController(App app, Configuration conf, RequestContext ctx,
String title) {
@ -110,6 +112,54 @@ public void jobCounters() {
render(countersPage());
}
/**
* Display a page showing a task's counters
*/
public void taskCounters() {
requireTask();
if (app.getTask() != null) {
setTitle(StringHelper.join("Counters for ", $(TASK_ID)));
}
render(countersPage());
}
/**
* @return the class that will render the /singlejobcounter page
*/
protected Class<? extends View> singleCounterPage() {
return SingleCounterPage.class;
}
/**
* Render the /singlejobcounter page
* @throws IOException on any error.
*/
public void singleJobCounter() throws IOException{
requireJob();
set(COUNTER_GROUP, URLDecoder.decode($(COUNTER_GROUP), "UTF-8"));
set(COUNTER_NAME, URLDecoder.decode($(COUNTER_NAME), "UTF-8"));
if (app.getJob() != null) {
setTitle(StringHelper.join($(COUNTER_GROUP)," ",$(COUNTER_NAME),
" for ", $(JOB_ID)));
}
render(singleCounterPage());
}
/**
* Render the /singletaskcounter page
* @throws IOException on any error.
*/
public void singleTaskCounter() throws IOException{
requireTask();
set(COUNTER_GROUP, URLDecoder.decode($(COUNTER_GROUP), "UTF-8"));
set(COUNTER_NAME, URLDecoder.decode($(COUNTER_NAME), "UTF-8"));
if (app.getTask() != null) {
setTitle(StringHelper.join($(COUNTER_GROUP)," ",$(COUNTER_NAME),
" for ", $(TASK_ID)));
}
render(singleCounterPage());
}
/**
* @return the class that will render the /tasks page
*/

View File

@ -61,6 +61,29 @@ public class CountersBlock extends HtmlBlock {
p()._("Sorry, no counters for nonexistent", $(TASK_ID, "task"))._();
return;
}
if(total == null || total.getAllCounterGroups() == null ||
total.getAllCounterGroups().size() <= 0) {
String type = $(TASK_ID);
if(type == null || type.isEmpty()) {
type = $(JOB_ID, "the job");
}
html.
p()._("Sorry it looks like ",type," has no counters.")._();
return;
}
String urlBase;
String urlId;
if(task != null) {
urlBase = "singletaskcounter";
urlId = MRApps.toString(task.getID());
} else {
urlBase = "singlejobcounter";
urlId = MRApps.toString(job.getID());
}
int numGroups = 0;
TBODY<TABLE<DIV<Hamlet>>> tbody = html.
div(_INFO_WRAP).
@ -79,12 +102,13 @@ public class CountersBlock extends HtmlBlock {
// serves as an indicator of where we're in the tag hierarchy.
TR<THEAD<TABLE<TD<TR<TBODY<TABLE<DIV<Hamlet>>>>>>>> groupHeadRow = tbody.
tr().
th().$title(g.getName()).
th().$title(g.getName()).$class("ui-state-default").
_(fixGroupDisplayName(g.getDisplayName()))._().
td().$class(C_TABLE).
table(".dt-counters").
thead().
tr().th(".name", "Name");
if (map != null) {
groupHeadRow.th("Map").th("Reduce");
}
@ -97,7 +121,9 @@ public class CountersBlock extends HtmlBlock {
TR<TBODY<TABLE<TD<TR<TBODY<TABLE<DIV<Hamlet>>>>>>>> groupRow = group.
tr().
td().$title(counter.getName()).
_(counter.getDisplayName())._();
a(url(urlBase,urlId,g.getName(),
counter.getName()), counter.getDisplayName()).
_();
if (map != null) {
Counter mc = mg == null ? null : mg.getCounter(counter.getName());
Counter rc = rg == null ? null : rg.getCounter(counter.getName());
@ -121,7 +147,7 @@ private void getCounters(AppContext ctx) {
jobID = taskID.getJobId();
} else {
String jid = $(JOB_ID);
if (!jid.isEmpty()) {
if (jid != null && !jid.isEmpty()) {
jobID = MRApps.toJobID(jid);
}
}

View File

@ -20,13 +20,19 @@
import org.apache.hadoop.yarn.webapp.SubView;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
public class CountersPage extends AppView {
@Override protected void preHead(Page.HTML<_> html) {
commonPreHead(html);
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}");
String tid = $(TASK_ID);
String activeNav = "3";
if(tid == null || tid.isEmpty()) {
activeNav = "2";
}
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
set(DATATABLES_SELECTOR, "#counters .dt-counters");
set(initSelector(DATATABLES),
"{bJQueryUI:true, sDom:'t', iDisplayLength:-1}");
@ -35,9 +41,9 @@ public class CountersPage extends AppView {
@Override protected void postHead(Page.HTML<_> html) {
html.
style("#counters, .dt-counters { table-layout: fixed }",
"#counters th { overflow: hidden; vertical-align: center }",
"#counters th { overflow: hidden; vertical-align: middle }",
"#counters .dataTables_wrapper { min-height: 1em }",
"#counters .group { width: 10em }",
"#counters .group { width: 15em }",
"#counters .name { width: 30em }");
}

View File

@ -55,6 +55,14 @@ public class NavBlock extends HtmlBlock {
li().a(url("conf", jobid), "Configuration")._().
li().a(url("tasks", jobid, "m"), "Map tasks")._().
li().a(url("tasks", jobid, "r"), "Reduce tasks")._()._();
if (app.getTask() != null) {
String taskid = MRApps.toString(app.getTask().getID());
nav.
h3("Task").
ul().
li().a(url("task", taskid), "Task Overview")._().
li().a(url("taskcounters", taskid), "Counters")._()._();
}
}
nav.
h3("Tools").

View File

@ -0,0 +1,151 @@
/**
* 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 com.google.inject.Inject;
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.mapreduce.v2.api.records.Counter;
import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
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.TR;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
public class SingleCounterBlock extends HtmlBlock {
protected TreeMap<String, Long> values = new TreeMap<String, Long>();
protected Job job;
protected Task task;
@Inject SingleCounterBlock(AppContext appCtx, ViewContext ctx) {
super(ctx);
this.populateMembers(appCtx);
}
@Override protected void render(Block html) {
if (job == null) {
html.
p()._("Sorry, no counters for nonexistent", $(JOB_ID, "job"))._();
return;
}
if (!$(TASK_ID).isEmpty() && task == null) {
html.
p()._("Sorry, no counters for nonexistent", $(TASK_ID, "task"))._();
return;
}
String columnType = task == null ? "Task" : "Task Attempt";
TBODY<TABLE<DIV<Hamlet>>> tbody = html.
div(_INFO_WRAP).
table("#singleCounter").
thead().
tr().
th(".ui-state-default", columnType).
th(".ui-state-default", "Value")._()._().
tbody();
for (Map.Entry<String, Long> entry : values.entrySet()) {
TR<TBODY<TABLE<DIV<Hamlet>>>> row = tbody.tr();
String id = entry.getKey();
String val = entry.getValue().toString();
if(task != null) {
row.td(id);
row.td().br().$title(val)._()._(val)._();
} else {
row.td().a(url("singletaskcounter",entry.getKey(),
$(COUNTER_GROUP), $(COUNTER_NAME)), id)._();
row.td().br().$title(val)._().a(url("singletaskcounter",entry.getKey(),
$(COUNTER_GROUP), $(COUNTER_NAME)), val)._();
}
row._();
}
tbody._()._()._();
}
private void populateMembers(AppContext ctx) {
JobId jobID = null;
TaskId taskID = null;
String tid = $(TASK_ID);
if (!tid.isEmpty()) {
taskID = MRApps.toTaskID(tid);
jobID = taskID.getJobId();
} else {
String jid = $(JOB_ID);
if (!jid.isEmpty()) {
jobID = MRApps.toJobID(jid);
}
}
if (jobID == null) {
return;
}
job = ctx.getJob(jobID);
if (job == null) {
return;
}
if (taskID != null) {
task = job.getTask(taskID);
if (task == null) {
return;
}
for(Map.Entry<TaskAttemptId, TaskAttempt> entry :
task.getAttempts().entrySet()) {
long value = 0;
CounterGroup group = entry.getValue().getCounters()
.getCounterGroup($(COUNTER_GROUP));
if(group != null) {
Counter c = group.getCounter($(COUNTER_NAME));
if(c != null) {
value = c.getValue();
}
}
values.put(MRApps.toString(entry.getKey()), value);
}
return;
}
// Get all types of counters
Map<TaskId, Task> tasks = job.getTasks();
for(Map.Entry<TaskId, Task> entry : tasks.entrySet()) {
long value = 0;
CounterGroup group = entry.getValue().getCounters()
.getCounterGroup($(COUNTER_GROUP));
if(group != null) {
Counter c = group.getCounter($(COUNTER_NAME));
if(c != null) {
value = c.getValue();
}
}
values.put(MRApps.toString(entry.getKey()), value);
}
}
}

View File

@ -0,0 +1,69 @@
/**
* 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.TASK_ID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
import org.apache.hadoop.mapreduce.v2.app.webapp.SingleCounterBlock;
import org.apache.hadoop.yarn.webapp.SubView;
/**
* Render the counters page
*/
public class SingleCounterPage extends AppView {
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
*/
@Override protected void preHead(Page.HTML<_> html) {
commonPreHead(html);
String tid = $(TASK_ID);
String activeNav = "3";
if(tid == null || tid.isEmpty()) {
activeNav = "2";
}
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
set(DATATABLES_ID, "singleCounter");
set(initID(DATATABLES, "singleCounter"), counterTableInit());
setTableStyles(html, "singleCounter");
}
/**
* @return The end of a javascript map that is the jquery datatable
* configuration for the jobs table. the Jobs table is assumed to be
* rendered by the class returned from {@link #content()}
*/
private String counterTableInit() {
return tableInit().
append(",aoColumnDefs:[").
append("{'sType':'title-numeric', 'aTargets': [ 1 ] }").
append("]}").
toString();
}
/**
* The content of this page is the CountersBlock now.
* @return CountersBlock.class
*/
@Override protected Class<? extends SubView> content() {
return SingleCounterBlock.class;
}
}

View File

@ -108,7 +108,7 @@ protected Collection<TaskAttempt> getTaskAttempts() {
@Override protected void preHead(Page.HTML<_> html) {
commonPreHead(html);
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}");
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:3}");
set(DATATABLES_ID, "attempts");
set(initID(DATATABLES, "attempts"), attemptsTableInit());
setTableStyles(html, "attempts");

View File

@ -0,0 +1,76 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app;
import java.io.IOException;
import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.junit.Test;
public class TestMRAppMaster {
@Test
public void testMRAppMasterForDifferentUser() throws IOException,
InterruptedException {
String applicationAttemptIdStr = "appattempt_1317529182569_0004_000001";
String stagingDir = "/tmp/staging";
String userName = "TestAppMasterUser";
ApplicationAttemptId applicationAttemptId = ConverterUtils
.toApplicationAttemptId(applicationAttemptIdStr);
MRAppMasterTest appMaster = new MRAppMasterTest(applicationAttemptId);
YarnConfiguration conf = new YarnConfiguration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
Assert.assertEquals(stagingDir + Path.SEPARATOR + userName + Path.SEPARATOR
+ ".staging", appMaster.stagingDirPath.toString());
}
}
class MRAppMasterTest extends MRAppMaster {
Path stagingDirPath;
private Configuration conf;
public MRAppMasterTest(ApplicationAttemptId applicationAttemptId) {
super(applicationAttemptId);
}
@Override
public void init(Configuration conf) {
this.conf = conf;
}
@Override
public void start() {
try {
String user = UserGroupInformation.getCurrentUser().getShortUserName();
stagingDirPath = MRApps.getStagingAreaDir(conf, user);
} catch (Exception e) {
Assert.fail(e.getMessage());
}
}
}

View File

@ -116,7 +116,7 @@ public void testSimple() throws Exception {
Job mockJob = mock(Job.class);
when(mockJob.getReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING,
0, 0, 0, 0, 0, 0));
0, 0, 0, 0, 0, 0, "jobfile"));
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
appAttemptId, mockJob);
@ -193,7 +193,7 @@ public void testResource() throws Exception {
Job mockJob = mock(Job.class);
when(mockJob.getReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING,
0, 0, 0, 0, 0, 0));
0, 0, 0, 0, 0, 0, "jobfile"));
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
appAttemptId, mockJob);
@ -259,7 +259,7 @@ public void testMapReduceScheduling() throws Exception {
Job mockJob = mock(Job.class);
when(mockJob.getReport()).thenReturn(
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING,
0, 0, 0, 0, 0, 0));
0, 0, 0, 0, 0, 0, "jobfile"));
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
appAttemptId, mockJob);
@ -373,7 +373,7 @@ void setProgress(float setupProgress, float mapProgress,
public JobReport getReport() {
return MRBuilderUtils.newJobReport(this.jobId, "job", "user",
JobState.RUNNING, 0, 0, this.setupProgress, this.mapProgress,
this.reduceProgress, this.cleanupProgress);
this.reduceProgress, this.cleanupProgress, "jobfile");
}
}

View File

@ -36,7 +36,6 @@
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.WebApps;
import org.apache.hadoop.yarn.webapp.test.WebAppTests;
import org.junit.Test;
@ -87,6 +86,7 @@ public Map<JobId, Job> getAllJobs() {
return jobs; // OK
}
@SuppressWarnings("rawtypes")
@Override
public EventHandler getEventHandler() {
return null;
@ -163,6 +163,23 @@ public static Map<String, String> getTaskParams(AppContext appContext) {
new TestAppContext());
}
@Test public void testCountersView() {
AppContext appContext = new TestAppContext();
Map<String, String> params = getJobParams(appContext);
WebAppTests.testPage(CountersPage.class, AppContext.class,
appContext, params);
}
@Test public void testSingleCounterView() {
AppContext appContext = new TestAppContext();
Map<String, String> params = getJobParams(appContext);
params.put(AMParams.COUNTER_GROUP,
"org.apache.hadoop.mapreduce.FileSystemCounter");
params.put(AMParams.COUNTER_NAME, "HDFS_WRITE_OPS");
WebAppTests.testPage(SingleCounterPage.class, AppContext.class,
appContext, params);
}
public static void main(String[] args) {
WebApps.$for("yarn", AppContext.class, new TestAppContext(0, 8, 88, 4)).
at(58888).inDevMode().start(new AMWebApp()).joinThread();

View File

@ -22,6 +22,7 @@
import java.util.HashMap;
import java.util.List;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobPriority;
import org.apache.hadoop.mapred.TaskCompletionEvent;
@ -44,7 +45,7 @@
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueState;
@ -280,16 +281,14 @@ public static Counters toYarn(org.apache.hadoop.mapreduce.Counters counters) {
return yCntrs;
}
public static org.apache.hadoop.mapred.JobStatus fromYarn(
JobReport jobreport, String jobFile) {
public static JobStatus fromYarn(JobReport jobreport, String trackingUrl) {
JobPriority jobPriority = JobPriority.NORMAL;
org.apache.hadoop.mapred.JobStatus jobStatus =
new org.apache.hadoop.mapred.JobStatus(fromYarn(jobreport.getJobId()),
jobreport.getSetupProgress(), jobreport.getMapProgress(),
jobreport.getReduceProgress(), jobreport.getCleanupProgress(),
fromYarn(jobreport.getJobState()),
jobPriority, jobreport.getUser(), jobreport.getJobName(),
jobFile, jobreport.getTrackingUrl());
JobStatus jobStatus = new org.apache.hadoop.mapred.JobStatus(
fromYarn(jobreport.getJobId()), jobreport.getSetupProgress(), jobreport
.getMapProgress(), jobreport.getReduceProgress(), jobreport
.getCleanupProgress(), fromYarn(jobreport.getJobState()),
jobPriority, jobreport.getUser(), jobreport.getJobName(), jobreport
.getJobFile(), trackingUrl);
jobStatus.setFailureInfo(jobreport.getDiagnostics());
return jobStatus;
}
@ -379,13 +378,13 @@ public static List<TaskReport> fromYarn(
return reports;
}
public static JobStatus.State fromYarn(ApplicationState state) {
public static JobStatus.State fromYarn(YarnApplicationState state) {
switch (state) {
case SUBMITTED:
return State.PREP;
case RUNNING:
return State.RUNNING;
case SUCCEEDED:
case FINISHED:
return State.SUCCEEDED;
case FAILED:
return State.FAILED;
@ -418,7 +417,7 @@ public static JobStatus fromYarn(ApplicationReport application,
new JobStatus(
TypeConverter.fromYarn(application.getApplicationId()),
0.0f, 0.0f, 0.0f, 0.0f,
TypeConverter.fromYarn(application.getState()),
TypeConverter.fromYarn(application.getYarnApplicationState()),
org.apache.hadoop.mapreduce.JobPriority.NORMAL,
application.getUser(), application.getName(),
application.getQueue(), jobFile, trackingUrl

View File

@ -31,6 +31,7 @@ public interface JobReport {
public abstract String getJobName();
public abstract String getTrackingUrl();
public abstract String getDiagnostics();
public abstract String getJobFile();
public abstract void setJobId(JobId jobId);
public abstract void setJobState(JobState jobState);
@ -44,4 +45,5 @@ public interface JobReport {
public abstract void setJobName(String jobName);
public abstract void setTrackingUrl(String trackingUrl);
public abstract void setDiagnostics(String diagnostics);
public abstract void setJobFile(String jobFile);
}

View File

@ -230,6 +230,18 @@ public void setDiagnostics(String diagnostics) {
builder.setDiagnostics(diagnostics);
}
@Override
public String getJobFile() {
JobReportProtoOrBuilder p = viaProto ? proto : builder;
return p.getJobFile();
}
@Override
public void setJobFile(String jobFile) {
maybeInitBuilder();
builder.setJobFile(jobFile);
}
private JobIdPBImpl convertFromProtoFormat(JobIdProto p) {
return new JobIdPBImpl(p);
}

View File

@ -55,7 +55,7 @@ public static TaskAttemptId newTaskAttemptId(TaskId taskId, int attemptId) {
public static JobReport newJobReport(JobId jobId, String jobName,
String userName, JobState state, long startTime, long finishTime,
float setupProgress, float mapProgress, float reduceProgress,
float cleanupProgress) {
float cleanupProgress, String jobFile) {
JobReport report = Records.newRecord(JobReport.class);
report.setJobId(jobId);
report.setJobName(jobName);
@ -67,6 +67,7 @@ public static JobReport newJobReport(JobId jobId, String jobName,
report.setCleanupProgress(cleanupProgress);
report.setMapProgress(mapProgress);
report.setReduceProgress(reduceProgress);
report.setJobFile(jobFile);
return report;
}
}

View File

@ -145,6 +145,7 @@ message JobReportProto {
optional string jobName = 10;
optional string trackingUrl = 11;
optional string diagnostics = 12;
optional string jobFile = 13;
}
enum TaskAttemptCompletionEventStatusProto {

View File

@ -21,7 +21,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl;
@ -35,11 +35,11 @@ public class TestTypeConverter {
@Test
public void testFromYarn() throws Exception {
int appStartTime = 612354;
ApplicationState state = ApplicationState.RUNNING;
YarnApplicationState state = YarnApplicationState.RUNNING;
ApplicationId applicationId = new ApplicationIdPBImpl();
ApplicationReportPBImpl applicationReport = new ApplicationReportPBImpl();
applicationReport.setApplicationId(applicationId);
applicationReport.setState(state);
applicationReport.setYarnApplicationState(state);
applicationReport.setStartTime(appStartTime);
applicationReport.setUser("TestTypeConverter-user");
JobStatus jobStatus = TypeConverter.fromYarn(applicationReport, "dummy-jobfile");
@ -56,7 +56,7 @@ public void testFromYarnApplicationReport() {
ApplicationReport mockReport = mock(ApplicationReport.class);
when(mockReport.getTrackingUrl()).thenReturn("dummy-tracking-url");
when(mockReport.getApplicationId()).thenReturn(mockAppId);
when(mockReport.getState()).thenReturn(ApplicationState.KILLED);
when(mockReport.getYarnApplicationState()).thenReturn(YarnApplicationState.KILLED);
when(mockReport.getUser()).thenReturn("dummy-user");
when(mockReport.getQueue()).thenReturn("dummy-queue");
String jobFile = "dummy-path/job.xml";

View File

@ -64,7 +64,8 @@
{"name": "launchTime", "type": "long"},
{"name": "totalMaps", "type": "int"},
{"name": "totalReduces", "type": "int"},
{"name": "jobStatus", "type": "string"}
{"name": "jobStatus", "type": "string"},
{"name": "uberized", "type": "boolean"}
]
},

View File

@ -0,0 +1,241 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import org.apache.hadoop.conf.Configuration;
/**
* A factory to allow applications to deal with inconsistencies between
* MapReduce Context Objects API between hadoop-0.20 and later versions.
*/
public class ContextFactory {
private static final Constructor<?> JOB_CONTEXT_CONSTRUCTOR;
private static final Constructor<?> TASK_CONTEXT_CONSTRUCTOR;
private static final Constructor<?> MAP_CONTEXT_CONSTRUCTOR;
private static final Constructor<?> MAP_CONTEXT_IMPL_CONSTRUCTOR;
private static final boolean useV21;
private static final Field REPORTER_FIELD;
private static final Field READER_FIELD;
private static final Field WRITER_FIELD;
private static final Field OUTER_MAP_FIELD;
private static final Field WRAPPED_CONTEXT_FIELD;
static {
boolean v21 = true;
final String PACKAGE = "org.apache.hadoop.mapreduce";
try {
Class.forName(PACKAGE + ".task.JobContextImpl");
} catch (ClassNotFoundException cnfe) {
v21 = false;
}
useV21 = v21;
Class<?> jobContextCls;
Class<?> taskContextCls;
Class<?> taskIOContextCls;
Class<?> mapCls;
Class<?> mapContextCls;
Class<?> innerMapContextCls;
try {
if (v21) {
jobContextCls =
Class.forName(PACKAGE+".task.JobContextImpl");
taskContextCls =
Class.forName(PACKAGE+".task.TaskAttemptContextImpl");
taskIOContextCls =
Class.forName(PACKAGE+".task.TaskInputOutputContextImpl");
mapContextCls = Class.forName(PACKAGE + ".task.MapContextImpl");
mapCls = Class.forName(PACKAGE + ".lib.map.WrappedMapper");
innerMapContextCls =
Class.forName(PACKAGE+".lib.map.WrappedMapper$Context");
} else {
jobContextCls =
Class.forName(PACKAGE+".JobContext");
taskContextCls =
Class.forName(PACKAGE+".TaskAttemptContext");
taskIOContextCls =
Class.forName(PACKAGE+".TaskInputOutputContext");
mapContextCls = Class.forName(PACKAGE + ".MapContext");
mapCls = Class.forName(PACKAGE + ".Mapper");
innerMapContextCls =
Class.forName(PACKAGE+".Mapper$Context");
}
} catch (ClassNotFoundException e) {
throw new IllegalArgumentException("Can't find class", e);
}
try {
JOB_CONTEXT_CONSTRUCTOR =
jobContextCls.getConstructor(Configuration.class, JobID.class);
JOB_CONTEXT_CONSTRUCTOR.setAccessible(true);
TASK_CONTEXT_CONSTRUCTOR =
taskContextCls.getConstructor(Configuration.class,
TaskAttemptID.class);
TASK_CONTEXT_CONSTRUCTOR.setAccessible(true);
if (useV21) {
MAP_CONTEXT_CONSTRUCTOR =
innerMapContextCls.getConstructor(mapCls,
MapContext.class);
MAP_CONTEXT_IMPL_CONSTRUCTOR =
mapContextCls.getDeclaredConstructor(Configuration.class,
TaskAttemptID.class,
RecordReader.class,
RecordWriter.class,
OutputCommitter.class,
StatusReporter.class,
InputSplit.class);
MAP_CONTEXT_IMPL_CONSTRUCTOR.setAccessible(true);
WRAPPED_CONTEXT_FIELD =
innerMapContextCls.getDeclaredField("mapContext");
WRAPPED_CONTEXT_FIELD.setAccessible(true);
} else {
MAP_CONTEXT_CONSTRUCTOR =
innerMapContextCls.getConstructor(mapCls,
Configuration.class,
TaskAttemptID.class,
RecordReader.class,
RecordWriter.class,
OutputCommitter.class,
StatusReporter.class,
InputSplit.class);
MAP_CONTEXT_IMPL_CONSTRUCTOR = null;
WRAPPED_CONTEXT_FIELD = null;
}
MAP_CONTEXT_CONSTRUCTOR.setAccessible(true);
REPORTER_FIELD = taskIOContextCls.getDeclaredField("reporter");
REPORTER_FIELD.setAccessible(true);
READER_FIELD = mapContextCls.getDeclaredField("reader");
READER_FIELD.setAccessible(true);
WRITER_FIELD = taskIOContextCls.getDeclaredField("output");
WRITER_FIELD.setAccessible(true);
OUTER_MAP_FIELD = innerMapContextCls.getDeclaredField("this$0");
OUTER_MAP_FIELD.setAccessible(true);
} catch (SecurityException e) {
throw new IllegalArgumentException("Can't run constructor ", e);
} catch (NoSuchMethodException e) {
throw new IllegalArgumentException("Can't find constructor ", e);
} catch (NoSuchFieldException e) {
throw new IllegalArgumentException("Can't find field ", e);
}
}
/**
* Clone a job or task attempt context with a new configuration.
* @param original the original context
* @param conf the new configuration
* @return a new context object
* @throws InterruptedException
* @throws IOException
*/
@SuppressWarnings("unchecked")
public static JobContext cloneContext(JobContext original,
Configuration conf
) throws IOException,
InterruptedException {
try {
if (original instanceof MapContext<?,?,?,?>) {
return cloneMapContext((Mapper.Context) original, conf, null, null);
} else if (original instanceof ReduceContext<?,?,?,?>) {
throw new IllegalArgumentException("can't clone ReduceContext");
} else if (original instanceof TaskAttemptContext) {
TaskAttemptContext spec = (TaskAttemptContext) original;
return (JobContext)
TASK_CONTEXT_CONSTRUCTOR.newInstance(conf, spec.getTaskAttemptID());
} else {
return (JobContext)
JOB_CONTEXT_CONSTRUCTOR.newInstance(conf, original.getJobID());
}
} catch (InstantiationException e) {
throw new IllegalArgumentException("Can't clone object", e);
} catch (IllegalAccessException e) {
throw new IllegalArgumentException("Can't clone object", e);
} catch (InvocationTargetException e) {
throw new IllegalArgumentException("Can't clone object", e);
}
}
/**
* Copy a mapper context, optionally replacing the input and output.
* @param <K1> input key type
* @param <V1> input value type
* @param <K2> output key type
* @param <V2> output value type
* @param context the context to clone
* @param conf a new configuration
* @param reader Reader to read from. Null means to clone from context.
* @param writer Writer to write to. Null means to clone from context.
* @return a new context. it will not be the same class as the original.
* @throws IOException
* @throws InterruptedException
*/
@SuppressWarnings("unchecked")
public static <K1,V1,K2,V2> Mapper<K1,V1,K2,V2>.Context
cloneMapContext(MapContext<K1,V1,K2,V2> context,
Configuration conf,
RecordReader<K1,V1> reader,
RecordWriter<K2,V2> writer
) throws IOException, InterruptedException {
try {
// get the outer object pointer
Object outer = OUTER_MAP_FIELD.get(context);
// if it is a wrapped 21 context, unwrap it
if ("org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context".equals
(context.getClass().getName())) {
context = (MapContext<K1,V1,K2,V2>) WRAPPED_CONTEXT_FIELD.get(context);
}
// if the reader or writer aren't given, use the same ones
if (reader == null) {
reader = (RecordReader<K1,V1>) READER_FIELD.get(context);
}
if (writer == null) {
writer = (RecordWriter<K2,V2>) WRITER_FIELD.get(context);
}
if (useV21) {
Object basis =
MAP_CONTEXT_IMPL_CONSTRUCTOR.newInstance(conf,
context.getTaskAttemptID(),
reader, writer,
context.getOutputCommitter(),
REPORTER_FIELD.get(context),
context.getInputSplit());
return (Mapper.Context)
MAP_CONTEXT_CONSTRUCTOR.newInstance(outer, basis);
} else {
return (Mapper.Context)
MAP_CONTEXT_CONSTRUCTOR.newInstance(outer,
conf, context.getTaskAttemptID(),
reader, writer,
context.getOutputCommitter(),
REPORTER_FIELD.get(context),
context.getInputSplit());
}
} catch (IllegalAccessException e) {
throw new IllegalArgumentException("Can't access field", e);
} catch (InstantiationException e) {
throw new IllegalArgumentException("Can't create object", e);
} catch (InvocationTargetException e) {
throw new IllegalArgumentException("Can't invoke constructor", e);
}
}
}

View File

@ -462,8 +462,6 @@ public String toString() {
sb.append(status.getReduceProgress()).append("\n");
sb.append("Job state: ");
sb.append(status.getState()).append("\n");
sb.append("history URL: ");
sb.append(status.getHistoryFile()).append("\n");
sb.append("retired: ").append(status.isRetired()).append("\n");
sb.append("reason for failure: ").append(reasonforFailure);
return sb.toString();

View File

@ -473,4 +473,6 @@ public interface MRJobConfig {
public static final String MAPREDUCE_V2_CHILD_CLASS =
"org.apache.hadoop.mapred.YarnChild";
public static final String APPLICATION_ATTEMPT_ID =
"mapreduce.job.application.attempt.id";
}

View File

@ -143,4 +143,35 @@ public abstract void commitTask(TaskAttemptContext taskContext)
*/
public abstract void abortTask(TaskAttemptContext taskContext)
throws IOException;
/**
* Is task output recovery supported for restarting jobs?
*
* If task output recovery is supported, job restart can be done more
* efficiently.
*
* @return <code>true</code> if task output recovery is supported,
* <code>false</code> otherwise
* @see #recoverTask(TaskAttemptContext)
*/
public boolean isRecoverySupported() {
return false;
}
/**
* Recover the task output.
*
* The retry-count for the job will be passed via the
* {@link MRJobConfig#APPLICATION_ATTEMPT_ID} key in
* {@link TaskAttemptContext#getConfiguration()} for the
* <code>OutputCommitter</code>.
*
* If an exception is thrown the task will be attempted again.
*
* @param taskContext Context of the task whose output is being recovered
* @throws IOException
*/
public void recoverTask(TaskAttemptContext taskContext)
throws IOException
{}
}

View File

@ -302,6 +302,7 @@ private void handleJobInitedEvent(JobInitedEvent event) {
info.launchTime = event.getLaunchTime();
info.totalMaps = event.getTotalMaps();
info.totalReduces = event.getTotalReduces();
info.uberized = event.getUberized();
}
private void handleJobInfoChangeEvent(JobInfoChangeEvent event) {
@ -346,6 +347,7 @@ public static class JobInfo {
Map<JobACL, AccessControlList> jobACLs;
Map<TaskID, TaskInfo> tasksMap;
boolean uberized;
/** Create a job info object where job information will be stored
* after a parse
@ -373,6 +375,7 @@ public void printAll() {
System.out.println("MAP_COUNTERS:" + mapCounters.toString());
System.out.println("REDUCE_COUNTERS:" + reduceCounters.toString());
System.out.println("TOTAL_COUNTERS: " + totalCounters.toString());
System.out.println("UBERIZED: " + uberized);
for (TaskInfo ti: tasksMap.values()) {
ti.printAll();
@ -421,6 +424,8 @@ public void printAll() {
/** @return the priority of this job */
public String getPriority() { return priority.toString(); }
public Map<JobACL, AccessControlList> getJobACLs() { return jobACLs; }
/** @return the uberized status of this job */
public boolean getUberized() { return uberized; }
}
/**

View File

@ -42,14 +42,16 @@ public class JobInitedEvent implements HistoryEvent {
* @param totalMaps
* @param totalReduces
* @param jobStatus
* @param uberized True if the job's map and reduce stages were combined
*/
public JobInitedEvent(JobID id, long launchTime, int totalMaps,
int totalReduces, String jobStatus) {
int totalReduces, String jobStatus, boolean uberized) {
datum.jobid = new Utf8(id.toString());
datum.launchTime = launchTime;
datum.totalMaps = totalMaps;
datum.totalReduces = totalReduces;
datum.jobStatus = new Utf8(jobStatus);
datum.uberized = uberized;
}
JobInitedEvent() { }
@ -71,5 +73,6 @@ public JobInitedEvent(JobID id, long launchTime, int totalMaps,
public EventType getEventType() {
return EventType.JOB_INITED;
}
/** Get whether the job's map and reduce stages were combined */
public boolean getUberized() { return datum.uberized; }
}

View File

@ -35,7 +35,6 @@
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.util.StringUtils;
/** An {@link OutputCommitter} that commits files specified
* in job output directory i.e. ${mapreduce.output.fileoutputformat.outputdir}.
@ -69,9 +68,8 @@ public FileOutputCommitter(Path outputPath,
this.outputPath = outputPath;
outputFileSystem = outputPath.getFileSystem(context.getConfiguration());
workPath = new Path(outputPath,
(FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR +
"_" + context.getTaskAttemptID().toString()
)).makeQualified(outputFileSystem);
getTaskAttemptBaseDirName(context))
.makeQualified(outputFileSystem);
}
}
@ -82,7 +80,8 @@ public FileOutputCommitter(Path outputPath,
*/
public void setupJob(JobContext context) throws IOException {
if (outputPath != null) {
Path tmpDir = new Path(outputPath, FileOutputCommitter.TEMP_DIR_NAME);
Path tmpDir = new Path(outputPath, getJobAttemptBaseDirName(context) +
Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME);
FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration());
if (!fileSys.mkdirs(tmpDir)) {
LOG.error("Mkdirs failed to create " + tmpDir.toString());
@ -106,11 +105,27 @@ private void markOutputDirSuccessful(MRJobConfig context) throws IOException {
}
/**
* Move all job output to the final place.
* Delete the temporary directory, including all of the work directories.
* Create a _SUCCESS file to make it as successful.
* @param context the job's context
*/
public void commitJob(JobContext context) throws IOException {
//delete the task temp directory from the current jobtempdir
Path tmpDir = new Path(outputPath, getJobAttemptBaseDirName(context) +
Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME);
FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration());
if (fileSys.exists(tmpDir)) {
fileSys.delete(tmpDir, true);
} else {
LOG.warn("Task temp dir could not be deleted " + tmpDir);
}
//move the job output to final place
Path jobOutputPath =
new Path(outputPath, getJobAttemptBaseDirName(context));
moveJobOutputs(outputFileSystem, outputPath, jobOutputPath);
// delete the _temporary folder and create a _done file in the o/p folder
cleanupJob(context);
if (shouldMarkOutputDir(context.getConfiguration())) {
@ -118,6 +133,31 @@ public void commitJob(JobContext context) throws IOException {
}
}
private void moveJobOutputs(FileSystem fs,
Path finalOutputDir, Path jobOutput) throws IOException {
if (fs.isFile(jobOutput)) {
Path finalOutputPath = getFinalPath(finalOutputDir, jobOutput, jobOutput);
if (!fs.rename(jobOutput, finalOutputPath)) {
if (!fs.delete(finalOutputPath, true)) {
throw new IOException("Failed to delete earlier output of job");
}
if (!fs.rename(jobOutput, finalOutputPath)) {
throw new IOException("Failed to save output of job");
}
}
LOG.debug("Moved " + jobOutput + " to " + finalOutputPath);
} else if (fs.getFileStatus(jobOutput).isDirectory()) {
FileStatus[] paths = fs.listStatus(jobOutput);
Path finalOutputPath = getFinalPath(finalOutputDir, jobOutput, jobOutput);
fs.mkdirs(finalOutputPath);
if (paths != null) {
for (FileStatus path : paths) {
moveJobOutputs(fs, finalOutputDir, path.getPath());
}
}
}
}
@Override
@Deprecated
public void cleanupJob(JobContext context) throws IOException {
@ -163,8 +203,10 @@ public void commitTask(TaskAttemptContext context)
if (workPath != null) {
context.progress();
if (outputFileSystem.exists(workPath)) {
// Move the task outputs to their final place
moveTaskOutputs(context, outputFileSystem, outputPath, workPath);
// Move the task outputs to the current job attempt output dir
Path jobOutputPath =
new Path(outputPath, getJobAttemptBaseDirName(context));
moveTaskOutputs(context, outputFileSystem, jobOutputPath, workPath);
// Delete the temporary task-specific output directory
if (!outputFileSystem.delete(workPath, true)) {
LOG.warn("Failed to delete the temporary output" +
@ -271,4 +313,50 @@ public boolean needsTaskCommit(TaskAttemptContext context
public Path getWorkPath() throws IOException {
return workPath;
}
@Override
public boolean isRecoverySupported() {
return true;
}
@Override
public void recoverTask(TaskAttemptContext context)
throws IOException {
context.progress();
Path jobOutputPath =
new Path(outputPath, getJobAttemptBaseDirName(context));
int previousAttempt =
context.getConfiguration().getInt(
MRJobConfig.APPLICATION_ATTEMPT_ID, 0) - 1;
if (previousAttempt < 0) {
throw new IOException ("Cannot recover task output for first attempt...");
}
Path pathToRecover =
new Path(outputPath, getJobAttemptBaseDirName(previousAttempt));
if (outputFileSystem.exists(pathToRecover)) {
// Move the task outputs to their final place
moveJobOutputs(outputFileSystem, jobOutputPath, pathToRecover);
LOG.info("Saved output of job to " + jobOutputPath);
}
}
protected static String getJobAttemptBaseDirName(JobContext context) {
int appAttemptId =
context.getConfiguration().getInt(
MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
return getJobAttemptBaseDirName(appAttemptId);
}
protected static String getJobAttemptBaseDirName(int appAttemptId) {
return FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR +
+ appAttemptId;
}
protected static String getTaskAttemptBaseDirName(
TaskAttemptContext context) {
return getJobAttemptBaseDirName(context) + Path.SEPARATOR +
FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR +
"_" + context.getTaskAttemptID().toString();
}
}

View File

@ -114,15 +114,15 @@ SplitMetaInfo[] writeNewSplits(Configuration conf,
if (array.length != 0) {
SerializationFactory factory = new SerializationFactory(conf);
int i = 0;
long offset = out.size();
long offset = out.getPos();
for(T split: array) {
int prevCount = out.size();
long prevCount = out.getPos();
Text.writeString(out, split.getClass().getName());
Serializer<T> serializer =
factory.getSerializer((Class<T>) split.getClass());
serializer.open(out);
serializer.serialize(split);
int currCount = out.size();
long currCount = out.getPos();
info[i++] =
new JobSplit.SplitMetaInfo(
split.getLocations(), offset,
@ -139,12 +139,12 @@ private static SplitMetaInfo[] writeOldSplits(
SplitMetaInfo[] info = new SplitMetaInfo[splits.length];
if (splits.length != 0) {
int i = 0;
long offset = out.size();
long offset = out.getPos();
for(org.apache.hadoop.mapred.InputSplit split: splits) {
int prevLen = out.size();
long prevLen = out.getPos();
Text.writeString(out, split.getClass().getName());
split.write(out);
int currLen = out.size();
long currLen = out.getPos();
info[i++] = new JobSplit.SplitMetaInfo(
split.getLocations(), offset,
split.getLength());

View File

@ -1174,7 +1174,7 @@
<property>
<name>yarn.app.mapreduce.am.staging-dir</name>
<value>/tmp/hadoop-yarn/${user.name}/staging</value>
<value>/tmp/hadoop-yarn/staging</value>
<description>The staging dir used while submitting jobs.
</description>
</property>

View File

@ -0,0 +1,290 @@
/**
* 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.lib.output;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.net.URI;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RawLocalFileSystem;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.JobStatus;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.task.JobContextImpl;
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
@SuppressWarnings("unchecked")
public class TestFileOutputCommitter extends TestCase {
private static Path outDir = new Path(System.getProperty("test.build.data",
"/tmp"), "output");
// A random task attempt id for testing.
private static String attempt = "attempt_200707121733_0001_m_000000_0";
private static String partFile = "part-m-00000";
private static TaskAttemptID taskID = TaskAttemptID.forName(attempt);
private Text key1 = new Text("key1");
private Text key2 = new Text("key2");
private Text val1 = new Text("val1");
private Text val2 = new Text("val2");
private void writeOutput(RecordWriter theRecordWriter,
TaskAttemptContext context) throws IOException, InterruptedException {
NullWritable nullWritable = NullWritable.get();
try {
theRecordWriter.write(key1, val1);
theRecordWriter.write(null, nullWritable);
theRecordWriter.write(null, val1);
theRecordWriter.write(nullWritable, val2);
theRecordWriter.write(key2, nullWritable);
theRecordWriter.write(key1, null);
theRecordWriter.write(null, null);
theRecordWriter.write(key2, val2);
} finally {
theRecordWriter.close(context);
}
}
public void testRecovery() throws Exception {
Job job = Job.getInstance();
FileOutputFormat.setOutputPath(job, outDir);
Configuration conf = job.getConfiguration();
conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1);
JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
FileOutputCommitter committer = new FileOutputCommitter(outDir, tContext);
// setup
committer.setupJob(jContext);
committer.setupTask(tContext);
// write output
TextOutputFormat theOutputFormat = new TextOutputFormat();
RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(tContext);
writeOutput(theRecordWriter, tContext);
// do commit
committer.commitTask(tContext);
Path jobTempDir1 = new Path(outDir,
FileOutputCommitter.getJobAttemptBaseDirName(
conf.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0)));
assertTrue((new File(jobTempDir1.toString()).exists()));
validateContent(jobTempDir1);
//now while running the second app attempt,
//recover the task output from first attempt
Configuration conf2 = job.getConfiguration();
conf2.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
conf2.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 2);
JobContext jContext2 = new JobContextImpl(conf2, taskID.getJobID());
TaskAttemptContext tContext2 = new TaskAttemptContextImpl(conf2, taskID);
FileOutputCommitter committer2 = new FileOutputCommitter(outDir, tContext2);
committer.setupJob(tContext2);
Path jobTempDir2 = new Path(outDir,
FileOutputCommitter.getJobAttemptBaseDirName(
conf2.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0)));
assertTrue((new File(jobTempDir2.toString()).exists()));
tContext2.getConfiguration().setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 2);
committer2.recoverTask(tContext2);
validateContent(jobTempDir2);
committer2.commitJob(jContext2);
validateContent(outDir);
FileUtil.fullyDelete(new File(outDir.toString()));
}
private void validateContent(Path dir) throws IOException {
File expectedFile = new File(new Path(dir, partFile).toString());
StringBuffer expectedOutput = new StringBuffer();
expectedOutput.append(key1).append('\t').append(val1).append("\n");
expectedOutput.append(val1).append("\n");
expectedOutput.append(val2).append("\n");
expectedOutput.append(key2).append("\n");
expectedOutput.append(key1).append("\n");
expectedOutput.append(key2).append('\t').append(val2).append("\n");
String output = slurp(expectedFile);
assertEquals(output, expectedOutput.toString());
}
public void testCommitter() throws Exception {
Job job = Job.getInstance();
FileOutputFormat.setOutputPath(job, outDir);
Configuration conf = job.getConfiguration();
conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
FileOutputCommitter committer = new FileOutputCommitter(outDir, tContext);
// setup
committer.setupJob(jContext);
committer.setupTask(tContext);
// write output
TextOutputFormat theOutputFormat = new TextOutputFormat();
RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(tContext);
writeOutput(theRecordWriter, tContext);
// do commit
committer.commitTask(tContext);
committer.commitJob(jContext);
// validate output
validateContent(outDir);
FileUtil.fullyDelete(new File(outDir.toString()));
}
public void testAbort() throws IOException, InterruptedException {
Job job = Job.getInstance();
FileOutputFormat.setOutputPath(job, outDir);
Configuration conf = job.getConfiguration();
conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
FileOutputCommitter committer = new FileOutputCommitter(outDir, tContext);
// do setup
committer.setupJob(jContext);
committer.setupTask(tContext);
// write output
TextOutputFormat theOutputFormat = new TextOutputFormat();
RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(tContext);
writeOutput(theRecordWriter, tContext);
// do abort
committer.abortTask(tContext);
File expectedFile = new File(new Path(committer.getWorkPath(), partFile)
.toString());
assertFalse("task temp dir still exists", expectedFile.exists());
committer.abortJob(jContext, JobStatus.State.FAILED);
expectedFile = new File(new Path(outDir, FileOutputCommitter.TEMP_DIR_NAME)
.toString());
assertFalse("job temp dir still exists", expectedFile.exists());
assertEquals("Output directory not empty", 0, new File(outDir.toString())
.listFiles().length);
FileUtil.fullyDelete(new File(outDir.toString()));
}
public static class FakeFileSystem extends RawLocalFileSystem {
public FakeFileSystem() {
super();
}
public URI getUri() {
return URI.create("faildel:///");
}
@Override
public boolean delete(Path p, boolean recursive) throws IOException {
throw new IOException("fake delete failed");
}
}
public void testFailAbort() throws IOException, InterruptedException {
Job job = Job.getInstance();
Configuration conf = job.getConfiguration();
conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "faildel:///");
conf.setClass("fs.faildel.impl", FakeFileSystem.class, FileSystem.class);
conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1);
FileOutputFormat.setOutputPath(job, outDir);
JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
FileOutputCommitter committer = new FileOutputCommitter(outDir, tContext);
// do setup
committer.setupJob(jContext);
committer.setupTask(tContext);
// write output
TextOutputFormat<?, ?> theOutputFormat = new TextOutputFormat();
RecordWriter<?, ?> theRecordWriter = theOutputFormat
.getRecordWriter(tContext);
writeOutput(theRecordWriter, tContext);
// do abort
Throwable th = null;
try {
committer.abortTask(tContext);
} catch (IOException ie) {
th = ie;
}
assertNotNull(th);
assertTrue(th instanceof IOException);
assertTrue(th.getMessage().contains("fake delete failed"));
File jobTmpDir = new File(new Path(outDir,
FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR +
conf.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0) +
Path.SEPARATOR +
FileOutputCommitter.TEMP_DIR_NAME).toString());
File taskTmpDir = new File(jobTmpDir, "_" + taskID);
File expectedFile = new File(taskTmpDir, partFile);
assertTrue(expectedFile + " does not exists", expectedFile.exists());
th = null;
try {
committer.abortJob(jContext, JobStatus.State.FAILED);
} catch (IOException ie) {
th = ie;
}
assertNotNull(th);
assertTrue(th instanceof IOException);
assertTrue(th.getMessage().contains("fake delete failed"));
assertTrue("job temp dir does not exists", jobTmpDir.exists());
FileUtil.fullyDelete(new File(outDir.toString()));
}
public static String slurp(File f) throws IOException {
int len = (int) f.length();
byte[] buf = new byte[len];
FileInputStream in = new FileInputStream(f);
String contents = null;
try {
in.read(buf, 0, len);
contents = new String(buf, "UTF-8");
} finally {
in.close();
}
return contents;
}
}

View File

@ -48,6 +48,7 @@
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.jobhistory.JobHistoryUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.YarnException;
@ -96,9 +97,11 @@ public CompletedJob(Configuration conf, JobId jobId, Path historyFile,
report.setFinishTime(jobInfo.getFinishTime());
report.setJobName(jobInfo.getJobname());
report.setUser(jobInfo.getUsername());
//TODO Possibly populate job progress. Never used.
//report.setMapProgress(progress)
//report.setReduceProgress(progress)
report.setMapProgress((float) getCompletedMaps() / getTotalMaps());
report.setReduceProgress((float) getCompletedReduces() / getTotalReduces());
report.setJobFile(confFile.toString());
report.setTrackingUrl(JobHistoryUtils.getHistoryUrl(conf, TypeConverter
.toYarn(TypeConverter.fromYarn(jobId)).getAppId()));
}
@Override
@ -287,8 +290,7 @@ public int getTotalReduces() {
@Override
public boolean isUber() {
LOG.warn("isUber is not yet implemented");
return false;
return jobInfo.getUberized();
}
@Override

View File

@ -101,12 +101,9 @@ public HistoryClientService(HistoryContext history) {
}
public void start() {
YarnRPC rpc = YarnRPC.create(getConfig());
Configuration conf = new Configuration(getConfig());
conf.setClass(
YarnConfiguration.YARN_SECURITY_INFO,
ClientHSSecurityInfo.class, SecurityInfo.class);
initializeWebApp(getConfig());
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
initializeWebApp(conf);
String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
InetSocketAddress address = NetUtils.createSocketAddr(serviceAddr);

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.app.webapp.App;
import org.apache.hadoop.mapreduce.v2.app.webapp.AppController;
@ -57,7 +59,7 @@ protected Class<? extends View> jobPage() {
* @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#countersPage()
*/
@Override
protected Class<? extends View> countersPage() {
public Class<? extends View> countersPage() {
return HsCountersPage.class;
}
@ -109,6 +111,15 @@ public void jobCounters() {
super.jobCounters();
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#taskCounters()
*/
@Override
public void taskCounters() {
super.taskCounters();
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#tasks()
@ -157,4 +168,31 @@ protected Class<? extends View> aboutPage() {
public void about() {
render(aboutPage());
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#singleCounterPage()
*/
@Override
protected Class<? extends View> singleCounterPage() {
return HsSingleCounterPage.class;
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#singleJobCounter()
*/
@Override
public void singleJobCounter() throws IOException{
super.singleJobCounter();
}
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#singleTaskCounter()
*/
@Override
public void singleTaskCounter() throws IOException{
super.singleTaskCounter();
}
}

View File

@ -18,11 +18,12 @@
package org.apache.hadoop.mapreduce.v2.hs.webapp;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
import org.apache.hadoop.mapreduce.v2.app.webapp.CountersBlock;
import org.apache.hadoop.yarn.webapp.SubView;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
/**
* Render the counters page
*/
@ -34,7 +35,12 @@ public class HsCountersPage extends HsView {
*/
@Override protected void preHead(Page.HTML<_> html) {
commonPreHead(html);
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
String tid = $(TASK_ID);
String activeNav = "2";
if(tid == null || tid.isEmpty()) {
activeNav = "1";
}
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
set(DATATABLES_SELECTOR, "#counters .dt-counters");
set(initSelector(DATATABLES),
"{bJQueryUI:true, sDom:'t', iDisplayLength:-1}");
@ -47,9 +53,9 @@ public class HsCountersPage extends HsView {
@Override protected void postHead(Page.HTML<_> html) {
html.
style("#counters, .dt-counters { table-layout: fixed }",
"#counters th { overflow: hidden; vertical-align: center }",
"#counters th { overflow: hidden; vertical-align: middle }",
"#counters .dataTables_wrapper { min-height: 1em }",
"#counters .group { width: 10em }",
"#counters .group { width: 15em }",
"#counters .name { width: 30em }");
}

View File

@ -55,6 +55,14 @@ public class HsNavBlock extends HtmlBlock {
li().a(url("conf", jobid), "Configuration")._().
li().a(url("tasks", jobid, "m"), "Map tasks")._().
li().a(url("tasks", jobid, "r"), "Reduce tasks")._()._();
if (app.getTask() != null) {
String taskid = MRApps.toString(app.getTask().getID());
nav.
h3("Task").
ul().
li().a(url("task", taskid), "Task Overview")._().
li().a(url("taskcounters", taskid), "Counters")._()._();
}
}
nav.
h3("Tools").

View File

@ -0,0 +1,69 @@
/**
* 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_ID;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
import org.apache.hadoop.mapreduce.v2.app.webapp.SingleCounterBlock;
import org.apache.hadoop.yarn.webapp.SubView;
/**
* Render the counters page
*/
public class HsSingleCounterPage extends HsView {
/*
* (non-Javadoc)
* @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
*/
@Override protected void preHead(Page.HTML<_> html) {
commonPreHead(html);
String tid = $(TASK_ID);
String activeNav = "2";
if(tid == null || tid.isEmpty()) {
activeNav = "1";
}
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
set(DATATABLES_ID, "singleCounter");
set(initID(DATATABLES, "singleCounter"), counterTableInit());
setTableStyles(html, "singleCounter");
}
/**
* @return The end of a javascript map that is the jquery datatable
* configuration for the jobs table. the Jobs table is assumed to be
* rendered by the class returned from {@link #content()}
*/
private String counterTableInit() {
return tableInit().
append(", aoColumnDefs:[").
append("{'sType':'title-numeric', 'aTargets': [ 1 ] }").
append("]}").
toString();
}
/**
* The content of this page is the CountersBlock now.
* @return CountersBlock.class
*/
@Override protected Class<? extends SubView> content() {
return SingleCounterBlock.class;
}
}

View File

@ -250,7 +250,7 @@ protected Collection<TaskAttempt> getTaskAttempts() {
@Override protected void preHead(Page.HTML<_> html) {
commonPreHead(html);
//override the nav config from commonPReHead
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}");
//Set up the java script and CSS for the attempts table
set(DATATABLES_ID, "attempts");
set(initID(DATATABLES, "attempts"), attemptsTableInit());

View File

@ -41,10 +41,15 @@ public void setup() {
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("/singlejobcounter",JOB_ID, COUNTER_GROUP, COUNTER_NAME),
HsController.class, "singleJobCounter");
route(pajoin("/tasks", JOB_ID, TASK_TYPE), HsController.class, "tasks");
route(pajoin("/attempts", JOB_ID, TASK_TYPE, ATTEMPT_STATE),
HsController.class, "attempts");
route(pajoin("/task", TASK_ID), HsController.class, "task");
route(pajoin("/taskcounters", TASK_ID), HsController.class, "taskCounters");
route(pajoin("/singletaskcounter",TASK_ID, COUNTER_GROUP, COUNTER_NAME),
HsController.class, "singleTaskCounter");
route("/about", HsController.class, "about");
}
}

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