Skip to content

Commit

Permalink
[FLINK-2978] [web-dashboard] Integrate web client into Web Frontend
Browse files Browse the repository at this point in the history
This closes apache#1338
  • Loading branch information
sachingoel0101 authored and StephanEwen committed Dec 30, 2015
1 parent 4f8c5e8 commit 72f9a6c
Show file tree
Hide file tree
Showing 33 changed files with 1,896 additions and 36 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -346,6 +346,9 @@ public final class ConfigConstants {
*/
public static final String JOB_MANAGER_WEB_LOG_PATH_KEY = "jobmanager.web.log.path";

/** Config parameter indicating whether jobs can be uploaded and run from the web-frontend. */
public static final String JOB_MANAGER_WEB_SUBMISSION_KEY = "jobmanager.web.submit.enable";

/** Flag to disable checkpoint stats. */
public static final String JOB_MANAGER_WEB_CHECKPOINTS_DISABLE = "jobmanager.web.checkpoints.disable";

Expand Down Expand Up @@ -690,17 +693,16 @@ public final class ConfigConstants {

// ------------------------- JobManager Web Frontend ----------------------

/**
* The config key for the port of the JobManager web frontend.
* Setting this value to {@code -1} disables the web frontend.
*/
/** The config key for the port of the JobManager web frontend.
* Setting this value to {@code -1} disables the web frontend. */
public static final int DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT = 8081;

/**
* The default number of archived jobs for the jobmanager
*/
/** The default number of archived jobs for the jobmanager */
public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;

/** By default, submitting jobs from the web-frontend is allowed. */
public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMISSION = true;

/** Default flag to disable checkpoint stats. */
public static final boolean DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE = false;

Expand Down
4 changes: 4 additions & 0 deletions flink-dist/src/main/resources/flink-conf.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,10 @@ parallelism.default: 1

jobmanager.web.port: 8081

# Flag to specify whether job submission is enabled from the web-based
# runtime monitor. Uncomment to disable.

#jobmanager.web.submit.enable: false

# The port uder which the standalone web client
# (for job upload and submit) listens.
Expand Down
5 changes: 5 additions & 0 deletions flink-runtime-web/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,11 @@ under the License.
<artifactId>flink-runtime</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-clients</artifactId>
<version>${project.version}</version>
</dependency>

<!-- ===================================================
Dependencies for the Web Server
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* 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.webmonitor;

/*****************************************************************************
* This code is based on the "HttpUploadServerHandler" from the
* Netty project's HTTP server example.
*
* See http://netty.io and
* https://github.com/netty/netty/blob/netty-4.0.31.Final/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
*****************************************************************************/

import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.http.HttpContent;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpMethod;
import io.netty.handler.codec.http.HttpObject;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.LastHttpContent;
import io.netty.handler.codec.http.QueryStringDecoder;
import io.netty.handler.codec.http.QueryStringEncoder;
import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
import io.netty.handler.codec.http.multipart.DiskFileUpload;
import io.netty.handler.codec.http.multipart.HttpDataFactory;
import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
import io.netty.handler.codec.http.multipart.InterfaceHttpData;
import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;

import java.io.File;
import java.util.UUID;

/**
* Simple code which handles all HTTP requests from the user, and passes them to the Router
* handler directly if they do not involve file upload requests.
* If a file is required to be uploaded, it handles the upload, and in the http request to the
* next handler, passes the name of the file to the next handler.
*/
public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject> {

private HttpRequest request;

private boolean readingChunks;

private static final HttpDataFactory factory = new DefaultHttpDataFactory(true); // use disk

private String requestPath;

private HttpPostRequestDecoder decoder;

private final File uploadDir;

/**
* The directory where files should be uploaded.
*/
public HttpRequestHandler(File uploadDir) {
this.uploadDir = uploadDir;
}

@Override
public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
if (decoder != null) {
decoder.cleanFiles();
}
}

@Override
public void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception {
if (msg instanceof HttpRequest) {
request = (HttpRequest) msg;
requestPath = new QueryStringDecoder(request.getUri()).path();
if (request.getMethod() != HttpMethod.POST) {
ctx.fireChannelRead(request);
} else {
// try to decode the posted data now.
decoder = new HttpPostRequestDecoder(factory, request);
readingChunks = HttpHeaders.isTransferEncodingChunked(request);
if (readingChunks) {
readingChunks = true;
}
}
} else if (decoder != null && msg instanceof HttpContent) {
// New chunk is received
HttpContent chunk = (HttpContent) msg;
decoder.offer(chunk);
try {
while (decoder.hasNext()) {
InterfaceHttpData data = decoder.next();
// IF SOMETHING EVER NEEDS POST PARAMETERS, THIS WILL BE THE PLACE TO HANDLE IT
// all fields values will be passed with type Attribute.
if (data.getHttpDataType() == HttpDataType.FileUpload) {
DiskFileUpload file = (DiskFileUpload) data;
if (file.isCompleted()) {
String newName = UUID.randomUUID() + "_" + file.getFilename();
file.renameTo(new File(uploadDir, newName));
QueryStringEncoder encoder = new QueryStringEncoder(requestPath);
encoder.addParam("file", newName);
request.setUri(encoder.toString());
}
}
data.release();
}
} catch (EndOfDataDecoderException e) {
//
}
if (chunk instanceof LastHttpContent) {
readingChunks = false;
decoder.destroy();
decoder = null;
ctx.fireChannelRead(request);
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* 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.webmonitor;

import com.fasterxml.jackson.core.JsonGenerator;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.HttpVersion;
import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.io.StringWriter;

/**
* This is the last handler in the pipeline and logs all error messages.
*/
@ChannelHandler.Sharable
public class PipelineErrorHandler extends SimpleChannelInboundHandler<Object> {

private static final Logger LOG = LoggerFactory.getLogger(PipelineErrorHandler.class);

@Override
protected void channelRead0(ChannelHandlerContext ctx, Object message) {
// we can't deal with this message. No one in the pipeline handled it. Log it.
LOG.debug("Unknown message received: {}", message);
sendError(ctx, "Unknown message received.");
}

@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
LOG.debug("Unhandled exception: {}", cause);
sendError(ctx, cause.getMessage());
}

private void sendError(ChannelHandlerContext ctx, String error) {
DefaultFullHttpResponse response;
StringWriter writer = new StringWriter();
try {
JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
gen.writeStartObject();
gen.writeStringField("error", error);
gen.writeEndObject();
gen.close();
// send a bad request status code.
response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1,
HttpResponseStatus.BAD_REQUEST, Unpooled.wrappedBuffer(writer.toString().getBytes()));
response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json");
} catch (IOException e) {
// seriously? Let's just send some plain text.
response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1,
HttpResponseStatus.BAD_REQUEST, Unpooled.wrappedBuffer(error.getBytes()));
response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain");
}
response.headers().set(HttpHeaders.Names.CONTENT_ENCODING, "utf-8");
response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes());
if (ctx.channel().isActive()) {
ctx.writeAndFlush(response);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,10 @@
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;

import java.net.InetSocketAddress;
import java.nio.charset.Charset;
import java.util.HashMap;
import java.util.Map;

import static com.google.common.base.Preconditions.checkNotNull;

Expand All @@ -67,6 +70,8 @@ public class RuntimeMonitorHandler extends SimpleChannelInboundHandler<Routed> {

private String localJobManagerAddress;

public static final String WEB_MONITOR_ADDRESS_KEY = "web.monitor.address";

public RuntimeMonitorHandler(
RequestHandler handler,
JobManagerRetriever retriever,
Expand Down Expand Up @@ -113,7 +118,16 @@ private void respondAsLeader(ChannelHandlerContext ctx, Routed routed, ActorGate
DefaultFullHttpResponse response;

try {
String result = handler.handleRequest(routed.pathParams(), jobManager);
// we only pass the first element in the list to the handlers.
Map<String, String> queryParams = new HashMap<>();
for (String key : routed.queryParams().keySet()) {
queryParams.put(key, routed.queryParam(key));
}

InetSocketAddress address = (InetSocketAddress) ctx.channel().localAddress();
queryParams.put(WEB_MONITOR_ADDRESS_KEY, address.getHostName() + ":" + address.getPort());

String result = handler.handleRequest(routed.pathParams(), queryParams, jobManager);
byte[] bytes = result.getBytes(ENCODING);

response = new DefaultFullHttpResponse(
Expand Down
Loading

0 comments on commit 72f9a6c

Please sign in to comment.