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 2017/07/26 10:16:11 UTC
[5/5] flink git commit: [FLINK-7226] [webUI] Properly include UTF-8
in content-type header
[FLINK-7226] [webUI] Properly include UTF-8 in content-type header
This closes #4392.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/113b8c41
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/113b8c41
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/113b8c41
Branch: refs/heads/master
Commit: 113b8c41da4b5d5be2e890e3e38567272f7dff9a
Parents: 8d55f5f
Author: zentol <ch...@apache.org>
Authored: Wed Jul 19 13:51:35 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Wed Jul 26 11:03:21 2017 +0200
----------------------------------------------------------------------
.../webmonitor/RuntimeMonitorHandler.java | 6 +--
.../handlers/AbstractJsonRequestHandler.java | 2 +-
.../JobCancellationWithSavepointHandlers.java | 8 ++--
.../runtime/webmonitor/WebFrontendITCase.java | 44 ++++++++++++++++++--
.../webmonitor/WebRuntimeMonitorITCase.java | 4 +-
...obCancellationWithSavepointHandlersTest.java | 14 +++----
.../apache/flink/test/util/TestBaseUtils.java | 2 +-
7 files changed, 57 insertions(+), 23 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/113b8c41/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
index a21a871..c5ee081 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java
@@ -112,7 +112,7 @@ public class RuntimeMonitorHandler extends RuntimeMonitorHandlerBase {
ByteBuf message = e.getMessage() == null ? Unpooled.buffer(0)
: Unpooled.wrappedBuffer(e.getMessage().getBytes(ENCODING));
response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NOT_FOUND, message);
- response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain");
+ response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=" + ENCODING.name());
response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
LOG.debug("Error while handling request", e);
}
@@ -120,15 +120,13 @@ public class RuntimeMonitorHandler extends RuntimeMonitorHandlerBase {
byte[] bytes = ExceptionUtils.stringifyException(e).getBytes(ENCODING);
response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1,
HttpResponseStatus.INTERNAL_SERVER_ERROR, Unpooled.wrappedBuffer(bytes));
- response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain");
+ response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=" + ENCODING.name());
response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
LOG.debug("Error while handling request", e);
}
response.headers().set(HttpHeaders.Names.ACCESS_CONTROL_ALLOW_ORIGIN, allowOrigin);
- // Content-Encoding:utf-8
- response.headers().set(HttpHeaders.Names.CONTENT_ENCODING, ENCODING.name());
KeepAliveWrite.flush(ctx, routed.request(), response);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/113b8c41/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
index 1d317c4..6831eaf 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java
@@ -45,7 +45,7 @@ public abstract class AbstractJsonRequestHandler implements RequestHandler {
DefaultFullHttpResponse response = new DefaultFullHttpResponse(
HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(bytes));
- response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json");
+ response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
return response;
http://git-wip-us.apache.org/repos/asf/flink/blob/113b8c41/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
index f396a7f..7dcac7f 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java
@@ -266,7 +266,7 @@ public class JobCancellationWithSavepointHandlers {
response.headers().set(HttpHeaders.Names.LOCATION, location);
- response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json");
+ response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
FullHttpResponse accepted = response;
@@ -377,7 +377,7 @@ public class JobCancellationWithSavepointHandlers {
HttpResponseStatus.CREATED,
Unpooled.wrappedBuffer(bytes));
- response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json");
+ response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
return response;
@@ -402,7 +402,7 @@ public class JobCancellationWithSavepointHandlers {
HttpResponseStatus.ACCEPTED,
Unpooled.wrappedBuffer(bytes));
- response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json");
+ response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
return response;
@@ -428,7 +428,7 @@ public class JobCancellationWithSavepointHandlers {
code,
Unpooled.wrappedBuffer(bytes));
- response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json");
+ response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name());
response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
return response;
http://git-wip-us.apache.org/repos/asf/flink/blob/113b8c41/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java
index 711bbdb..db2b7eb 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java
@@ -27,7 +27,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.JobVertex;
import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
import org.apache.flink.runtime.testutils.StoppableInvokable;
-import org.apache.flink.runtime.webmonitor.files.MimeTypes;
import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient;
import org.apache.flink.test.util.TestBaseUtils;
import org.apache.flink.util.TestLogger;
@@ -37,10 +36,15 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
import io.netty.handler.codec.http.HttpResponseStatus;
import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.File;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
import java.nio.file.Files;
import java.util.Map;
import java.util.concurrent.TimeUnit;
@@ -104,6 +108,38 @@ public class WebFrontendITCase extends TestLogger {
}
@Test
+ public void testResponseHeaders() throws Exception {
+ // check headers for successful json response
+ URL taskManagersUrl = new URL("http://localhost:" + port + "/taskmanagers");
+ HttpURLConnection taskManagerConnection = (HttpURLConnection) taskManagersUrl.openConnection();
+ taskManagerConnection.setConnectTimeout(100000);
+ taskManagerConnection.connect();
+ if (taskManagerConnection.getResponseCode() >= 400) {
+ // error!
+ InputStream is = taskManagerConnection.getErrorStream();
+ String errorMessage = IOUtils.toString(is, ConfigConstants.DEFAULT_CHARSET);
+ throw new RuntimeException(errorMessage);
+ }
+
+ // we don't set the content-encoding header
+ Assert.assertNull(taskManagerConnection.getContentEncoding());
+ Assert.assertEquals("application/json; charset=UTF-8", taskManagerConnection.getContentType());
+
+ // check headers in case of an error
+ URL notFoundJobUrl = new URL("http://localhost:" + port + "/jobs/dontexist");
+ HttpURLConnection notFoundJobConnection = (HttpURLConnection) notFoundJobUrl.openConnection();
+ notFoundJobConnection.setConnectTimeout(100000);
+ notFoundJobConnection.connect();
+ if (notFoundJobConnection.getResponseCode() >= 400) {
+ // we don't set the content-encoding header
+ Assert.assertNull(notFoundJobConnection.getContentEncoding());
+ Assert.assertEquals("text/plain; charset=UTF-8", notFoundJobConnection.getContentType());
+ } else {
+ throw new RuntimeException("Request for non-existing job did not return an error.");
+ }
+ }
+
+ @Test
public void getNumberOfTaskManagers() {
try {
String json = TestBaseUtils.getFromHTTP("http://localhost:" + port + "/taskmanagers/");
@@ -227,7 +263,7 @@ public class WebFrontendITCase extends TestLogger {
HttpTestClient.SimpleHttpResponse response = client.getNextResponse(deadline.timeLeft());
assertEquals(HttpResponseStatus.OK, response.getStatus());
- assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("json"));
+ assertEquals("application/json; charset=UTF-8", response.getType());
assertEquals("{}", response.getContent());
}
@@ -241,7 +277,7 @@ public class WebFrontendITCase extends TestLogger {
HttpTestClient.SimpleHttpResponse response = client.getNextResponse(timeout);
assertEquals(HttpResponseStatus.OK, response.getStatus());
- assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("json"));
+ assertEquals("application/json; charset=UTF-8", response.getType());
assertEquals("{\"jid\":\"" + jid + "\",\"name\":\"Stoppable streaming test job\"," +
"\"execution-config\":{\"execution-mode\":\"PIPELINED\",\"restart-strategy\":\"default\"," +
"\"job-parallelism\":-1,\"object-reuse-mode\":false,\"user-config\":{}}}", response.getContent());
@@ -280,7 +316,7 @@ public class WebFrontendITCase extends TestLogger {
.getNextResponse(deadline.timeLeft());
assertEquals(HttpResponseStatus.OK, response.getStatus());
- assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("json"));
+ assertEquals("application/json; charset=UTF-8", response.getType());
assertEquals("{}", response.getContent());
}
http://git-wip-us.apache.org/repos/asf/flink/blob/113b8c41/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
index 6da11d1..23d8a51 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -108,7 +108,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
response = client.getNextResponse(deadline.timeLeft());
assertEquals(HttpResponseStatus.OK, response.getStatus());
- assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("json"));
+ assertEquals("application/json; charset=UTF-8", response.getType());
assertTrue(response.getContent().contains("\"taskmanagers\":1"));
}
}
@@ -255,7 +255,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
response = followingClient.getNextResponse(deadline.timeLeft());
assertEquals(HttpResponseStatus.OK, response.getStatus());
- assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("json"));
+ assertEquals("application/json; charset=UTF-8", response.getType());
assertTrue(response.getContent().contains("\"taskmanagers\":1") ||
response.getContent().contains("\"taskmanagers\":0"));
} finally {
http://git-wip-us.apache.org/repos/asf/flink/blob/113b8c41/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
index 0ccd34c..71b715a 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java
@@ -199,7 +199,7 @@ public class JobCancellationWithSavepointHandlersTest {
String location = String.format("/jobs/%s/cancel-with-savepoint/in-progress/1", jobId);
assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
- assertEquals("application/json", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+ assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
assertEquals(location, response.headers().get(HttpHeaders.Names.LOCATION));
@@ -213,7 +213,7 @@ public class JobCancellationWithSavepointHandlersTest {
// Trigger again
response = trigger.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
- assertEquals("application/json", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+ assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
assertEquals(location, response.headers().get(HttpHeaders.Names.LOCATION));
@@ -232,7 +232,7 @@ public class JobCancellationWithSavepointHandlersTest {
response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus());
- assertEquals("application/json", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+ assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
json = response.content().toString(Charset.forName("UTF-8"));
@@ -247,7 +247,7 @@ public class JobCancellationWithSavepointHandlersTest {
response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
assertEquals(HttpResponseStatus.CREATED, response.getStatus());
- assertEquals("application/json", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+ assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
json = response.content().toString(Charset.forName("UTF-8"));
@@ -263,7 +263,7 @@ public class JobCancellationWithSavepointHandlersTest {
response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
assertEquals(HttpResponseStatus.CREATED, response.getStatus());
- assertEquals("application/json", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+ assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
json = response.content().toString(Charset.forName("UTF-8"));
@@ -279,7 +279,7 @@ public class JobCancellationWithSavepointHandlersTest {
response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
assertEquals(HttpResponseStatus.BAD_REQUEST, response.getStatus());
- assertEquals("application/json", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+ assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
json = response.content().toString(Charset.forName("UTF-8"));
@@ -326,7 +326,7 @@ public class JobCancellationWithSavepointHandlersTest {
FullHttpResponse response = progress.handleRequest(params, Collections.<String, String>emptyMap(), jobManager);
assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR, response.getStatus());
- assertEquals("application/json", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
+ assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE));
assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH));
String json = response.content().toString(Charset.forName("UTF-8"));
http://git-wip-us.apache.org/repos/asf/flink/blob/113b8c41/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index c7ec890..da6d9a8 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -614,7 +614,7 @@ public class TestBaseUtils extends TestLogger {
is = connection.getInputStream();
}
- return IOUtils.toString(is, connection.getContentEncoding() != null ? connection.getContentEncoding() : "UTF-8");
+ return IOUtils.toString(is, ConfigConstants.DEFAULT_CHARSET);
}
/**