Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 14428200AE4 for ; Fri, 24 Jun 2016 23:49:32 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 12BFF160A58; Fri, 24 Jun 2016 21:49:32 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 3623F160A2E for ; Fri, 24 Jun 2016 23:49:31 +0200 (CEST) Received: (qmail 42210 invoked by uid 500); 24 Jun 2016 21:49:30 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 42201 invoked by uid 99); 24 Jun 2016 21:49:30 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 24 Jun 2016 21:49:30 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 39E13E049D; Fri, 24 Jun 2016 21:49:30 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sjlee@apache.org To: common-commits@hadoop.apache.org Message-Id: <2654da07596f4d72a4efc53324e039f6@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: hadoop git commit: HDFS-7959. WebHdfs logging is missing on Datanode (Kihwal Lee via sjlee) Date: Fri, 24 Jun 2016 21:49:30 +0000 (UTC) archived-at: Fri, 24 Jun 2016 21:49:32 -0000 Repository: hadoop Updated Branches: refs/heads/branch-2 e751bb449 -> ae90d4dd9 HDFS-7959. WebHdfs logging is missing on Datanode (Kihwal Lee via sjlee) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ae90d4dd Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ae90d4dd Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ae90d4dd Branch: refs/heads/branch-2 Commit: ae90d4dd908cf3f9e9ff26fa8e92f028057a9ca1 Parents: e751bb4 Author: Sangjin Lee Authored: Fri Jun 24 14:48:36 2016 -0700 Committer: Sangjin Lee Committed: Fri Jun 24 14:48:36 2016 -0700 ---------------------------------------------------------------------- .../src/main/conf/log4j.properties | 13 +++++++ .../datanode/web/webhdfs/WebHdfsHandler.java | 38 +++++++++++++++----- 2 files changed, 43 insertions(+), 8 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae90d4dd/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 dc7e705..95afc61 100644 --- a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties +++ b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties @@ -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 http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae90d4dd/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 dffe34d..ea824c8 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 static io.netty.handler.codec.http.HttpMethod.POST; 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 io.netty.handler.stream.ChunkedStream; 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.security.PrivilegedExceptionAction; @@ -78,6 +80,7 @@ import com.google.common.base.Preconditions; public class WebHdfsHandler extends SimpleChannelInboundHandler { 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 { 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 class WebHdfsHandler extends SimpleChannelInboundHandler { ugi.doAs(new PrivilegedExceptionAction() { @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 class WebHdfsHandler extends SimpleChannelInboundHandler { @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 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler { 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 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler { 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 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler { 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 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler { data = in; } - ctx.write(response); + ctx.write(resp); ctx.writeAndFlush(new ChunkedStream(data) { @Override public void close() throws Exception { @@ -253,7 +275,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler { 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-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-commits-help@hadoop.apache.org