You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2018/09/03 08:33:52 UTC

[flink] 01/02: [hotfix][rest] Update error handling in FileServerHandlers

This is an automated email from the ASF dual-hosted git repository.

chesnay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 79c341209b90a3cf35f7026d67aa6ddbe02c2c40
Author: zentol <ch...@apache.org>
AuthorDate: Wed Aug 22 11:51:01 2018 +0200

    [hotfix][rest] Update error handling in FileServerHandlers
---
 .../HistoryServerStaticFileServerHandler.java      | 45 +++++++++++++---
 .../legacy/files/StaticFileServerHandler.java      | 61 ++++++++++++++--------
 2 files changed, 77 insertions(+), 29 deletions(-)

diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
index b0c2102..2042088 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java
@@ -28,6 +28,8 @@ package org.apache.flink.runtime.webmonitor.history;
 
 import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandler;
 import org.apache.flink.runtime.rest.handler.router.RoutedRequest;
+import org.apache.flink.runtime.rest.handler.util.HandlerUtils;
+import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
 
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
@@ -57,6 +59,7 @@ import java.net.URL;
 import java.nio.file.Files;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
+import java.util.Collections;
 import java.util.Date;
 import java.util.Locale;
 
@@ -158,7 +161,12 @@ public class HistoryServerStaticFileServerHandler extends SimpleChannelInboundHa
 				} finally {
 					if (!success) {
 						LOG.debug("Unable to load requested file {} from classloader", pathToLoad);
-						StaticFileServerHandler.sendError(ctx, NOT_FOUND);
+						HandlerUtils.sendErrorResponse(
+							ctx,
+							request,
+							new ErrorResponseBody(String.format("Unable to load requested file %s.", requestPath)),
+							NOT_FOUND,
+							Collections.emptyMap());
 						return;
 					}
 				}
@@ -166,12 +174,22 @@ public class HistoryServerStaticFileServerHandler extends SimpleChannelInboundHa
 		}
 
 		if (!file.exists() || file.isHidden() || file.isDirectory() || !file.isFile()) {
-			StaticFileServerHandler.sendError(ctx, NOT_FOUND);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				request,
+				new ErrorResponseBody("File not found."),
+				NOT_FOUND,
+				Collections.emptyMap());
 			return;
 		}
 
 		if (!file.getCanonicalFile().toPath().startsWith(rootPath.toPath())) {
-			StaticFileServerHandler.sendError(ctx, NOT_FOUND);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				request,
+				new ErrorResponseBody("File not found."),
+				NOT_FOUND,
+				Collections.emptyMap());
 			return;
 		}
 
@@ -204,7 +222,12 @@ public class HistoryServerStaticFileServerHandler extends SimpleChannelInboundHa
 		try {
 			raf = new RandomAccessFile(file, "r");
 		} catch (FileNotFoundException e) {
-			StaticFileServerHandler.sendError(ctx, NOT_FOUND);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				request,
+				new ErrorResponseBody("File not found."),
+				NOT_FOUND,
+				Collections.emptyMap());
 			return;
 		}
 
@@ -244,7 +267,12 @@ public class HistoryServerStaticFileServerHandler extends SimpleChannelInboundHa
 		} catch (Exception e) {
 			raf.close();
 			LOG.error("Failed to serve file.", e);
-			StaticFileServerHandler.sendError(ctx, INTERNAL_SERVER_ERROR);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				request,
+				new ErrorResponseBody("Internal server error."),
+				INTERNAL_SERVER_ERROR,
+				Collections.emptyMap());
 		}
 	}
 
@@ -252,7 +280,12 @@ public class HistoryServerStaticFileServerHandler extends SimpleChannelInboundHa
 	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
 		if (ctx.channel().isActive()) {
 			LOG.error("Caught exception", cause);
-			StaticFileServerHandler.sendError(ctx, INTERNAL_SERVER_ERROR);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				false,
+				new ErrorResponseBody("Internal server error."),
+				INTERNAL_SERVER_ERROR,
+				Collections.emptyMap());
 		}
 	}
 }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
index 62b94e5..969945f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
@@ -29,11 +29,12 @@ package org.apache.flink.runtime.rest.handler.legacy.files;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.rest.handler.RedirectHandler;
 import org.apache.flink.runtime.rest.handler.router.RoutedRequest;
+import org.apache.flink.runtime.rest.handler.util.HandlerUtils;
 import org.apache.flink.runtime.rest.handler.util.MimeTypes;
+import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
 import org.apache.flink.runtime.webmonitor.RestfulGateway;
 import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
 
-import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
@@ -50,7 +51,6 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseSt
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
 import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
 import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
-import org.apache.flink.shaded.netty4.io.netty.util.CharsetUtil;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -184,7 +184,12 @@ public class StaticFileServerHandler<T extends RestfulGateway> extends RedirectH
 				} finally {
 					if (!success) {
 						logger.debug("Unable to load requested file {} from classloader", requestPath);
-						sendError(ctx, NOT_FOUND);
+						HandlerUtils.sendErrorResponse(
+							ctx,
+							request,
+							new ErrorResponseBody(String.format("Unable to load requested file %s.", requestPath)),
+							NOT_FOUND,
+							responseHeaders);
 						return;
 					}
 				}
@@ -192,12 +197,22 @@ public class StaticFileServerHandler<T extends RestfulGateway> extends RedirectH
 		}
 
 		if (!file.exists() || file.isHidden() || file.isDirectory() || !file.isFile()) {
-			sendError(ctx, NOT_FOUND);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				request,
+				new ErrorResponseBody("File not found."),
+				NOT_FOUND,
+				responseHeaders);
 			return;
 		}
 
 		if (!file.getCanonicalFile().toPath().startsWith(rootPath.toPath())) {
-			sendError(ctx, NOT_FOUND);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				request,
+				new ErrorResponseBody("File not found."),
+				NOT_FOUND,
+				responseHeaders);
 			return;
 		}
 
@@ -231,7 +246,12 @@ public class StaticFileServerHandler<T extends RestfulGateway> extends RedirectH
 			raf = new RandomAccessFile(file, "r");
 		}
 		catch (FileNotFoundException e) {
-			sendError(ctx, NOT_FOUND);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				request,
+				new ErrorResponseBody("File not found."),
+				HttpResponseStatus.NOT_FOUND,
+				responseHeaders);
 			return;
 		}
 
@@ -271,7 +291,12 @@ public class StaticFileServerHandler<T extends RestfulGateway> extends RedirectH
 		} catch (Exception e) {
 			raf.close();
 			logger.error("Failed to serve file.", e);
-			sendError(ctx, INTERNAL_SERVER_ERROR);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				request,
+				new ErrorResponseBody("Internal server error."),
+				INTERNAL_SERVER_ERROR,
+				responseHeaders);
 		}
 	}
 
@@ -279,7 +304,12 @@ public class StaticFileServerHandler<T extends RestfulGateway> extends RedirectH
 	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
 		if (ctx.channel().isActive()) {
 			logger.error("Caught exception", cause);
-			sendError(ctx, INTERNAL_SERVER_ERROR);
+			HandlerUtils.sendErrorResponse(
+				ctx,
+				false,
+				new ErrorResponseBody("Internal server error."),
+				INTERNAL_SERVER_ERROR,
+				Collections.emptyMap());
 		}
 	}
 
@@ -288,21 +318,6 @@ public class StaticFileServerHandler<T extends RestfulGateway> extends RedirectH
 	// ------------------------------------------------------------------------
 
 	/**
-	 * Writes a simple  error response message.
-	 *
-	 * @param ctx    The channel context to write the response to.
-	 * @param status The response status.
-	 */
-	public static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
-		FullHttpResponse response = new DefaultFullHttpResponse(
-				HTTP_1_1, status, Unpooled.copiedBuffer("Failure: " + status + "\r\n", CharsetUtil.UTF_8));
-		response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8");
-
-		// close the connection as soon as the error message is sent.
-		ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
-	}
-
-	/**
 	 * Send the "304 Not Modified" response. This response can be used when the
 	 * file timestamp is the same as what the browser is sending up.
 	 *