Repository: hadoop Updated Branches: refs/heads/branch-2.7 dc57c1d18 -> 53a72f634
HDFS-7959. WebHdfs logging is missing on Datanode (Kihwal Lee via sjlee) (cherry picked from commit ae90d4dd908cf3f9e9ff26fa8e92f028057a9ca1) Backport HDFS-13126 by Erik Krogen Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/53a72f63 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/53a72f63 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/53a72f63 Branch: refs/heads/branch-2.7 Commit: 53a72f63471a2e66c37bd00210114b13b27e8e7c Parents: dc57c1d Author: Kihwal Lee <kih...@apache.org> Authored: Fri Aug 19 15:32:11 2016 -0500 Committer: Konstantin V Shvachko <s...@apache.org> Committed: Thu Feb 8 19:09:21 2018 -0800 ---------------------------------------------------------------------- .../src/main/conf/log4j.properties | 11 ++++++ hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 ++ .../datanode/web/webhdfs/WebHdfsHandler.java | 38 +++++++++++++++----- 3 files changed, 44 insertions(+), 8 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a72f63/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties ---------------------------------------------------------------------- diff --git a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties index 316c48e..e435c10 100644 --- a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties +++ b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties @@ -266,3 +266,14 @@ log4j.appender.RMSUMMARY.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n #log4j.appender.nodemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender #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 http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a72f63/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 3077906..ade5516 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -29,6 +29,9 @@ Release 2.7.6 - UNRELEASED HDFS-13120. Snapshot diff could be corrupted after concat. (xyao) + HDFS-7959. WebHdfs logging is missing on Datanode (Kihwal Lee via sjlee) + Backport HDFS-13126 by Erik Krogen. + Release 2.7.5 - 2017-12-14 INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a72f63/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java index 4d705b0..f510447 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java @@ -29,6 +29,7 @@ import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.QueryStringDecoder; import io.netty.handler.stream.ChunkedStream; +import java.net.InetSocketAddress; import org.apache.commons.io.Charsets; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -71,11 +72,13 @@ 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.OK; import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; +import static io.netty.handler.codec.rtsp.RtspResponseStatuses.INTERNAL_SERVER_ERROR; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HDFS_URI_SCHEME; import static org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier.HDFS_DELEGATION_KIND; 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 = @@ -89,6 +92,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 { @@ -110,12 +114,30 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> { 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.getMethod() + " " + req.getUri() + " " + + getResponseCode()); + } return null; } }); } + int getResponseCode() { + return (resp == null) ? INTERNAL_SERVER_ERROR.code() : + resp.getStatus().code(); + } + public void handle(ChannelHandlerContext ctx, HttpRequest req) throws IOException, URISyntaxException { String op = params.op(); @@ -140,7 +162,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> { @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); } @@ -163,7 +185,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> { OutputStream out = dfsClient.createWrappedOutputStream(dfsClient.create( path, permission, flags, 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()); @@ -180,7 +202,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> { 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)); @@ -192,8 +214,8 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> { 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, "*"); @@ -217,7 +239,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> { data = in; } - ctx.write(response); + ctx.write(resp); ctx.writeAndFlush(new ChunkedStream(data) { @Override public void close() throws Exception { @@ -239,7 +261,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> { IOUtils.cleanup(LOG, dfsclient); } final byte[] js = JsonUtil.toJsonString(checksum).getBytes(Charsets.UTF_8); - DefaultFullHttpResponse resp = + resp = new DefaultFullHttpResponse(HTTP_1_1, OK, Unpooled.wrappedBuffer(js)); resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8); --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-commits-h...@hadoop.apache.org