HDFS-7959. WebHdfs logging is missing on Datanode (Kihwal Lee via sjlee)

(cherry picked from commit ae90d4dd90)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java
This commit is contained in:
Kihwal Lee 2016-08-19 15:32:11 -05:00
parent dd6c149c4a
commit 4f3df8e22a
2 changed files with 43 additions and 8 deletions

View File

@ -300,6 +300,19 @@ log4j.appender.RMSUMMARY.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
#log4j.appender.nodemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-nodemanager-yyyy_mm_dd.log
#log4j.appender.nodemanagerrequestlog.RetainDays=3
# WebHdfs request log on datanodes
# Specify -Ddatanode.webhdfs.logger=INFO,HTTPDRFA on datanode startup to
# direct the log to a separate file.
#datanode.webhdfs.logger=INFO,console
#log4j.logger.datanode.webhdfs=${datanode.webhdfs.logger}
#log4j.appender.HTTPDRFA=org.apache.log4j.DailyRollingFileAppender
#log4j.appender.HTTPDRFA.File=${hadoop.log.dir}/hadoop-datanode-webhdfs.log
#log4j.appender.HTTPDRFA.layout=org.apache.log4j.PatternLayout
#log4j.appender.HTTPDRFA.layout.ConversionPattern=%d{ISO8601} %m%n
#log4j.appender.HTTPDRFA.DatePattern=.yyyy-MM-dd
# Appender for viewing information for errors and warnings
yarn.ewma.cleanupInterval=300
yarn.ewma.messageAgeLimitSeconds=86400

View File

@ -29,6 +29,7 @@
import static io.netty.handler.codec.http.HttpMethod.PUT;
import static io.netty.handler.codec.http.HttpResponseStatus.CONTINUE;
import static io.netty.handler.codec.http.HttpResponseStatus.CREATED;
import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
import static io.netty.handler.codec.http.HttpResponseStatus.OK;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HDFS_URI_SCHEME;
@ -48,6 +49,7 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
@ -78,6 +80,7 @@
public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
static final Log LOG = LogFactory.getLog(WebHdfsHandler.class);
static final Log REQLOG = LogFactory.getLog("datanode.webhdfs");
public static final String WEBHDFS_PREFIX = WebHdfsFileSystem.PATH_PREFIX;
public static final int WEBHDFS_PREFIX_LENGTH = WEBHDFS_PREFIX.length();
public static final String APPLICATION_OCTET_STREAM =
@ -94,6 +97,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
private String path;
private ParameterParser params;
private UserGroupInformation ugi;
private DefaultHttpResponse resp = null;
public WebHdfsHandler(Configuration conf, Configuration confForCreate)
throws IOException {
@ -115,12 +119,30 @@ public void channelRead0(final ChannelHandlerContext ctx,
ugi.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
handle(ctx, req);
try {
handle(ctx, req);
} finally {
String host = null;
try {
host = ((InetSocketAddress)ctx.channel().remoteAddress()).
getAddress().getHostAddress();
} catch (Exception e) {
LOG.warn("Error retrieving hostname: ", e);
host = "unknown";
}
REQLOG.info(host + " " + req.method() + " " + req.uri() + " " +
getResponseCode());
}
return null;
}
});
}
int getResponseCode() {
return (resp == null) ? INTERNAL_SERVER_ERROR.code() :
resp.status().code();
}
public void handle(ChannelHandlerContext ctx, HttpRequest req)
throws IOException, URISyntaxException {
String op = params.op();
@ -145,7 +167,7 @@ public void handle(ChannelHandlerContext ctx, HttpRequest req)
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
LOG.debug("Error ", cause);
DefaultHttpResponse resp = ExceptionHandler.exceptionCaught(cause);
resp = ExceptionHandler.exceptionCaught(cause);
resp.headers().set(CONNECTION, CLOSE);
ctx.writeAndFlush(resp).addListener(ChannelFutureListener.CLOSE);
}
@ -177,7 +199,7 @@ private void onCreate(ChannelHandlerContext ctx)
path, permission, flags, createParent, replication, blockSize, null,
bufferSize, null), null);
DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1, CREATED);
resp = new DefaultHttpResponse(HTTP_1_1, CREATED);
final URI uri = new URI(HDFS_URI_SCHEME, nnId, path, null, null);
resp.headers().set(LOCATION, uri.toString());
@ -194,7 +216,7 @@ private void onAppend(ChannelHandlerContext ctx) throws IOException {
DFSClient dfsClient = newDfsClient(nnId, conf);
OutputStream out = dfsClient.append(path, bufferSize,
EnumSet.of(CreateFlag.APPEND), null, null);
DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1, OK);
resp = new DefaultHttpResponse(HTTP_1_1, OK);
resp.headers().set(CONTENT_LENGTH, 0);
ctx.pipeline().replace(this, HdfsWriter.class.getSimpleName(),
new HdfsWriter(dfsClient, out, resp));
@ -206,8 +228,8 @@ private void onOpen(ChannelHandlerContext ctx) throws IOException {
final long offset = params.offset();
final long length = params.length();
DefaultHttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
HttpHeaders headers = response.headers();
resp = new DefaultHttpResponse(HTTP_1_1, OK);
HttpHeaders headers = resp.headers();
// Allow the UI to access the file
headers.set(ACCESS_CONTROL_ALLOW_METHODS, GET);
headers.set(ACCESS_CONTROL_ALLOW_ORIGIN, "*");
@ -231,7 +253,7 @@ private void onOpen(ChannelHandlerContext ctx) throws IOException {
data = in;
}
ctx.write(response);
ctx.write(resp);
ctx.writeAndFlush(new ChunkedStream(data) {
@Override
public void close() throws Exception {
@ -254,7 +276,7 @@ private void onGetFileChecksum(ChannelHandlerContext ctx) throws IOException {
}
final byte[] js =
JsonUtil.toJsonString(checksum).getBytes(StandardCharsets.UTF_8);
DefaultFullHttpResponse resp =
resp =
new DefaultFullHttpResponse(HTTP_1_1, OK, Unpooled.wrappedBuffer(js));
resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8);