[
https://issues.apache.org/jira/browse/FLINK-7527?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16171601#comment-16171601
]
ASF GitHub Bot commented on FLINK-7527:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/4597#discussion_r139675681
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java
---
@@ -0,0 +1,140 @@
+/*
+ * 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.flink.runtime.rest.handler.util;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
+import org.apache.flink.runtime.rest.messages.ResponseBody;
+import org.apache.flink.runtime.rest.util.RestMapperUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+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.ChannelHandlerContext;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+import
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.io.StringWriter;
+
+import static
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+/**
+ * Utilities for the REST handlers.
+ */
+public class HandlerUtils {
+
+ private static final ObjectMapper mapper =
RestMapperUtils.getStrictObjectMapper();
+
+ /**
+ * Sends the given response and status code to the given channel.
+ *
+ * @param channelHandlerContext identifying the open channel
+ * @param httpRequest originating http request
+ * @param response which should be sent
+ * @param statusCode of the message to send
+ * @param <P> type of the response
+ */
+ public static <P extends ResponseBody> void sendResponse(
+ ChannelHandlerContext channelHandlerContext,
+ HttpRequest httpRequest,
+ P response,
+ HttpResponseStatus statusCode) {
+ StringWriter sw = new StringWriter();
+ try {
+ mapper.writeValue(sw, response);
+ } catch (IOException ioe) {
+ sendErrorResponse(channelHandlerContext, httpRequest,
new ErrorResponseBody("Internal server error. Could not map response to
JSON."), HttpResponseStatus.INTERNAL_SERVER_ERROR);
+ return;
+ }
+ sendResponse(channelHandlerContext, httpRequest, sw.toString(),
statusCode);
+ }
+
+ /**
+ * Sends the given error response and status code to the given channel.
+ *
+ * @param channelHandlerContext identifying the open channel
+ * @param httpRequest originating http request
+ * @param errorMessage which should be sent
+ * @param statusCode of the message to send
+ */
+ public static void sendErrorResponse(
+ ChannelHandlerContext channelHandlerContext,
+ HttpRequest httpRequest,
+ ErrorResponseBody errorMessage,
+ HttpResponseStatus statusCode) {
+
+ StringWriter sw = new StringWriter();
+ try {
+ mapper.writeValue(sw, errorMessage);
+ } catch (IOException e) {
+ // this should never happen
+ sendResponse(channelHandlerContext, httpRequest,
"Internal server error. Could not map error response to JSON.",
HttpResponseStatus.INTERNAL_SERVER_ERROR);
+ }
+ sendResponse(channelHandlerContext, httpRequest, sw.toString(),
statusCode);
+ }
+
+ /**
+ * Sends the given response and status code to the given channel.
+ * @param channelHandlerContext identifying the open channel
--- End diff --
good catch. Will fix it.
> Add redirection logic to AbstractRestHandler
> --------------------------------------------
>
> Key: FLINK-7527
> URL: https://issues.apache.org/jira/browse/FLINK-7527
> Project: Flink
> Issue Type: Improvement
> Components: REST
> Affects Versions: 1.4.0
> Reporter: Till Rohrmann
> Assignee: Till Rohrmann
> Labels: flip-6
>
> The {{AbstractRestHandler}} should extend the {{RedirectHandler}} introduced
> with FLINK-7459 in order to add redirection functionality to the
> {{AbstractRestHandler}}.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)