提交 d481f295 编写于 作者: Z zentol

[FLINK-6687] [web] Activate strict checkstyle for flink-runtime-web

上级 bf92055a
......@@ -128,6 +128,41 @@ under the License.
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-checkstyle-plugin</artifactId>
<version>2.17</version>
<dependencies>
<dependency>
<groupId>com.puppycrawl.tools</groupId>
<artifactId>checkstyle</artifactId>
<version>6.19</version>
</dependency>
</dependencies>
<configuration>
<configLocation>/tools/maven/strict-checkstyle.xml</configLocation>
<suppressionsLocation>/tools/maven/suppressions.xml</suppressionsLocation>
<includeTestSourceDirectory>true</includeTestSourceDirectory>
<logViolationsToConsole>true</logViolationsToConsole>
<failOnViolation>true</failOnViolation>
</configuration>
<executions>
<!--
Execute checkstyle after compilation but before tests.
This ensures that any parsing or type checking errors are from
javac, so they look as expected. Beyond that, we want to
fail as early as possible.
-->
<execution>
<phase>test-compile</phase>
<goals>
<goal>check</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
......
......@@ -18,9 +18,6 @@
package org.apache.flink.runtime.webmonitor;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.collect.Maps;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.concurrent.BiFunction;
import org.apache.flink.runtime.concurrent.Future;
......@@ -28,9 +25,12 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
import org.apache.flink.runtime.executiongraph.ExecutionVertex;
import org.apache.flink.runtime.jobgraph.JobStatus;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.collect.Maps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Option;
import java.util.Arrays;
import java.util.HashSet;
......@@ -41,6 +41,8 @@ import java.util.Set;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import scala.Option;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
......
......@@ -27,18 +27,19 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.WeakHashMap;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import java.util.WeakHashMap;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* Gateway to obtaining an {@link ExecutionGraph} from a source, like JobManager or Archive.
* <p>
* The holder will cache the ExecutionGraph behind a weak reference, which will be cleared
*
* <p>The holder will cache the ExecutionGraph behind a weak reference, which will be cleared
* at some point once no one else is pointing to the ExecutionGraph.
* Note that while the holder runs in the same JVM as the JobManager or Archive, the reference should
* stay valid.
......
......@@ -26,6 +26,9 @@ package org.apache.flink.runtime.webmonitor;
* https://github.com/netty/netty/blob/netty-4.0.31.Final/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
*****************************************************************************/
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.util.ExceptionUtils;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
......@@ -48,8 +51,6 @@ 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 org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.util.ExceptionUtils;
import java.io.File;
import java.io.IOException;
......@@ -67,7 +68,7 @@ public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject>
private static final Charset ENCODING = ConfigConstants.DEFAULT_CHARSET;
/** A decoder factory that always stores POST chunks on disk */
/** A decoder factory that always stores POST chunks on disk. */
private static final HttpDataFactory DATA_FACTORY = new DefaultHttpDataFactory(true);
private final File tmpDir;
......
......@@ -18,19 +18,24 @@
package org.apache.flink.runtime.webmonitor;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.dispatch.Futures;
import akka.dispatch.Mapper;
import akka.dispatch.OnComplete;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.instance.AkkaActorGateway;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
import org.apache.flink.runtime.messages.JobManagerMessages;
import org.apache.flink.runtime.messages.JobManagerMessages.ResponseWebMonitorPort;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.dispatch.Futures;
import akka.dispatch.Mapper;
import akka.dispatch.OnComplete;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import scala.Option;
import scala.Tuple2;
import scala.concurrent.Await;
......@@ -39,9 +44,6 @@ import scala.concurrent.Promise;
import scala.concurrent.duration.Deadline;
import scala.concurrent.duration.FiniteDuration;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
......@@ -106,7 +108,7 @@ public class JobManagerRetriever implements LeaderRetrievalListener {
Future<Tuple2<ActorGateway, Integer>> gatewayPortFuture = null;
Deadline deadline = timeout.fromNow();
while(!deadline.isOverdue()) {
while (!deadline.isOverdue()) {
synchronized (waitLock) {
gatewayPortFuture = leaderGatewayPortFuture;
......
......@@ -18,6 +18,9 @@
package org.apache.flink.runtime.webmonitor;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.util.ExceptionUtils;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
......@@ -26,8 +29,6 @@ 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.configuration.ConfigConstants;
import org.apache.flink.util.ExceptionUtils;
import org.slf4j.Logger;
/**
......@@ -37,7 +38,7 @@ import org.slf4j.Logger;
@ChannelHandler.Sharable
public class PipelineErrorHandler extends SimpleChannelInboundHandler<Object> {
/** The logger to which the handler writes the log statements */
/** The logger to which the handler writes the log statements. */
private final Logger logger;
public PipelineErrorHandler(Logger logger) {
......
......@@ -18,6 +18,11 @@
package org.apache.flink.runtime.webmonitor;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
import org.apache.flink.util.ExceptionUtils;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandler;
......@@ -29,24 +34,18 @@ import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.HttpVersion;
import io.netty.handler.codec.http.router.KeepAliveWrite;
import io.netty.handler.codec.http.router.Routed;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
import org.apache.flink.util.ExceptionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import java.net.InetSocketAddress;
import java.net.URLDecoder;
import java.nio.charset.Charset;
import java.util.HashMap;
import java.util.Map;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
......
......@@ -18,15 +18,17 @@
package org.apache.flink.runtime.webmonitor;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils;
import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.http.HttpResponse;
import io.netty.handler.codec.http.router.KeepAliveWrite;
import io.netty.handler.codec.http.router.Routed;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils;
import org.apache.flink.runtime.webmonitor.handlers.RequestHandler;
import scala.Option;
import scala.Tuple2;
import scala.concurrent.Await;
......@@ -50,11 +52,11 @@ public abstract class RuntimeMonitorHandlerBase extends SimpleChannelInboundHand
protected final FiniteDuration timeout;
/** Whether the web service has https enabled */
/** Whether the web service has https enabled. */
protected final boolean httpsEnabled;
protected String localJobManagerAddress;
public RuntimeMonitorHandlerBase(
JobManagerRetriever retriever,
Future<String> localJobManagerAddressFuture,
......
......@@ -33,7 +33,7 @@ import static org.apache.flink.util.Preconditions.checkArgument;
*/
public class StackTraceSample {
/** ID of this sample (unique per job) */
/** ID of this sample (unique per job). */
private final int sampleId;
/** Time stamp, when the sample was triggered. */
......@@ -46,7 +46,7 @@ public class StackTraceSample {
private final Map<ExecutionAttemptID, List<StackTraceElement[]>> stackTracesByTask;
/**
* Creates a stack trace sample
* Creates a stack trace sample.
*
* @param sampleId ID of the sample.
* @param startTime Time stamp, when the sample was triggered.
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor;
import com.google.common.collect.Maps;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.concurrent.BiFunction;
import org.apache.flink.runtime.concurrent.CompletableFuture;
......@@ -30,6 +29,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionVertex;
import org.apache.flink.runtime.messages.StackTraceSampleResponse;
import org.apache.flink.util.Preconditions;
import com.google.common.collect.Maps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -57,7 +58,7 @@ public class StackTraceSampleCoordinator {
private final Object lock = new Object();
/** Executor used to run the futures */
/** Executor used to run the futures. */
private final Executor executor;
/** Time out after the expected sampling duration. */
......
......@@ -21,12 +21,14 @@ package org.apache.flink.runtime.webmonitor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
/**
* Configuration object for {@link WebMonitor}.
*/
public class WebMonitorConfig {
/** The configuration queried by this config object */
/** The configuration queried by this config object. */
private final Configuration config;
public WebMonitorConfig(Configuration config) {
if (config == null) {
throw new NullPointerException();
......@@ -45,7 +47,7 @@ public class WebMonitorConfig {
public long getRefreshInterval() {
return config.getLong(JobManagerOptions.WEB_REFRESH_INTERVAL);
}
public boolean isProgramSubmitEnabled() {
return config.getBoolean(JobManagerOptions.WEB_SUBMIT_ENABLE);
}
......
......@@ -18,10 +18,6 @@
package org.apache.flink.runtime.webmonitor;
import akka.actor.ActorSystem;
import io.netty.handler.codec.http.router.Router;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
......@@ -67,8 +63,8 @@ import org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler;
import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointConfigHandler;
import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsCache;
import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsDetailsHandler;
import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsHandler;
import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsDetailsSubtasksHandler;
import org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsHandler;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.metrics.JobManagerMetricsHandler;
import org.apache.flink.runtime.webmonitor.metrics.JobMetricsHandler;
......@@ -78,15 +74,13 @@ import org.apache.flink.runtime.webmonitor.metrics.TaskManagerMetricsHandler;
import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
import org.apache.flink.util.FileUtils;
import akka.actor.ActorSystem;
import io.netty.handler.codec.http.router.Router;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.concurrent.ExecutionContext$;
import scala.concurrent.ExecutionContextExecutor;
import scala.concurrent.Promise;
import scala.concurrent.duration.FiniteDuration;
import javax.net.ssl.SSLContext;
import java.io.File;
import java.io.IOException;
import java.util.UUID;
......@@ -95,32 +89,37 @@ import java.util.concurrent.ForkJoinPool;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import scala.concurrent.ExecutionContext$;
import scala.concurrent.ExecutionContextExecutor;
import scala.concurrent.Promise;
import scala.concurrent.duration.FiniteDuration;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* The root component of the web runtime monitor. This class starts the web server and creates
* all request handlers for the REST API.
* <p>
* The web runtime monitor is based in Netty HTTP. It uses the Netty-Router library to route
*
* <p>The web runtime monitor is based in Netty HTTP. It uses the Netty-Router library to route
* HTTP requests of different paths to different response handlers. In addition, it serves the static
* files of the web frontend, such as HTML, CSS, or JS files.
*/
public class WebRuntimeMonitor implements WebMonitor {
/** By default, all requests to the JobManager have a timeout of 10 seconds */
/** By default, all requests to the JobManager have a timeout of 10 seconds. */
public static final FiniteDuration DEFAULT_REQUEST_TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS);
/** Logger for web frontend startup / shutdown messages */
/** Logger for web frontend startup / shutdown messages. */
private static final Logger LOG = LoggerFactory.getLogger(WebRuntimeMonitor.class);
// ------------------------------------------------------------------------
/** Guarding concurrent modifications to the server channel pipeline during startup and shutdown */
/** Guarding concurrent modifications to the server channel pipeline during startup and shutdown. */
private final Object startupShutdownLock = new Object();
private final LeaderRetrievalService leaderRetrievalService;
/** LeaderRetrievalListener which stores the currently leading JobManager and its archive */
/** LeaderRetrievalListener which stores the currently leading JobManager and its archive. */
private final JobManagerRetriever retriever;
private final SSLContext serverSSLContext;
......@@ -163,9 +162,9 @@ public class WebRuntimeMonitor implements WebMonitor {
if (configuredPort < 0) {
throw new IllegalArgumentException("Web frontend port is invalid: " + configuredPort);
}
final WebMonitorUtils.LogFileLocation logFiles = WebMonitorUtils.LogFileLocation.find(config);
// create an empty directory in temp for the web server
String rootDirFileName = "flink-web-" + UUID.randomUUID();
webRootDir = new File(getBaseDir(config), rootDirFileName);
......@@ -236,42 +235,42 @@ public class WebRuntimeMonitor implements WebMonitor {
Router router = new Router();
// config how to interact with this web server
GET(router, new DashboardConfigHandler(cfg.getRefreshInterval()));
get(router, new DashboardConfigHandler(cfg.getRefreshInterval()));
// the overview - how many task managers, slots, free slots, ...
GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT));
get(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT));
// job manager configuration
GET(router, new JobManagerConfigHandler(config));
get(router, new JobManagerConfigHandler(config));
// overview over jobs
GET(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, true));
GET(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, false));
GET(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, false, true));
GET(router, new CurrentJobIdsHandler(DEFAULT_REQUEST_TIMEOUT));
GET(router, new JobDetailsHandler(currentGraphs, metricFetcher));
GET(router, new JobVertexDetailsHandler(currentGraphs, metricFetcher));
GET(router, new SubtasksTimesHandler(currentGraphs));
GET(router, new JobVertexTaskManagersHandler(currentGraphs, metricFetcher));
GET(router, new JobVertexAccumulatorsHandler(currentGraphs));
GET(router, new JobVertexBackPressureHandler(currentGraphs, backPressureStatsTracker, refreshInterval));
GET(router, new JobVertexMetricsHandler(metricFetcher));
GET(router, new SubtasksAllAccumulatorsHandler(currentGraphs));
GET(router, new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher));
GET(router, new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher));
GET(router, new SubtaskExecutionAttemptAccumulatorsHandler(currentGraphs));
GET(router, new JobPlanHandler(currentGraphs));
GET(router, new JobConfigHandler(currentGraphs));
GET(router, new JobExceptionsHandler(currentGraphs));
GET(router, new JobAccumulatorsHandler(currentGraphs));
GET(router, new JobMetricsHandler(metricFetcher));
GET(router, new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher));
GET(router,
get(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, true));
get(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, false));
get(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, false, true));
get(router, new CurrentJobIdsHandler(DEFAULT_REQUEST_TIMEOUT));
get(router, new JobDetailsHandler(currentGraphs, metricFetcher));
get(router, new JobVertexDetailsHandler(currentGraphs, metricFetcher));
get(router, new SubtasksTimesHandler(currentGraphs));
get(router, new JobVertexTaskManagersHandler(currentGraphs, metricFetcher));
get(router, new JobVertexAccumulatorsHandler(currentGraphs));
get(router, new JobVertexBackPressureHandler(currentGraphs, backPressureStatsTracker, refreshInterval));
get(router, new JobVertexMetricsHandler(metricFetcher));
get(router, new SubtasksAllAccumulatorsHandler(currentGraphs));
get(router, new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher));
get(router, new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher));
get(router, new SubtaskExecutionAttemptAccumulatorsHandler(currentGraphs));
get(router, new JobPlanHandler(currentGraphs));
get(router, new JobConfigHandler(currentGraphs));
get(router, new JobExceptionsHandler(currentGraphs));
get(router, new JobAccumulatorsHandler(currentGraphs));
get(router, new JobMetricsHandler(metricFetcher));
get(router, new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher));
get(router,
new TaskManagerLogHandler(
retriever,
context,
......@@ -281,7 +280,7 @@ public class WebRuntimeMonitor implements WebMonitor {
config,
enableSSL,
blobView));
GET(router,
get(router,
new TaskManagerLogHandler(
retriever,
context,
......@@ -291,7 +290,7 @@ public class WebRuntimeMonitor implements WebMonitor {
config,
enableSSL,
blobView));
GET(router, new TaskManagerMetricsHandler(metricFetcher));
get(router, new TaskManagerMetricsHandler(metricFetcher));
router
// log and stdout
......@@ -303,51 +302,51 @@ public class WebRuntimeMonitor implements WebMonitor {
new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile,
enableSSL));
GET(router, new JobManagerMetricsHandler(metricFetcher));
get(router, new JobManagerMetricsHandler(metricFetcher));
// Cancel a job via GET (for proper integration with YARN this has to be performed via GET)
GET(router, new JobCancellationHandler());
get(router, new JobCancellationHandler());
// DELETE is the preferred way of canceling a job (Rest-conform)
DELETE(router, new JobCancellationHandler());
delete(router, new JobCancellationHandler());
GET(router, triggerHandler);
GET(router, inProgressHandler);
get(router, triggerHandler);
get(router, inProgressHandler);
// stop a job via GET (for proper integration with YARN this has to be performed via GET)
GET(router, new JobStoppingHandler());
get(router, new JobStoppingHandler());
// DELETE is the preferred way of stopping a job (Rest-conform)
DELETE(router, new JobStoppingHandler());
delete(router, new JobStoppingHandler());
int maxCachedEntries = config.getInteger(JobManagerOptions.WEB_CHECKPOINTS_HISTORY_SIZE);
CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries);
// Register the checkpoint stats handlers
GET(router, new CheckpointStatsHandler(currentGraphs));
GET(router, new CheckpointConfigHandler(currentGraphs));
GET(router, new CheckpointStatsDetailsHandler(currentGraphs, cache));
GET(router, new CheckpointStatsDetailsSubtasksHandler(currentGraphs, cache));
get(router, new CheckpointStatsHandler(currentGraphs));
get(router, new CheckpointConfigHandler(currentGraphs));
get(router, new CheckpointStatsDetailsHandler(currentGraphs, cache));
get(router, new CheckpointStatsDetailsSubtasksHandler(currentGraphs, cache));
if (webSubmitAllow) {
// fetch the list of uploaded jars.
GET(router, new JarListHandler(uploadDir));
get(router, new JarListHandler(uploadDir));
// get plan for an uploaded jar
GET(router, new JarPlanHandler(uploadDir));
get(router, new JarPlanHandler(uploadDir));
// run a jar
POST(router, new JarRunHandler(uploadDir, timeout, config));
post(router, new JarRunHandler(uploadDir, timeout, config));
// upload a jar
POST(router, new JarUploadHandler(uploadDir));
post(router, new JarUploadHandler(uploadDir));
// delete an uploaded jar from submission interface
DELETE(router, new JarDeleteHandler(uploadDir));
delete(router, new JarDeleteHandler(uploadDir));
} else {
// send an Access Denied message
JarAccessDeniedHandler jad = new JarAccessDeniedHandler();
GET(router, jad);
POST(router, jad);
DELETE(router, jad);
get(router, jad);
post(router, jad);
delete(router, jad);
}
// this handler serves all the static contents
......@@ -375,15 +374,15 @@ public class WebRuntimeMonitor implements WebMonitor {
/**
* Returns an array of all {@link JsonArchivist}s that are relevant for the history server.
*
* This method is static to allow easier access from the {@link MemoryArchivist}. Requiring a reference
*
* <p>This method is static to allow easier access from the {@link MemoryArchivist}. Requiring a reference
* would imply that the WebRuntimeMonitor is always created before the archivist, which may not hold for all
* deployment modes.
*
* Similarly, no handler implements the JsonArchivist interface itself but instead contains a separate implementing
*
<p>Similarly, no handler implements the JsonArchivist interface itself but instead contains a separate implementing
* class; otherwise we would either instantiate several handlers even though their main functionality isn't
* required, or yet again require that the WebRuntimeMonitor is started before the archivist.
*
*
* @return array of all JsonArchivists relevant for the history server
*/
public static JsonArchivist[] getJsonArchivists() {
......@@ -400,13 +399,13 @@ public class WebRuntimeMonitor implements WebMonitor {
new CheckpointConfigHandler.CheckpointConfigJsonArchivist(),
new CheckpointStatsDetailsHandler.CheckpointStatsDetailsJsonArchivist(),
new CheckpointStatsDetailsSubtasksHandler.CheckpointStatsDetailsSubtasksJsonArchivist(),
new JobVertexDetailsHandler.JobVertexDetailsJsonArchivist(),
new SubtasksTimesHandler.SubtasksTimesJsonArchivist(),
new JobVertexTaskManagersHandler.JobVertexTaskManagersJsonArchivist(),
new JobVertexAccumulatorsHandler.JobVertexAccumulatorsJsonArchivist(),
new SubtasksAllAccumulatorsHandler.SubtasksAllAccumulatorsJsonArchivist(),
new SubtaskExecutionAttemptDetailsHandler.SubtaskExecutionAttemptDetailsJsonArchivist(),
new SubtaskExecutionAttemptAccumulatorsHandler.SubtaskExecutionAttemptAccumulatorsJsonArchivist()
};
......@@ -416,7 +415,7 @@ public class WebRuntimeMonitor implements WebMonitor {
@Override
public void start(String jobManagerAkkaUrl) throws Exception {
LOG.info("Starting with JobManager {} on port {}", jobManagerAkkaUrl, getServerPort());
synchronized (startupShutdownLock) {
jobManagerAddressPromise.success(jobManagerAkkaUrl);
leaderRetrievalService.start(retriever);
......@@ -487,31 +486,31 @@ public class WebRuntimeMonitor implements WebMonitor {
* {@link RuntimeMonitorHandlerBase} with the given {@link Router} for the respective REST method.
* The REST paths under which they are registered are defined by the handlers. **/
private void GET(Router router, RequestHandler handler) {
GET(router, handler(handler));
private void get(Router router, RequestHandler handler) {
get(router, handler(handler));
}
private void GET(Router router, RuntimeMonitorHandlerBase handler) {
private void get(Router router, RuntimeMonitorHandlerBase handler) {
for (String path : handler.getPaths()) {
router.GET(path, handler);
}
}
private void DELETE(Router router, RequestHandler handler) {
DELETE(router, handler(handler));
private void delete(Router router, RequestHandler handler) {
delete(router, handler(handler));
}
private void DELETE(Router router, RuntimeMonitorHandlerBase handler) {
private void delete(Router router, RuntimeMonitorHandlerBase handler) {
for (String path : handler.getPaths()) {
router.DELETE(path, handler);
}
}
private void POST(Router router, RequestHandler handler) {
POST(router, handler(handler));
private void post(Router router, RequestHandler handler) {
post(router, handler(handler));
}
private void POST(Router router, RuntimeMonitorHandlerBase handler) {
private void post(Router router, RuntimeMonitorHandlerBase handler) {
for (String path : handler.getPaths()) {
router.POST(path, handler);
}
......
......@@ -22,23 +22,23 @@ import java.util.HashMap;
/**
* Simple utility class that resolves file extensions to MIME types.
* <p>
* There are various solutions built into Java that depend on extra resource and configuration
*
* <p>There are various solutions built into Java that depend on extra resource and configuration
* files. They are designed to be composable and extensible, but also unfortunately tricky to control.
* This is meant to be a simple solution that may eventually be subsumed by a better one.
*/
public class MimeTypes {
/** The default mime type */
/** The default mime type. */
private static final String DEFAULT_MIME_TYPE = "application/octet-stream";
/** The map with the constants */
/** The map with the constants. */
private static final HashMap<String, String> MIME_MAP = new HashMap<String, String>();
/**
* Gets the MIME type for the file with the given extension. If the mime type is not recognized,
* this method returns null.
*
*
* @param fileExtension The file extension.
* @return The MIME type, or {@code null}, if the file extension is not recognized.
*/
......@@ -68,7 +68,7 @@ public class MimeTypes {
/**
* Gets the default MIME type, which is {@code "application/octet-stream"}.
*
*
* @return The default MIME type.
*/
public static String getDefaultMimeType() {
......@@ -78,13 +78,13 @@ public class MimeTypes {
// ------------------------------------------------------------------------
// prevent instantiation
// ------------------------------------------------------------------------
private MimeTypes() {}
// ------------------------------------------------------------------------
// initialization
// ------------------------------------------------------------------------
static {
// text types
MIME_MAP.put("html", "text/html");
......@@ -96,11 +96,11 @@ public class MimeTypes {
MIME_MAP.put("err", "text/plain");
MIME_MAP.put("xml", "text/xml");
MIME_MAP.put("csv", "text/csv");
// application types
MIME_MAP.put("js", "application/javascript");
MIME_MAP.put("json", "application/json");
// image types
MIME_MAP.put("png", "image/png");
MIME_MAP.put("jpg", "image/jpeg");
......@@ -110,7 +110,7 @@ public class MimeTypes {
MIME_MAP.put("tiff", "image/tiff");
MIME_MAP.put("tff", "image/tiff");
MIME_MAP.put("bmp", "image/bmp");
// fonts
MIME_MAP.put("woff", "application/font-woff");
MIME_MAP.put("woff2", "application/font-woff2");
......
......@@ -26,6 +26,10 @@ package org.apache.flink.runtime.webmonitor.files;
* https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
*****************************************************************************/
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
import org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
......@@ -47,16 +51,8 @@ import io.netty.handler.codec.http.router.Routed;
import io.netty.handler.ssl.SslHandler;
import io.netty.handler.stream.ChunkedFile;
import io.netty.util.CharsetUtil;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
import org.apache.flink.runtime.webmonitor.handlers.HandlerRedirectUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Option;
import scala.Tuple2;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import java.io.File;
import java.io.FileNotFoundException;
......@@ -75,6 +71,12 @@ import java.util.GregorianCalendar;
import java.util.Locale;
import java.util.TimeZone;
import scala.Option;
import scala.Tuple2;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import static io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
......@@ -99,34 +101,33 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
@ChannelHandler.Sharable
public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed> {
/** Default logger, if none is specified */
/** Default logger, if none is specified. */
private static final Logger DEFAULT_LOGGER = LoggerFactory.getLogger(StaticFileServerHandler.class);
/** Timezone in which this server answers its "if-modified" requests */
/** Timezone in which this server answers its "if-modified" requests. */
private static final TimeZone GMT_TIMEZONE = TimeZone.getTimeZone("GMT");
/** Date format for HTTP */
/** Date format for HTTP. */
public static final String HTTP_DATE_FORMAT = "EEE, dd MMM yyyy HH:mm:ss zzz";
/** Be default, we allow files to be cached for 5 minutes */
/** Be default, we allow files to be cached for 5 minutes. */
private static final int HTTP_CACHE_SECONDS = 300;
// ------------------------------------------------------------------------
/** JobManager retriever */
private final JobManagerRetriever retriever;
private final Future<String> localJobManagerAddressFuture;
private final FiniteDuration timeout;
/** The path in which the static documents are */
/** The path in which the static documents are. */
private final File rootPath;
/** Whether the web service has https enabled */
/** Whether the web service has https enabled. */
private final boolean httpsEnabled;
/** The log for all error reporting */
/** The log for all error reporting. */
private final Logger logger;
private String localJobManagerAddress;
......@@ -218,7 +219,7 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
// file does not exist. Try to load it with the classloader
ClassLoader cl = StaticFileServerHandler.class.getClassLoader();
try(InputStream resourceStream = cl.getResourceAsStream("web" + requestPath)) {
try (InputStream resourceStream = cl.getResourceAsStream("web" + requestPath)) {
boolean success = false;
try {
if (resourceStream != null) {
......@@ -282,7 +283,7 @@ public class StaticFileServerHandler extends SimpleChannelInboundHandler<Routed>
return;
}
}
if (logger.isDebugEnabled()) {
logger.debug("Responding with file '" + file.getAbsolutePath() + '\'');
}
......
......@@ -31,9 +31,9 @@ import java.util.Map;
* that can be retrieved via "jobid" parameter.
*/
public abstract class AbstractExecutionGraphRequestHandler extends AbstractJsonRequestHandler {
private final ExecutionGraphHolder executionGraphHolder;
public AbstractExecutionGraphRequestHandler(ExecutionGraphHolder executionGraphHolder) {
this.executionGraphHolder = executionGraphHolder;
}
......@@ -50,16 +50,16 @@ public abstract class AbstractExecutionGraphRequestHandler extends AbstractJsonR
jid = JobID.fromHexString(jidString);
}
catch (Exception e) {
throw new RuntimeException("Invalid JobID string '" + jidString + "': " + e.getMessage());
throw new RuntimeException("Invalid JobID string '" + jidString + "': " + e.getMessage());
}
AccessExecutionGraph eg = executionGraphHolder.getExecutionGraph(jid, jobManager);
if (eg == null) {
throw new NotFoundException("Could not find job with id " + jid);
}
return handleRequest(eg, pathParams);
}
public abstract String handleRequest(AccessExecutionGraph graph, Map<String, String> params) throws Exception;
}
......@@ -27,10 +27,10 @@ import java.util.Map;
/**
* Base class for request handlers whose response depends on a specific job vertex (defined
* via the "vertexid" parameter) in a specific job, defined via (defined voa the "jobid" parameter).
* via the "vertexid" parameter) in a specific job, defined via (defined voa the "jobid" parameter).
*/
public abstract class AbstractJobVertexRequestHandler extends AbstractExecutionGraphRequestHandler {
public AbstractJobVertexRequestHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -65,6 +65,6 @@ public abstract class AbstractJobVertexRequestHandler extends AbstractExecutionG
return null;
}
}
public abstract String handleRequest(AccessExecutionJobVertex jobVertex, Map<String, String> params) throws Exception;
}
......@@ -18,13 +18,14 @@
package org.apache.flink.runtime.webmonitor.handlers;
import org.apache.flink.runtime.instance.ActorGateway;
import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.FullHttpResponse;
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.instance.ActorGateway;
import java.nio.charset.Charset;
import java.util.Map;
......
......@@ -28,14 +28,14 @@ import java.util.Map;
* Base class for request handlers whose response depends on a specific subtask execution attempt
* (defined via the "attempt" parameter) of a specific subtask (defined via the
* "subtasknum" parameter) in a specific job vertex (defined via the "vertexid" parameter) in a
* specific job, defined via (defined voa the "jobid" parameter).
* specific job, defined via (defined voa the "jobid" parameter).
*/
public abstract class AbstractSubtaskAttemptRequestHandler extends AbstractSubtaskRequestHandler {
public AbstractSubtaskAttemptRequestHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
@Override
public String handleRequest(AccessExecutionVertex vertex, Map<String, String> params) throws Exception {
final String attemptNumberString = params.get("attempt");
......@@ -50,7 +50,7 @@ public abstract class AbstractSubtaskAttemptRequestHandler extends AbstractSubta
catch (NumberFormatException e) {
throw new RuntimeException("Invalid attempt number parameter");
}
final AccessExecution currentAttempt = vertex.getCurrentExecutionAttempt();
if (attempt == currentAttempt.getAttemptNumber()) {
return handleRequest(currentAttempt, params);
......
......@@ -27,10 +27,10 @@ import java.util.Map;
/**
* Base class for request handlers whose response depends on a specific subtask (defined via the
* "subtasknum" parameter) in a specific job vertex (defined via the "vertexid" parameter) in a
* specific job, defined via (defined voa the "jobid" parameter).
* specific job, defined via (defined voa the "jobid" parameter).
*/
public abstract class AbstractSubtaskRequestHandler extends AbstractJobVertexRequestHandler {
public AbstractSubtaskRequestHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -49,11 +49,11 @@ public abstract class AbstractSubtaskRequestHandler extends AbstractJobVertexReq
catch (NumberFormatException e) {
throw new RuntimeException("Invalid subtask number parameter");
}
if (subtask < 0 || subtask >= jobVertex.getParallelism()) {
throw new RuntimeException("subtask does not exist: " + subtask);
throw new RuntimeException("subtask does not exist: " + subtask);
}
final AccessExecutionVertex vertex = jobVertex.getTaskVertices()[subtask];
return handleRequest(vertex, params);
}
......
......@@ -18,18 +18,20 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.messages.webmonitor.RequestStatusOverview;
import org.apache.flink.runtime.messages.webmonitor.StatusOverview;
import org.apache.flink.runtime.util.EnvironmentInformation;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.StringWriter;
import java.util.Map;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
......@@ -45,7 +47,7 @@ public class ClusterOverviewHandler extends AbstractJsonRequestHandler {
private static final String commitID = EnvironmentInformation.getRevisionInformation().commitId;
private final FiniteDuration timeout;
public ClusterOverviewHandler(FiniteDuration timeout) {
this.timeout = checkNotNull(timeout);
}
......@@ -64,7 +66,7 @@ public class ClusterOverviewHandler extends AbstractJsonRequestHandler {
StatusOverview overview = (StatusOverview) Await.result(future, timeout);
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeNumberField("taskmanagers", overview.getNumTaskManagersConnected());
......
......@@ -18,6 +18,8 @@
package org.apache.flink.runtime.webmonitor.handlers;
import org.apache.flink.configuration.ConfigConstants;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
......@@ -29,7 +31,6 @@ import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.HttpVersion;
import io.netty.handler.codec.http.router.KeepAliveWrite;
import io.netty.handler.codec.http.router.Routed;
import org.apache.flink.configuration.ConfigConstants;
/**
* Responder that returns a constant String.
......
......@@ -18,18 +18,20 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.messages.webmonitor.JobsWithIDsOverview;
import org.apache.flink.runtime.messages.webmonitor.RequestJobsWithIDsOverview;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.StringWriter;
import java.util.Map;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import static java.util.Objects.requireNonNull;
/**
......@@ -42,7 +44,7 @@ public class CurrentJobIdsHandler extends AbstractJsonRequestHandler {
private static final String CURRENT_JOB_IDS_REST_PATH = "/jobs";
private final FiniteDuration timeout;
public CurrentJobIdsHandler(FiniteDuration timeout) {
this.timeout = requireNonNull(timeout);
}
......@@ -51,7 +53,7 @@ public class CurrentJobIdsHandler extends AbstractJsonRequestHandler {
public String[] getPaths() {
return new String[]{CURRENT_JOB_IDS_REST_PATH};
}
@Override
public String handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, ActorGateway jobManager) throws Exception {
// we need no parameters, get all requests
......@@ -59,38 +61,38 @@ public class CurrentJobIdsHandler extends AbstractJsonRequestHandler {
if (jobManager != null) {
Future<Object> future = jobManager.ask(RequestJobsWithIDsOverview.getInstance(), timeout);
JobsWithIDsOverview overview = (JobsWithIDsOverview) Await.result(future, timeout);
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeArrayFieldStart("jobs-running");
for (JobID jid : overview.getJobsRunningOrPending()) {
gen.writeString(jid.toString());
}
gen.writeEndArray();
gen.writeArrayFieldStart("jobs-finished");
for (JobID jid : overview.getJobsFinished()) {
gen.writeString(jid.toString());
}
gen.writeEndArray();
gen.writeArrayFieldStart("jobs-cancelled");
for (JobID jid : overview.getJobsCancelled()) {
gen.writeString(jid.toString());
}
gen.writeEndArray();
gen.writeArrayFieldStart("jobs-failed");
for (JobID jid : overview.getJobsFailed()) {
gen.writeString(jid.toString());
}
gen.writeEndArray();
gen.writeEndObject();
gen.close();
return writer.toString();
}
......
......@@ -18,19 +18,17 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.messages.webmonitor.RequestJobDetails;
import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
......@@ -38,6 +36,10 @@ import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
......@@ -50,11 +52,10 @@ public class CurrentJobsOverviewHandler extends AbstractJsonRequestHandler {
private static final String COMPLETED_JOBS_REST_PATH = "/joboverview/completed";
private final FiniteDuration timeout;
private final boolean includeRunningJobs;
private final boolean includeFinishedJobs;
public CurrentJobsOverviewHandler(
FiniteDuration timeout,
boolean includeRunningJobs,
......@@ -83,23 +84,22 @@ public class CurrentJobsOverviewHandler extends AbstractJsonRequestHandler {
if (jobManager != null) {
Future<Object> future = jobManager.ask(
new RequestJobDetails(includeRunningJobs, includeFinishedJobs), timeout);
MultipleJobsDetails result = (MultipleJobsDetails) Await.result(future, timeout);
final long now = System.currentTimeMillis();
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
if (includeRunningJobs && includeFinishedJobs) {
gen.writeArrayFieldStart("running");
for (JobDetails detail : result.getRunningJobs()) {
writeJobDetailOverviewAsJson(detail, gen, now);
}
gen.writeEndArray();
gen.writeArrayFieldStart("finished");
for (JobDetails detail : result.getFinishedJobs()) {
writeJobDetailOverviewAsJson(detail, gen, now);
......@@ -113,7 +113,7 @@ public class CurrentJobsOverviewHandler extends AbstractJsonRequestHandler {
}
gen.writeEndArray();
}
gen.writeEndObject();
gen.close();
return writer.toString();
......@@ -127,12 +127,15 @@ public class CurrentJobsOverviewHandler extends AbstractJsonRequestHandler {
}
}
/**
* Archivist for the CurrentJobsOverviewHandler.
*/
public static class CurrentJobsOverviewJsonArchivist implements JsonArchivist {
@Override
public Collection<ArchivedJson> archiveJsonWithPath(AccessExecutionGraph graph) throws IOException {
StringWriter writer = new StringWriter();
try (JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer)) {
try (JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer)) {
gen.writeStartObject();
gen.writeArrayFieldStart("running");
gen.writeEndArray();
......
......@@ -18,10 +18,11 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.util.EnvironmentInformation;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Map;
......@@ -34,10 +35,10 @@ import java.util.TimeZone;
*/
public class DashboardConfigHandler extends AbstractJsonRequestHandler {
private static String DASHBOARD_CONFIG_REST_PATH = "/config";
private static final String DASHBOARD_CONFIG_REST_PATH = "/config";
private final String configString;
public DashboardConfigHandler(long refreshInterval) {
try {
this.configString = createConfigJson(refreshInterval);
......@@ -52,7 +53,7 @@ public class DashboardConfigHandler extends AbstractJsonRequestHandler {
public String[] getPaths() {
return new String[]{DASHBOARD_CONFIG_REST_PATH};
}
@Override
public String handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, ActorGateway jobManager) throws Exception {
return this.configString;
......@@ -60,8 +61,8 @@ public class DashboardConfigHandler extends AbstractJsonRequestHandler {
public static String createConfigJson(long refreshInterval) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
TimeZone timeZone = TimeZone.getDefault();
String timeZoneName = timeZone.getDisplayName();
long timeZoneOffset = timeZone.getRawOffset();
......
......@@ -18,23 +18,25 @@
package org.apache.flink.runtime.webmonitor.handlers;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.files.MimeTypes;
import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpResponse;
import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.HttpVersion;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.files.MimeTypes;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Tuple2;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import scala.Tuple2;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
......@@ -48,8 +50,8 @@ public class HandlerRedirectUtils {
private static final Logger LOG = LoggerFactory.getLogger(HandlerRedirectUtils.class);
/** Pattern to extract the host from an remote Akka URL */
private final static Pattern LeaderAddressHostPattern = Pattern.compile("^.+@(.+):([0-9]+)/user/.+$");
/** Pattern to extract the host from an remote Akka URL. */
private static final Pattern LeaderAddressHostPattern = Pattern.compile("^.+@(.+):([0-9]+)/user/.+$");
public static String getRedirectAddress(
String localJobManagerAddress,
......
......@@ -22,6 +22,9 @@ import org.apache.flink.runtime.instance.ActorGateway;
import java.util.Map;
/**
* Handler to deny access to jar-related REST calls.
*/
public class JarAccessDeniedHandler extends AbstractJsonRequestHandler {
private static final String ERROR_MESSAGE = "{\"error\": \"Web submission interface is not " +
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.client.program.ClusterClient;
......@@ -38,6 +37,8 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
import org.apache.flink.util.ExceptionUtils;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.File;
import java.io.StringWriter;
import java.net.URISyntaxException;
......@@ -50,10 +51,9 @@ import java.util.Map;
* Abstract handler for fetching plan for a jar or running a jar.
*/
public abstract class JarActionHandler extends AbstractJsonRequestHandler {
private final File jarDir;
public JarActionHandler(File jarDirectory) {
jarDir = jarDirectory;
}
......@@ -96,13 +96,13 @@ public abstract class JarActionHandler extends AbstractJsonRequestHandler {
protected String sendError(Exception e) throws Exception {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("error", ExceptionUtils.stringifyException(e));
gen.writeEndObject();
gen.close();
return writer.toString();
}
......@@ -162,7 +162,7 @@ public abstract class JarActionHandler extends AbstractJsonRequestHandler {
// Program args
String[] programArgs = new String[0];
String programArgsOpt = queryParams.get("program-args");
if (programArgsOpt!= null && !programArgsOpt.equals("")) {
if (programArgsOpt != null && !programArgsOpt.equals("")) {
List<String> args = tokenizeArguments(programArgsOpt);
programArgs = args.toArray(new String[args.size()]);
}
......
......@@ -18,9 +18,10 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.instance.ActorGateway;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.File;
import java.io.FilenameFilter;
import java.io.StringWriter;
......@@ -60,7 +61,7 @@ public class JarDeleteHandler extends AbstractJsonRequestHandler {
success = success || f.delete();
}
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
if (!success) {
// this seems to always fail on Windows.
......
......@@ -18,11 +18,12 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException;
......@@ -31,6 +32,9 @@ import java.util.Map;
import java.util.jar.JarFile;
import java.util.jar.Manifest;
/**
* Handle request for listing uploaded jars.
*/
public class JarListHandler extends AbstractJsonRequestHandler {
static final String JAR_LIST_REST_PATH = "/jars";
......@@ -50,8 +54,8 @@ public class JarListHandler extends AbstractJsonRequestHandler {
public String handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, ActorGateway jobManager) throws Exception {
try {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("address", queryParams.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY));
gen.writeArrayFieldStart("files");
......@@ -62,11 +66,11 @@ public class JarListHandler extends AbstractJsonRequestHandler {
return name.endsWith(".jar");
}
});
for (File f : list) {
// separate the uuid and the name parts.
String id = f.getName();
int startIndex = id.indexOf("_");
if (startIndex < 0) {
continue;
......@@ -75,13 +79,13 @@ public class JarListHandler extends AbstractJsonRequestHandler {
if (name.length() < 5 || !name.endsWith(".jar")) {
continue;
}
gen.writeStartObject();
gen.writeStringField("id", id);
gen.writeStringField("name", name);
gen.writeNumberField("uploaded", f.lastModified());
gen.writeArrayFieldStart("entry");
String[] classes = new String[0];
try {
JarFile jar = new JarFile(f);
......
......@@ -18,11 +18,12 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.File;
import java.io.StringWriter;
import java.util.Map;
......@@ -49,7 +50,7 @@ public class JarPlanHandler extends JarActionHandler {
JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams);
JobGraph graph = getJobGraphAndClassLoader(config).f0;
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeFieldName("plan");
gen.writeRawValue(JsonPlanGenerator.generatePlan(graph));
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.configuration.Configuration;
......@@ -26,13 +25,16 @@ import org.apache.flink.runtime.client.JobClient;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.jobgraph.JobGraph;
import scala.concurrent.duration.FiniteDuration;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.File;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Map;
import scala.concurrent.duration.FiniteDuration;
/**
* This handler handles requests to fetch plan for a jar.
*/
......@@ -72,7 +74,7 @@ public class JarRunHandler extends JarActionHandler {
}
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("jobid", graph.f0.getJobID().toString());
gen.writeEndObject();
......
......@@ -47,10 +47,10 @@ public class JarUploadHandler extends AbstractJsonRequestHandler {
Map<String, String> pathParams,
Map<String, String> queryParams,
ActorGateway jobManager) throws Exception {
String tempFilePath = queryParams.get("filepath");
String filename = queryParams.get("filename");
File tempFile;
if (tempFilePath != null && (tempFile = new File(tempFilePath)).exists()) {
if (!tempFile.getName().endsWith(".jar")) {
......@@ -58,7 +58,7 @@ public class JarUploadHandler extends AbstractJsonRequestHandler {
tempFile.delete();
return "{\"error\": \"Only Jar files are allowed.\"}";
}
String filenameWithUUID = UUID.randomUUID() + "_" + filename;
File newFile = new File(jarDir, filenameWithUUID);
if (tempFile.renameTo(newFile)) {
......@@ -70,7 +70,7 @@ public class JarUploadHandler extends AbstractJsonRequestHandler {
tempFile.delete();
}
}
return "{\"error\": \"Failed to upload the file.\"}";
}
}
......@@ -18,13 +18,14 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Collection;
......@@ -37,7 +38,7 @@ import java.util.Map;
public class JobAccumulatorsHandler extends AbstractExecutionGraphRequestHandler {
private static final String JOB_ACCUMULATORS_REST_PATH = "/jobs/:jobid/accumulators";
public JobAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -52,6 +53,9 @@ public class JobAccumulatorsHandler extends AbstractExecutionGraphRequestHandler
return createJobAccumulatorsJson(graph);
}
/**
* Archivist for the JobAccumulatorsHandler.
*/
public static class JobAccumulatorsJsonArchivist implements JsonArchivist {
@Override
......@@ -65,7 +69,7 @@ public class JobAccumulatorsHandler extends AbstractExecutionGraphRequestHandler
public static String createJobAccumulatorsJson(AccessExecutionGraph graph) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
StringifiedAccumulatorResult[] allAccumulators = graph.getAccumulatorResultsStringified();
......@@ -74,7 +78,7 @@ public class JobAccumulatorsHandler extends AbstractExecutionGraphRequestHandler
gen.writeArrayFieldStart("job-accumulators");
// empty for now
gen.writeEndArray();
gen.writeArrayFieldStart("user-task-accumulators");
for (StringifiedAccumulatorResult acc : allAccumulators) {
gen.writeStartObject();
......
......@@ -18,14 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import akka.dispatch.OnComplete;
import com.fasterxml.jackson.core.JsonGenerator;
import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.FullHttpResponse;
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.api.common.JobID;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.ConfigConstants;
......@@ -36,11 +28,18 @@ import org.apache.flink.runtime.messages.JobManagerMessages.CancelJobWithSavepoi
import org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure;
import org.apache.flink.runtime.messages.JobManagerMessages.CancellationSuccess;
import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import scala.concurrent.ExecutionContext;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import akka.dispatch.OnComplete;
import com.fasterxml.jackson.core.JsonGenerator;
import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.FullHttpResponse;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.HttpVersion;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.StringWriter;
import java.nio.charset.Charset;
......@@ -48,6 +47,10 @@ import java.util.ArrayDeque;
import java.util.HashMap;
import java.util.Map;
import scala.concurrent.ExecutionContext;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
......@@ -67,13 +70,13 @@ public class JobCancellationWithSavepointHandlers {
/** Shared lock between Trigger and In-Progress handlers. */
private final Object lock = new Object();
/** In-Progress requests */
/** In-Progress requests. */
private final Map<JobID, Long> inProgress = new HashMap<>();
/** Succeeded/failed request. Either String or Throwable. */
private final Map<Long, Object> completed = new HashMap<>();
/** Atomic request counter */
/** Atomic request counter. */
private long requestCounter;
/** Handler for trigger requests. */
......@@ -244,7 +247,7 @@ public class JobCancellationWithSavepointHandlers {
// Accepted response
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("status", "accepted");
gen.writeNumberField("request-id", requestId);
......@@ -283,7 +286,7 @@ public class JobCancellationWithSavepointHandlers {
/** The number of recent checkpoints whose IDs are remembered. */
private static final int NUM_GHOST_REQUEST_IDS = 16;
/** Remember some recently completed */
/** Remember some recently completed. */
private final ArrayDeque<Tuple2<Long, Object>> recentlyCompleted = new ArrayDeque<>(NUM_GHOST_REQUEST_IDS);
@Override
......@@ -324,7 +327,7 @@ public class JobCancellationWithSavepointHandlers {
if (inProgressRequestId == requestId) {
return createInProgressResponse(requestId);
} else {
String msg= "Request ID does not belong to JobID";
String msg = "Request ID does not belong to JobID";
return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, msg);
}
}
......@@ -355,7 +358,7 @@ public class JobCancellationWithSavepointHandlers {
private FullHttpResponse createSuccessResponse(long requestId, String savepointPath) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("status", "success");
......@@ -381,7 +384,7 @@ public class JobCancellationWithSavepointHandlers {
private FullHttpResponse createInProgressResponse(long requestId) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("status", "in-progress");
......@@ -406,7 +409,7 @@ public class JobCancellationWithSavepointHandlers {
private FullHttpResponse createFailureResponse(HttpResponseStatus code, long requestId, String errMsg) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("status", "failed");
......
......@@ -18,19 +18,19 @@
package org.apache.flink.runtime.webmonitor.handlers;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Map;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.api.common.ArchivedExecutionConfig;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
/**
* Request handler that returns the execution config of a job.
......@@ -53,6 +53,9 @@ public class JobConfigHandler extends AbstractExecutionGraphRequestHandler {
return createJobConfigJson(graph);
}
/**
* Archivist for the JobConfigHandler.
*/
public static class JobConfigJsonArchivist implements JsonArchivist {
@Override
......@@ -66,7 +69,7 @@ public class JobConfigHandler extends AbstractExecutionGraphRequestHandler {
public static String createJobConfigJson(AccessExecutionGraph graph) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("jid", graph.getJobID().toString());
......@@ -86,7 +89,7 @@ public class JobConfigHandler extends AbstractExecutionGraphRequestHandler {
Map<String, String> ucVals = summary.getGlobalJobParameters();
if (ucVals != null) {
gen.writeObjectFieldStart("user-config");
for (Map.Entry<String, String> ucVal : ucVals.entrySet()) {
gen.writeStringField(ucVal.getKey(), ucVal.getValue());
}
......@@ -97,7 +100,7 @@ public class JobConfigHandler extends AbstractExecutionGraphRequestHandler {
gen.writeEndObject();
}
gen.writeEndObject();
gen.close();
return writer.toString();
}
......
......@@ -18,8 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
......@@ -32,7 +30,10 @@ import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
import org.apache.flink.runtime.webmonitor.utils.MutableIOMetrics;
import com.fasterxml.jackson.core.JsonGenerator;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -40,7 +41,7 @@ import java.util.Collection;
import java.util.Map;
/**
* Request handler that returns details about a job, including:
* Request handler that returns details about a job. This includes:
* <ul>
* <li>Dataflow plan</li>
* <li>id, name, and current status</li>
......@@ -71,6 +72,9 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
return createJobDetailsJson(graph, fetcher);
}
/**
* Archivist for the JobDetailsHandler.
*/
public static class JobDetailsJsonArchivist implements JsonArchivist {
@Override
......@@ -89,18 +93,18 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
public static String createJobDetailsJson(AccessExecutionGraph graph, @Nullable MetricFetcher fetcher) throws IOException {
final StringWriter writer = new StringWriter();
final JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
final JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
final long now = System.currentTimeMillis();
gen.writeStartObject();
// basic info
gen.writeStringField("jid", graph.getJobID().toString());
gen.writeStringField("name", graph.getJobName());
gen.writeBooleanField("isStoppable", graph.isStoppable());
gen.writeStringField("state", graph.getState().name());
// times and duration
final long jobStartTime = graph.getStatusTimestamp(JobStatus.CREATED);
final long jobEndTime = graph.getState().isGloballyTerminalState() ?
......@@ -109,14 +113,14 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
gen.writeNumberField("end-time", jobEndTime);
gen.writeNumberField("duration", (jobEndTime > 0 ? jobEndTime : now) - jobStartTime);
gen.writeNumberField("now", now);
// timestamps
gen.writeObjectFieldStart("timestamps");
for (JobStatus status : JobStatus.values()) {
gen.writeNumberField(status.name(), graph.getStatusTimestamp(status));
}
gen.writeEndObject();
// job vertices
int[] jobVerticesPerState = new int[ExecutionState.values().length];
gen.writeArrayFieldStart("vertices");
......@@ -126,7 +130,7 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
long startTime = Long.MAX_VALUE;
long endTime = 0;
boolean allFinished = true;
for (AccessExecutionVertex vertex : ejv.getTaskVertices()) {
final ExecutionState state = vertex.getExecutionState();
tasksPerState[state.ordinal()]++;
......@@ -136,11 +140,11 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
if (started > 0) {
startTime = Math.min(startTime, started);
}
allFinished &= state.isTerminal();
endTime = Math.max(endTime, vertex.getStateTimestamp(state));
}
long duration;
if (startTime < Long.MAX_VALUE) {
if (allFinished) {
......@@ -156,8 +160,8 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
endTime = -1L;
duration = -1L;
}
ExecutionState jobVertexState =
ExecutionState jobVertexState =
ExecutionJobVertex.getAggregateJobVertexState(tasksPerState, ejv.getParallelism());
jobVerticesPerState[jobVertexState.ordinal()]++;
......@@ -170,13 +174,13 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
gen.writeNumberField("start-time", startTime);
gen.writeNumberField("end-time", endTime);
gen.writeNumberField("duration", duration);
gen.writeObjectFieldStart("tasks");
for (ExecutionState state : ExecutionState.values()) {
gen.writeNumberField(state.name(), tasksPerState[state.ordinal()]);
}
gen.writeEndObject();
MutableIOMetrics counts = new MutableIOMetrics();
for (AccessExecutionVertex vertex : ejv.getTaskVertices()) {
......@@ -188,7 +192,7 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler {
}
counts.writeIOMetricsAsJson(gen);
gen.writeEndObject();
}
gen.writeEndArray();
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
......@@ -27,6 +26,8 @@ import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.util.ExceptionUtils;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Collection;
......@@ -41,7 +42,7 @@ public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
private static final String JOB_EXCEPTIONS_REST_PATH = "/jobs/:jobid/exceptions";
static final int MAX_NUMBER_EXCEPTION_TO_REPORT = 20;
public JobExceptionsHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -56,6 +57,9 @@ public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
return createJobExceptionsJson(graph);
}
/**
* Archivist for the JobExceptionsHandler.
*/
public static class JobExceptionsJsonArchivist implements JsonArchivist {
@Override
......@@ -69,10 +73,10 @@ public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
public static String createJobExceptionsJson(AccessExecutionGraph graph) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
// most important is the root failure cause
String rootException = graph.getFailureCauseAsString();
if (rootException != null && !rootException.equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
......@@ -84,7 +88,7 @@ public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler {
int numExceptionsSoFar = 0;
boolean truncated = false;
for (AccessExecutionVertex task : graph.getAllExecutionVertices()) {
String t = task.getFailureCauseAsString();
if (t != null && !t.equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) {
......
......@@ -18,10 +18,11 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.instance.ActorGateway;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.StringWriter;
import java.util.Map;
......@@ -46,7 +47,7 @@ public class JobManagerConfigHandler extends AbstractJsonRequestHandler {
@Override
public String handleJsonRequest(Map<String, String> pathParams, Map<String, String> queryParams, ActorGateway jobManager) throws Exception {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartArray();
for (String key : config.keySet()) {
......@@ -54,9 +55,9 @@ public class JobManagerConfigHandler extends AbstractJsonRequestHandler {
gen.writeStringField("key", key);
// Mask key values which contain sensitive information
if(key.toLowerCase().contains("password")) {
if (key.toLowerCase().contains("password")) {
String value = config.getString(key, null);
if(value != null) {
if (value != null) {
value = "******";
}
gen.writeStringField("value", value);
......
......@@ -34,7 +34,7 @@ import java.util.Map;
public class JobPlanHandler extends AbstractExecutionGraphRequestHandler {
private static final String JOB_PLAN_REST_PATH = "/jobs/:jobid/plan";
public JobPlanHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -49,6 +49,9 @@ public class JobPlanHandler extends AbstractExecutionGraphRequestHandler {
return graph.getJsonPlan();
}
/**
* Archivist for the JobPlanHandler.
*/
public static class JobPlanJsonArchivist implements JsonArchivist {
@Override
......
......@@ -18,8 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
......@@ -27,6 +25,8 @@ import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -34,11 +34,13 @@ import java.util.Collection;
import java.util.List;
import java.util.Map;
/**
* Request handler that returns the accummulators for a given vertex.
*/
public class JobVertexAccumulatorsHandler extends AbstractJobVertexRequestHandler {
private static final String JOB_VERTEX_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/accumulators";
public JobVertexAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -53,6 +55,9 @@ public class JobVertexAccumulatorsHandler extends AbstractJobVertexRequestHandle
return createVertexAccumulatorsJson(jobVertex);
}
/**
* Archivist for JobVertexAccumulatorsHandler.
*/
public static class JobVertexAccumulatorsJsonArchivist implements JsonArchivist {
@Override
......@@ -71,13 +76,13 @@ public class JobVertexAccumulatorsHandler extends AbstractJobVertexRequestHandle
public static String createVertexAccumulatorsJson(AccessExecutionJobVertex jobVertex) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
StringifiedAccumulatorResult[] accs = jobVertex.getAggregatedUserAccumulatorsStringified();
gen.writeStartObject();
gen.writeStringField("id", jobVertex.getJobVertexId().toString());
gen.writeArrayFieldStart("user-accumulators");
for (StringifiedAccumulatorResult acc : accs) {
gen.writeStartObject();
......@@ -87,7 +92,7 @@ public class JobVertexAccumulatorsHandler extends AbstractJobVertexRequestHandle
gen.writeEndObject();
}
gen.writeEndArray();
gen.writeEndObject();
gen.close();
......
......@@ -18,18 +18,20 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
import org.apache.flink.runtime.webmonitor.BackPressureStatsTracker;
import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import org.apache.flink.runtime.webmonitor.OperatorBackPressureStats;
import scala.Option;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.StringWriter;
import java.util.Map;
import scala.Option;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
......@@ -72,7 +74,7 @@ public class JobVertexBackPressureHandler extends AbstractJobVertexRequestHandle
}
ExecutionJobVertex jobVertex = (ExecutionJobVertex) accessJobVertex;
try (StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer)) {
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer)) {
gen.writeStartObject();
......
......@@ -18,8 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
......@@ -31,7 +29,10 @@ import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
import org.apache.flink.runtime.webmonitor.utils.MutableIOMetrics;
import com.fasterxml.jackson.core.JsonGenerator;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -45,7 +46,7 @@ import java.util.Map;
*/
public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
private static String JOB_VERTEX_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid";
private static final String JOB_VERTEX_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid";
private final MetricFetcher fetcher;
......@@ -64,6 +65,9 @@ public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
return createVertexDetailsJson(jobVertex, params.get("jobid"), fetcher);
}
/**
* Archivist for the JobVertexDetailsHandler.
*/
public static class JobVertexDetailsJsonArchivist implements JsonArchivist {
@Override
......@@ -85,9 +89,9 @@ public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
String jobID,
@Nullable MetricFetcher fetcher) throws IOException {
final long now = System.currentTimeMillis();
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
......@@ -100,7 +104,7 @@ public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
int num = 0;
for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
final ExecutionState status = vertex.getExecutionState();
TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
String locationString = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort();
......@@ -110,7 +114,7 @@ public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
}
long endTime = status.isTerminal() ? vertex.getStateTimestamp(status) : -1;
long duration = startTime > 0 ? ((endTime > 0 ? endTime : now) - startTime) : -1;
gen.writeStartObject();
gen.writeNumberField("subtask", num);
gen.writeStringField("status", status.name());
......@@ -130,13 +134,13 @@ public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler {
);
counts.writeIOMetricsAsJson(gen);
gen.writeEndObject();
num++;
}
gen.writeEndArray();
gen.writeEndObject();
gen.close();
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
......@@ -31,7 +30,10 @@ import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
import org.apache.flink.runtime.webmonitor.utils.MutableIOMetrics;
import com.fasterxml.jackson.core.JsonGenerator;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -65,6 +67,9 @@ public class JobVertexTaskManagersHandler extends AbstractJobVertexRequestHandle
return createVertexDetailsByTaskManagerJson(jobVertex, params.get("jobid"), fetcher);
}
/**
* Archivist for JobVertexTaskManagersHandler.
*/
public static class JobVertexTaskManagersJsonArchivist implements JsonArchivist {
@Override
......@@ -86,7 +91,7 @@ public class JobVertexTaskManagersHandler extends AbstractJobVertexRequestHandle
String jobID,
@Nullable MetricFetcher fetcher) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
// Build a map that groups tasks by TaskManager
Map<String, List<AccessExecutionVertex>> taskManagerVertices = new HashMap<>();
......@@ -108,7 +113,6 @@ public class JobVertexTaskManagersHandler extends AbstractJobVertexRequestHandle
// Build JSON response
final long now = System.currentTimeMillis();
gen.writeStartObject();
gen.writeStringField("id", jobVertex.getJobVertexId().toString());
......
......@@ -26,11 +26,10 @@ package org.apache.flink.runtime.webmonitor.handlers;
public class JsonFactory {
/** The singleton Jackson JSON factory. */
public static final com.fasterxml.jackson.core.JsonFactory jacksonFactory =
public static final com.fasterxml.jackson.core.JsonFactory JACKSON_FACTORY =
new com.fasterxml.jackson.core.JsonFactory();
// --------------------------------------------------------------------------------------------
/** Don't instantiate */
private JsonFactory() {}
}
......@@ -18,9 +18,10 @@
package org.apache.flink.runtime.webmonitor.handlers;
import io.netty.handler.codec.http.FullHttpResponse;
import org.apache.flink.runtime.instance.ActorGateway;
import io.netty.handler.codec.http.FullHttpResponse;
import java.util.Map;
/**
......@@ -36,13 +37,13 @@ public interface RequestHandler {
* respond with a full http response, including content-type, content-length, etc.
*
* <p>Exceptions may be throws and will be handled.
*
*
* @param pathParams The map of REST path parameters, decoded by the router.
* @param queryParams The map of query parameters.
* @param jobManager The JobManager actor.
*
* @return The full http response.
*
*
* @throws Exception Handlers may forward exceptions. Exceptions of type
* {@link org.apache.flink.runtime.webmonitor.NotFoundException} will cause a HTTP 404
* response with the exception message, other exceptions will cause a HTTP 500 response
......
......@@ -30,7 +30,7 @@ import java.util.Map;
public class SubtaskCurrentAttemptDetailsHandler extends SubtaskExecutionAttemptDetailsHandler {
public static final String SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum";
public SubtaskCurrentAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, MetricFetcher fetcher) {
super(executionGraphHolder, fetcher);
}
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
import org.apache.flink.runtime.executiongraph.AccessExecution;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
......@@ -28,6 +27,8 @@ import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -37,12 +38,12 @@ import java.util.Map;
/**
* Base class for request handlers whose response depends on a specific job vertex (defined
* via the "vertexid" parameter) in a specific job, defined via (defined voa the "jobid" parameter).
* via the "vertexid" parameter) in a specific job, defined via (defined voa the "jobid" parameter).
*/
public class SubtaskExecutionAttemptAccumulatorsHandler extends AbstractSubtaskAttemptRequestHandler {
private static final String SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt/accumulators";
public SubtaskExecutionAttemptAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -56,7 +57,10 @@ public class SubtaskExecutionAttemptAccumulatorsHandler extends AbstractSubtaskA
public String handleRequest(AccessExecution execAttempt, Map<String, String> params) throws Exception {
return createAttemptAccumulatorsJson(execAttempt);
}
/**
* Archivist for the SubtaskExecutionAttemptAccumulatorsHandler.
*/
public static class SubtaskExecutionAttemptAccumulatorsJsonArchivist implements JsonArchivist {
@Override
......@@ -91,8 +95,8 @@ public class SubtaskExecutionAttemptAccumulatorsHandler extends AbstractSubtaskA
public static String createAttemptAccumulatorsJson(AccessExecution execAttempt) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
final StringifiedAccumulatorResult[] accs = execAttempt.getUserAccumulatorsStringified();
gen.writeStartObject();
......@@ -100,7 +104,7 @@ public class SubtaskExecutionAttemptAccumulatorsHandler extends AbstractSubtaskA
gen.writeNumberField("subtask", execAttempt.getParallelSubtaskIndex());
gen.writeNumberField("attempt", execAttempt.getAttemptNumber());
gen.writeStringField("id", execAttempt.getAttemptId().toString());
gen.writeArrayFieldStart("user-accumulators");
for (StringifiedAccumulatorResult acc : accs) {
gen.writeStartObject();
......@@ -110,9 +114,9 @@ public class SubtaskExecutionAttemptAccumulatorsHandler extends AbstractSubtaskA
gen.writeEndObject();
}
gen.writeEndArray();
gen.writeEndObject();
gen.close();
return writer.toString();
}
......
......@@ -18,8 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecution;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
......@@ -32,7 +30,10 @@ import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
import org.apache.flink.runtime.webmonitor.utils.MutableIOMetrics;
import com.fasterxml.jackson.core.JsonGenerator;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -66,6 +67,9 @@ public class SubtaskExecutionAttemptDetailsHandler extends AbstractSubtaskAttemp
return createAttemptDetailsJson(execAttempt, params.get("jobid"), params.get("vertexid"), fetcher);
}
/**
* Archivist for the SubtaskExecutionAttemptDetailsHandler.
*/
public static class SubtaskExecutionAttemptDetailsJsonArchivist implements JsonArchivist {
@Override
......@@ -83,7 +87,7 @@ public class SubtaskExecutionAttemptDetailsHandler extends AbstractSubtaskAttemp
.replace(":vertexid", task.getJobVertexId().toString())
.replace(":subtasknum", String.valueOf(subtask.getParallelSubtaskIndex()))
.replace(":attempt", String.valueOf(subtask.getCurrentExecutionAttempt().getAttemptNumber()));
archive.add(new ArchivedJson(curAttemptPath1, curAttemptJson));
archive.add(new ArchivedJson(curAttemptPath2, curAttemptJson));
......@@ -109,7 +113,7 @@ public class SubtaskExecutionAttemptDetailsHandler extends AbstractSubtaskAttemp
String vertexID,
@Nullable MetricFetcher fetcher) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
final ExecutionState status = execAttempt.getState();
final long now = System.currentTimeMillis();
......@@ -141,7 +145,7 @@ public class SubtaskExecutionAttemptDetailsHandler extends AbstractSubtaskAttemp
jobID,
vertexID
);
counts.writeIOMetricsAsJson(gen);
gen.writeEndObject();
......
......@@ -18,8 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
......@@ -29,6 +27,8 @@ import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -42,7 +42,7 @@ import java.util.Map;
public class SubtasksAllAccumulatorsHandler extends AbstractJobVertexRequestHandler {
private static final String SUBTASKS_ALL_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/accumulators";
public SubtasksAllAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -57,6 +57,9 @@ public class SubtasksAllAccumulatorsHandler extends AbstractJobVertexRequestHand
return createSubtasksAccumulatorsJson(jobVertex);
}
/**
* Archivist for the SubtasksAllAccumulatorsHandler.
*/
public static class SubtasksAllAccumulatorsJsonArchivist implements JsonArchivist {
@Override
......@@ -75,22 +78,22 @@ public class SubtasksAllAccumulatorsHandler extends AbstractJobVertexRequestHand
public static String createSubtasksAccumulatorsJson(AccessExecutionJobVertex jobVertex) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("id", jobVertex.getJobVertexId().toString());
gen.writeNumberField("parallelism", jobVertex.getParallelism());
gen.writeArrayFieldStart("subtasks");
int num = 0;
for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
String locationString = location == null ? "(unassigned)" : location.getHostname();
gen.writeStartObject();
gen.writeNumberField("subtask", num++);
gen.writeNumberField("attempt", vertex.getCurrentExecutionAttempt().getAttemptNumber());
gen.writeStringField("host", locationString);
......@@ -105,7 +108,7 @@ public class SubtasksAllAccumulatorsHandler extends AbstractJobVertexRequestHand
gen.writeEndObject();
}
gen.writeEndArray();
gen.writeEndObject();
}
gen.writeEndArray();
......
......@@ -18,8 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
......@@ -29,6 +27,8 @@ import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -43,7 +43,7 @@ import java.util.Map;
public class SubtasksTimesHandler extends AbstractJobVertexRequestHandler {
private static final String SUBTASK_TIMES_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasktimes";
public SubtasksTimesHandler(ExecutionGraphHolder executionGraphHolder) {
super(executionGraphHolder);
}
......@@ -58,6 +58,9 @@ public class SubtasksTimesHandler extends AbstractJobVertexRequestHandler {
return createSubtaskTimesJson(jobVertex);
}
/**
* Archivist for the SubtasksTimesHandler.
*/
public static class SubtasksTimesJsonArchivist implements JsonArchivist {
@Override
......@@ -78,28 +81,28 @@ public class SubtasksTimesHandler extends AbstractJobVertexRequestHandler {
final long now = System.currentTimeMillis();
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeStringField("id", jobVertex.getJobVertexId().toString());
gen.writeStringField("name", jobVertex.getName());
gen.writeNumberField("now", now);
gen.writeArrayFieldStart("subtasks");
int num = 0;
for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
long[] timestamps = vertex.getCurrentExecutionAttempt().getStateTimestamps();
ExecutionState status = vertex.getExecutionState();
long scheduledTime = timestamps[ExecutionState.SCHEDULED.ordinal()];
long start = scheduledTime > 0 ? scheduledTime : -1;
long end = status.isTerminal() ? timestamps[status.ordinal()] : now;
long duration = start >= 0 ? end - start : -1L;
gen.writeStartObject();
gen.writeNumberField("subtask", num++);
......@@ -108,13 +111,13 @@ public class SubtasksTimesHandler extends AbstractJobVertexRequestHandler {
gen.writeStringField("host", locationString);
gen.writeNumberField("duration", duration);
gen.writeObjectFieldStart("timestamps");
for (ExecutionState state : ExecutionState.values()) {
gen.writeNumberField(state.name(), timestamps[state.ordinal()]);
}
gen.writeEndObject();
gen.writeEndObject();
}
......
......@@ -26,24 +26,6 @@ package org.apache.flink.runtime.webmonitor.handlers;
* https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
*****************************************************************************/
import akka.dispatch.Mapper;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.DefaultFileRegion;
import io.netty.handler.codec.http.DefaultHttpResponse;
import io.netty.handler.codec.http.HttpChunkedInput;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpResponse;
import io.netty.handler.codec.http.LastHttpContent;
import io.netty.handler.codec.http.router.Routed;
import io.netty.handler.ssl.SslHandler;
import io.netty.handler.stream.ChunkedFile;
import io.netty.util.concurrent.GenericFutureListener;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.ConfigConstants;
......@@ -65,12 +47,27 @@ import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandlerBase;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.StringUtils;
import akka.dispatch.Mapper;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.DefaultFileRegion;
import io.netty.handler.codec.http.DefaultHttpResponse;
import io.netty.handler.codec.http.HttpChunkedInput;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpResponse;
import io.netty.handler.codec.http.LastHttpContent;
import io.netty.handler.codec.http.router.Routed;
import io.netty.handler.ssl.SslHandler;
import io.netty.handler.stream.ChunkedFile;
import io.netty.util.concurrent.GenericFutureListener;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Option;
import scala.concurrent.ExecutionContextExecutor;
import scala.concurrent.duration.FiniteDuration;
import scala.reflect.ClassTag$;
import java.io.File;
import java.io.FileNotFoundException;
......@@ -81,6 +78,11 @@ import java.nio.channels.FileChannel;
import java.util.HashMap;
import java.util.concurrent.ConcurrentHashMap;
import scala.Option;
import scala.concurrent.ExecutionContextExecutor;
import scala.concurrent.duration.FiniteDuration;
import scala.reflect.ClassTag$;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
import static io.netty.handler.codec.http.HttpResponseStatus.OK;
......@@ -100,19 +102,19 @@ public class TaskManagerLogHandler extends RuntimeMonitorHandlerBase {
private static final String TASKMANAGER_LOG_REST_PATH = "/taskmanagers/:taskmanagerid/log";
private static final String TASKMANAGER_OUT_REST_PATH = "/taskmanagers/:taskmanagerid/stdout";
/** Keep track of last transmitted log, to clean up old ones */
/** Keep track of last transmitted log, to clean up old ones. */
private final HashMap<String, BlobKey> lastSubmittedLog = new HashMap<>();
private final HashMap<String, BlobKey> lastSubmittedStdout = new HashMap<>();
/** Keep track of request status, prevents multiple log requests for a single TM running concurrently */
/** Keep track of request status, prevents multiple log requests for a single TM running concurrently. */
private final ConcurrentHashMap<String, Boolean> lastRequestPending = new ConcurrentHashMap<>();
private final Configuration config;
/** Future of the blob cache */
/** Future of the blob cache. */
private Future<BlobCache> cache;
/** Indicates which log file should be displayed; true indicates .log, false indicates .out */
private boolean serveLogFile;
/** Indicates which log file should be displayed. */
private FileMode fileMode;
private final ExecutionContextExecutor executor;
......@@ -120,6 +122,7 @@ public class TaskManagerLogHandler extends RuntimeMonitorHandlerBase {
private final BlobView blobView;
/** Used to control whether this handler serves the .log or .out file. */
public enum FileMode {
LOG,
STDOUT
......@@ -138,14 +141,7 @@ public class TaskManagerLogHandler extends RuntimeMonitorHandlerBase {
this.executor = checkNotNull(executor);
this.config = config;
switch (fileMode) {
case LOG:
serveLogFile = true;
break;
case STDOUT:
serveLogFile = false;
break;
}
this.fileMode = fileMode;
this.blobView = Preconditions.checkNotNull(blobView, "blobView");
......@@ -154,10 +150,12 @@ public class TaskManagerLogHandler extends RuntimeMonitorHandlerBase {
@Override
public String[] getPaths() {
if (serveLogFile) {
return new String[]{TASKMANAGER_LOG_REST_PATH};
} else {
return new String[]{TASKMANAGER_OUT_REST_PATH};
switch (fileMode) {
case LOG:
return new String[]{TASKMANAGER_LOG_REST_PATH};
case STDOUT:
default:
return new String[]{TASKMANAGER_OUT_REST_PATH};
}
}
......@@ -199,10 +197,12 @@ public class TaskManagerLogHandler extends RuntimeMonitorHandlerBase {
public Future<BlobKey> apply(JobManagerMessages.TaskManagerInstance value) {
Instance taskManager = value.instance().get();
if (serveLogFile) {
return taskManager.getTaskManagerGateway().requestTaskManagerLog(timeTimeout);
} else {
return taskManager.getTaskManagerGateway().requestTaskManagerStdout(timeTimeout);
switch (fileMode) {
case LOG:
return taskManager.getTaskManagerGateway().requestTaskManagerLog(timeTimeout);
case STDOUT:
default:
return taskManager.getTaskManagerGateway().requestTaskManagerStdout(timeTimeout);
}
}
});
......@@ -223,7 +223,7 @@ public class TaskManagerLogHandler extends RuntimeMonitorHandlerBase {
final BlobCache blobCache = value.f1;
//delete previous log file, if it is different than the current one
HashMap<String, BlobKey> lastSubmittedFile = serveLogFile ? lastSubmittedLog : lastSubmittedStdout;
HashMap<String, BlobKey> lastSubmittedFile = fileMode == FileMode.LOG ? lastSubmittedLog : lastSubmittedStdout;
if (lastSubmittedFile.containsKey(taskManagerID)) {
if (!blobKey.equals(lastSubmittedFile.get(taskManagerID))) {
try {
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.instance.Instance;
import org.apache.flink.runtime.instance.InstanceID;
......@@ -28,28 +27,34 @@ import org.apache.flink.runtime.messages.JobManagerMessages.TaskManagerInstance;
import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
import org.apache.flink.runtime.webmonitor.metrics.MetricStore;
import org.apache.flink.util.StringUtils;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.StringWriter;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import static java.util.Objects.requireNonNull;
/**
* A request handler that provides an overview over all taskmanagers or details for a single one.
*/
public class TaskManagersHandler extends AbstractJsonRequestHandler {
private static final String TASKMANAGERS_REST_PATH = "/taskmanagers";
private static final String TASKMANAGER_DETAILS_REST_PATH = "/taskmanagers/:taskmanagerid";
public static final String TASK_MANAGER_ID_KEY = "taskmanagerid";
private final FiniteDuration timeout;
private final MetricFetcher fetcher;
public TaskManagersHandler(FiniteDuration timeout, MetricFetcher fetcher) {
this.timeout = requireNonNull(timeout);
this.fetcher = fetcher;
......@@ -88,7 +93,7 @@ public class TaskManagersHandler extends AbstractJsonRequestHandler {
}
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeArrayFieldStart("taskmanagers");
......@@ -112,17 +117,17 @@ public class TaskManagersHandler extends AbstractJsonRequestHandler {
MetricStore.TaskManagerMetricStore metrics = fetcher.getMetricStore().getTaskManagerMetricStore(instance.getId().toString());
if (metrics != null) {
gen.writeObjectFieldStart("metrics");
long heapUsed = Long.valueOf( metrics.getMetric("Status.JVM.Memory.Heap.Used", "0"));
long heapCommitted = Long.valueOf( metrics.getMetric("Status.JVM.Memory.Heap.Committed", "0"));
long heapTotal = Long.valueOf( metrics.getMetric("Status.JVM.Memory.Heap.Max", "0"));
long heapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Used", "0"));
long heapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Committed", "0"));
long heapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Max", "0"));
gen.writeNumberField("heapCommitted", heapCommitted);
gen.writeNumberField("heapUsed", heapUsed);
gen.writeNumberField("heapMax", heapTotal);
long nonHeapUsed = Long.valueOf( metrics.getMetric("Status.JVM.Memory.NonHeap.Used", "0"));
long nonHeapCommitted = Long.valueOf( metrics.getMetric("Status.JVM.Memory.NonHeap.Committed", "0"));
long nonHeapTotal = Long.valueOf( metrics.getMetric("Status.JVM.Memory.NonHeap.Max", "0"));
long nonHeapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Used", "0"));
long nonHeapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Committed", "0"));
long nonHeapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Max", "0"));
gen.writeNumberField("nonHeapCommitted", nonHeapCommitted);
gen.writeNumberField("nonHeapUsed", nonHeapUsed);
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers.checkpoints;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
......@@ -28,6 +27,8 @@ import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Collection;
......@@ -55,6 +56,9 @@ public class CheckpointConfigHandler extends AbstractExecutionGraphRequestHandle
return createCheckpointConfigJson(graph);
}
/**
* Archivist for the CheckpointConfigHandler.
*/
public static class CheckpointConfigJsonArchivist implements JsonArchivist {
@Override
......@@ -68,7 +72,7 @@ public class CheckpointConfigHandler extends AbstractExecutionGraphRequestHandle
private static String createCheckpointConfigJson(AccessExecutionGraph graph) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
JobCheckpointingSettings settings = graph.getJobCheckpointingSettings();
if (settings == null) {
......
......@@ -18,9 +18,10 @@
package org.apache.flink.runtime.webmonitor.handlers.checkpoints;
import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
import javax.annotation.Nullable;
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers.checkpoints;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
......@@ -32,6 +31,8 @@ import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -86,6 +87,9 @@ public class CheckpointStatsDetailsHandler extends AbstractExecutionGraphRequest
return createCheckpointDetailsJson(checkpoint);
}
/**
* Archivist for the CheckpointStatsDetails.
*/
public static class CheckpointStatsDetailsJsonArchivist implements JsonArchivist {
@Override
......@@ -109,7 +113,7 @@ public class CheckpointStatsDetailsHandler extends AbstractExecutionGraphRequest
public static String createCheckpointDetailsJson(AbstractCheckpointStats checkpoint) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
gen.writeNumberField("id", checkpoint.getCheckpointId());
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers.checkpoints;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
......@@ -35,6 +34,8 @@ import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
......@@ -107,10 +108,13 @@ public class CheckpointStatsDetailsSubtasksHandler extends AbstractExecutionGrap
if (taskStats == null) {
return "{}";
}
return createSubtaskCheckpointDetailsJson(checkpoint, taskStats);
}
/**
* Archivist for the CheckpointStatsDetailsSubtasksHandler.
*/
public static class CheckpointStatsDetailsSubtasksJsonArchivist implements JsonArchivist {
@Override
......@@ -137,7 +141,7 @@ public class CheckpointStatsDetailsSubtasksHandler extends AbstractExecutionGrap
private static String createSubtaskCheckpointDetailsJson(AbstractCheckpointStats checkpoint, TaskStateStats taskStats) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartObject();
// Overview
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers.checkpoints;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.checkpoint.AbstractCheckpointStats;
import org.apache.flink.runtime.checkpoint.CheckpointStatsCounts;
import org.apache.flink.runtime.checkpoint.CheckpointStatsHistory;
......@@ -35,7 +34,10 @@ import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import com.fasterxml.jackson.core.JsonGenerator;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Collection;
......@@ -63,6 +65,9 @@ public class CheckpointStatsHandler extends AbstractExecutionGraphRequestHandler
return createCheckpointStatsJson(graph);
}
/**
* Archivist for the CheckpointStatsJsonHandler.
*/
public static class CheckpointStatsJsonArchivist implements JsonArchivist {
@Override
......@@ -76,7 +81,7 @@ public class CheckpointStatsHandler extends AbstractExecutionGraphRequestHandler
private static String createCheckpointStatsJson(AccessExecutionGraph graph) throws IOException {
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot();
if (snapshot == null) {
......
......@@ -15,9 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.history;
import io.netty.handler.codec.http.router.Router;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.Configuration;
......@@ -34,10 +34,13 @@ import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
import io.netty.handler.codec.http.router.Router;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.net.ssl.SSLContext;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
......@@ -53,15 +56,15 @@ import java.util.concurrent.atomic.AtomicBoolean;
/**
* The HistoryServer provides a WebInterface and REST API to retrieve information about finished jobs for which
* the JobManager may have already shut down.
*
* The HistoryServer regularly checks a set of directories for job archives created by the {@link FsJobArchivist} and
*
* <p>The HistoryServer regularly checks a set of directories for job archives created by the {@link FsJobArchivist} and
* caches these in a local directory. See {@link HistoryServerArchiveFetcher}.
*
* All configuration options are defined in{@link HistoryServerOptions}.
*
* The WebInterface only displays the "Completed Jobs" page.
*
* The REST API is limited to
*
* <p>All configuration options are defined in{@link HistoryServerOptions}.
*
* <p>The WebInterface only displays the "Completed Jobs" page.
*
* <p>The REST API is limited to
* <ul>
* <li>/config</li>
* <li>/joboverview</li>
......@@ -110,7 +113,7 @@ public class HistoryServer {
});
System.exit(0);
} catch (UndeclaredThrowableException ute) {
Throwable cause = ute. getUndeclaredThrowable();
Throwable cause = ute.getUndeclaredThrowable();
LOG.error("Failed to run HistoryServer.", cause);
cause.printStackTrace();
System.exit(1);
......
......@@ -15,14 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.history;
import static org.apache.flink.util.Preconditions.checkNotNull;
package org.apache.flink.runtime.webmonitor.history;
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.HistoryServerOptions;
import org.apache.flink.core.fs.FileStatus;
......@@ -32,6 +27,11 @@ import org.apache.flink.runtime.history.FsJobArchivist;
import org.apache.flink.runtime.util.ExecutorThreadFactory;
import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
import org.apache.flink.util.FileUtils;
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -49,12 +49,14 @@ import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* This class is used by the {@link HistoryServer} to fetch the job archives that are located at
* {@link HistoryServerOptions#HISTORY_SERVER_ARCHIVE_DIRS}. The directories are polled in regular intervals, defined
* by {@link HistoryServerOptions#HISTORY_SERVER_ARCHIVE_REFRESH_INTERVAL}.
*
* The archives are downloaded and expanded into a file structure analog to the REST API defined in the WebRuntimeMonitor.
*
* <p>The archives are downloaded and expanded into a file structure analog to the REST API defined in the WebRuntimeMonitor.
*/
class HistoryServerArchiveFetcher {
......@@ -174,7 +176,7 @@ class HistoryServerArchiveFetcher {
}
java.nio.file.Path targetPath = target.toPath();
// We overwrite existing files since this may be another attempt at fetching this archive.
// Existing files may be incomplete/corrupt.
if (Files.exists(targetPath)) {
......@@ -224,10 +226,10 @@ class HistoryServerArchiveFetcher {
* This method replicates the JSON response that would be given by the {@link CurrentJobsOverviewHandler} when
* listing both running and finished jobs.
*
* Every job archive contains a joboverview.json file containing the same structure. Since jobs are archived on
* <p>Every job archive contains a joboverview.json file containing the same structure. Since jobs are archived on
* their own however the list of finished jobs only contains a single job.
*
* For the display in the HistoryServer WebFrontend we have to combine these overviews.
* <p>For the display in the HistoryServer WebFrontend we have to combine these overviews.
*/
private static void updateJobOverview(File webDir) {
File webOverviewDir = new File(webDir, "overviews");
......
......@@ -26,6 +26,8 @@ package org.apache.flink.runtime.webmonitor.history;
* https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
*****************************************************************************/
import org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
......@@ -41,7 +43,6 @@ import io.netty.handler.codec.http.LastHttpContent;
import io.netty.handler.codec.http.router.Routed;
import io.netty.handler.ssl.SslHandler;
import io.netty.handler.stream.ChunkedFile;
import org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -70,10 +71,10 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* Simple file server handler used by the {@link HistoryServer} that serves requests to web frontend's static files,
* such as HTML, CSS, JS or JSON files.
*
* This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server
* <p>This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server
* example.
*
* This class is a copy of the {@link StaticFileServerHandler}. The differences are that the request path is
*
* <p>This class is a copy of the {@link StaticFileServerHandler}. The differences are that the request path is
* modified to end on ".json" if it does not have a filename extension; when "index.html" is requested we load
* "index_hs.html" instead to inject the modified HistoryServer WebInterface and that the caching of the "/joboverview"
* page is prevented.
......@@ -81,12 +82,12 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
@ChannelHandler.Sharable
public class HistoryServerStaticFileServerHandler extends SimpleChannelInboundHandler<Routed> {
/** Default logger, if none is specified */
/** Default logger, if none is specified. */
private static final Logger LOG = LoggerFactory.getLogger(HistoryServerStaticFileServerHandler.class);
// ------------------------------------------------------------------------
/** The path in which the static documents are */
/** The path in which the static documents are. */
private final File rootPath;
public HistoryServerStaticFileServerHandler(File rootPath) throws IOException {
......
......@@ -15,14 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.metrics;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.webmonitor.handlers.AbstractJsonRequestHandler;
import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
import org.apache.flink.util.Preconditions;
import com.fasterxml.jackson.core.JsonGenerator;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Map;
......@@ -30,10 +32,10 @@ import java.util.Map;
/**
* Abstract request handler that returns a list of all available metrics or the values for a set of metrics.
*
* If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* {@code [ { "id" : "X" } ] }
*
* If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* {@code /get?X,Y}
* The handler will then return a list containing the values of the requested metrics.
* {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
......@@ -65,8 +67,8 @@ public abstract class AbstractMetricsHandler extends AbstractJsonRequestHandler
private String getMetricsValues(Map<String, String> pathParams, String requestedMetricsList) throws IOException {
if (requestedMetricsList.isEmpty()) {
/**
* The WebInterface doesn't check whether the list of available metrics was empty. This can lead to a
/*
* The WebInterface doesn't check whether the list of available metrics was empty. This can lead to a
* request for which the "get" parameter is an empty string.
*/
return "";
......@@ -80,7 +82,7 @@ public abstract class AbstractMetricsHandler extends AbstractJsonRequestHandler
String[] requestedMetrics = requestedMetricsList.split(",");
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartArray();
for (String requestedMetric : requestedMetrics) {
......@@ -107,7 +109,7 @@ public abstract class AbstractMetricsHandler extends AbstractJsonRequestHandler
return "";
}
StringWriter writer = new StringWriter();
JsonGenerator gen = JsonFactory.jacksonFactory.createGenerator(writer);
JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer);
gen.writeStartArray();
for (String m : metrics.keySet()) {
......
......@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.metrics;
import java.util.Map;
......@@ -22,10 +23,10 @@ import java.util.Map;
/**
* Request handler that returns for the job manager a list of all available metrics or the values for a set of metrics.
*
* If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
*
* If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* {@code /get?X,Y}
* The handler will then return a list containing the values of the requested metrics.
* {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
......
......@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.metrics;
import java.util.Map;
......@@ -22,10 +23,10 @@ import java.util.Map;
/**
* Request handler that returns for a given job a list of all available metrics or the values for a set of metrics.
*
* If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
*
* If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* {@code /get?X,Y}
* The handler will then return a list containing the values of the requested metrics.
* {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
......
......@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.metrics;
import java.util.Map;
......@@ -22,10 +23,10 @@ import java.util.Map;
/**
* Request handler that returns for a given task a list of all available metrics or the values for a set of metrics.
*
* If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
*
* If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* {@code /get?X,Y}
* The handler will then return a list containing the values of the requested metrics.
* {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
......
......@@ -15,14 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.metrics;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.dispatch.OnFailure;
import akka.dispatch.OnSuccess;
import akka.pattern.Patterns;
import akka.util.Timeout;
import org.apache.flink.configuration.AkkaOptions;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.instance.Instance;
......@@ -30,29 +25,37 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.messages.webmonitor.RequestJobDetails;
import org.apache.flink.runtime.metrics.dump.MetricDump;
import org.apache.flink.runtime.metrics.dump.MetricDumpSerialization;
import org.apache.flink.runtime.metrics.dump.MetricQueryService;
import org.apache.flink.runtime.metrics.dump.MetricDump;
import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
import org.apache.flink.util.Preconditions;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.dispatch.OnFailure;
import akka.dispatch.OnSuccess;
import akka.pattern.Patterns;
import akka.util.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;
import scala.Option;
import scala.concurrent.ExecutionContext;
import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;
import static org.apache.flink.runtime.metrics.dump.MetricDumpSerialization.MetricDumpDeserializer;
/**
* The MetricFetcher can be used to fetch metrics from the JobManager and all registered TaskManagers.
*
* Metrics will only be fetched when {@link MetricFetcher#update()} is called, provided that a sufficient time since
* <p>Metrics will only be fetched when {@link MetricFetcher#update()} is called, provided that a sufficient time since
* the last call has passed.
*/
public class MetricFetcher {
......@@ -135,7 +138,7 @@ public class MetricFetcher {
* We first request the list of all registered task managers from the job manager, and then
* request the respective metric dump from each task manager.
*
* All stored metrics that do not belong to a registered task manager will be removed.
* <p>All stored metrics that do not belong to a registered task manager will be removed.
*/
Future<Object> registeredTaskManagersFuture = jobManager.ask(JobManagerMessages.getRequestRegisteredTaskManagers(), timeout);
registeredTaskManagersFuture
......
......@@ -15,10 +15,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.metrics;
import org.apache.flink.runtime.metrics.dump.MetricDump;
import org.apache.flink.runtime.metrics.dump.QueryScopeInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -40,7 +42,7 @@ import static org.apache.flink.runtime.metrics.dump.QueryScopeInfo.INFO_CATEGORY
/**
* Nested data-structure to store metrics.
*
* This structure is not thread-safe.
* <p>This structure is not thread-safe.
*/
public class MetricStore {
private static final Logger LOG = LoggerFactory.getLogger(MetricStore.class);
......@@ -113,9 +115,9 @@ public class MetricStore {
}
/**
* The duplication is intended. Metrics scoped by subtask are useful for several job/task handlers,
* while the WebInterface task metric queries currently do not account for subtasks, so we don't
* while the WebInterface task metric queries currently do not account for subtasks, so we don't
* divide by subtask and instead use the concatenation of subtask index and metric name as the name
* for thos.
* for those.
*/
addMetric(subtask.metrics, name, metric);
addMetric(task.metrics, taskInfo.subtaskIndex + "." + name, metric);
......@@ -133,8 +135,8 @@ public class MetricStore {
job.tasks.put(operatorInfo.vertexID, task);
}
/**
* As the WebInterface does not account for operators (because it can't) we don't
* divide by operator and instead use the concatenation of subtask index, operator name and metric name
* As the WebInterface does not account for operators (because it can't) we don't
* divide by operator and instead use the concatenation of subtask index, operator name and metric name
* as the name.
*/
addMetric(task.metrics, operatorInfo.subtaskIndex + "." + operatorInfo.operatorName + "." + name, metric);
......@@ -245,7 +247,7 @@ public class MetricStore {
// -----------------------------------------------------------------------------------------------------------------
// sub MetricStore classes
// -----------------------------------------------------------------------------------------------------------------
private static abstract class ComponentMetricStore {
private abstract static class ComponentMetricStore {
public final Map<String, String> metrics = new HashMap<>();
public String getMetric(String name, String defaultValue) {
......@@ -267,7 +269,7 @@ public class MetricStore {
*/
public static class TaskManagerMetricStore extends ComponentMetricStore {
public final Set<String> garbageCollectorNames = new HashSet<>();
public void addGarbageCollectorName(String name) {
garbageCollectorNames.add(name);
}
......
......@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.metrics;
import org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler;
......@@ -24,10 +25,10 @@ import java.util.Map;
/**
* Request handler that returns for a given task manager a list of all available metrics or the values for a set of metrics.
*
* If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* <p>If the query parameters do not contain a "get" parameter the list of all metrics is returned.
* {@code {"available": [ { "name" : "X", "id" : "X" } ] } }
*
* If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* <p>If the query parameters do contain a "get" parameter a comma-separate list of metric names is expected as a value.
* {@code /get?X,Y}
* The handler will then return a list containing the values of the requested metrics.
* {@code [ { "id" : "X", "value" : "S" }, { "id" : "Y", "value" : "T" } ] }
......
......@@ -15,9 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.utils;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.runtime.executiongraph.AccessExecution;
import org.apache.flink.runtime.executiongraph.ExecutionGraph;
import org.apache.flink.runtime.executiongraph.IOMetrics;
......@@ -26,16 +26,19 @@ import org.apache.flink.runtime.webmonitor.handlers.JobVertexDetailsHandler;
import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher;
import org.apache.flink.runtime.webmonitor.metrics.MetricStore;
import com.fasterxml.jackson.core.JsonGenerator;
import javax.annotation.Nullable;
import java.io.IOException;
/**
* This class is a mutable version of the {@link IOMetrics} class that allows adding up IO-related metrics.
*
* For finished jobs these metrics are stored in the {@link ExecutionGraph} as another {@link IOMetrics}.
*
* <p>For finished jobs these metrics are stored in the {@link ExecutionGraph} as another {@link IOMetrics}.
* For running jobs these metrics are retrieved using the {@link MetricFetcher}.
*
* This class provides a common interface to handle both cases, reducing complexity in various handlers (like
*
* <p>This class provides a common interface to handle both cases, reducing complexity in various handlers (like
* the {@link JobVertexDetailsHandler}).
*/
public class MutableIOMetrics extends IOMetrics {
......@@ -50,7 +53,7 @@ public class MutableIOMetrics extends IOMetrics {
* Adds the IO metrics for the given attempt to this object. If the {@link AccessExecution} is in
* a terminal state the contained {@link IOMetrics} object is added. Otherwise the given {@link MetricFetcher} is
* used to retrieve the required metrics.
*
*
* @param attempt Attempt whose IO metrics should be added
* @param fetcher MetricFetcher to retrieve metrics for running jobs
* @param jobID JobID to which the attempt belongs
......@@ -83,21 +86,21 @@ public class MutableIOMetrics extends IOMetrics {
/**
* Writes the IO metrics contained in this object to the given {@link JsonGenerator}.
*
* The JSON structure written is as follows:
*
* <p>The JSON structure written is as follows:
* "metrics": {
* "read-bytes": 1,
* "write-bytes": 2,
* "read-records": 3,
* "write-records": 4
* }
*
*
* @param gen JsonGenerator to which the metrics should be written
* @throws IOException
*/
public void writeIOMetricsAsJson(JsonGenerator gen) throws IOException {
gen.writeObjectFieldStart("metrics");
gen.writeNumberField("read-bytes",this.numBytesInLocal + this.numBytesInRemote);
gen.writeNumberField("read-bytes", this.numBytesInLocal + this.numBytesInRemote);
gen.writeNumberField("write-bytes", this.numBytesOut);
gen.writeNumberField("read-records", this.numRecordsIn);
gen.writeNumberField("write-records", this.numRecordsOut);
......
......@@ -15,8 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.utils;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.net.SSLUtils;
import org.apache.flink.runtime.webmonitor.HttpRequestHandler;
import org.apache.flink.runtime.webmonitor.PipelineErrorHandler;
import org.apache.flink.util.Preconditions;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
......@@ -29,15 +36,11 @@ import io.netty.handler.codec.http.router.Handler;
import io.netty.handler.codec.http.router.Router;
import io.netty.handler.ssl.SslHandler;
import io.netty.handler.stream.ChunkedWriteHandler;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.net.SSLUtils;
import org.apache.flink.runtime.webmonitor.HttpRequestHandler;
import org.apache.flink.runtime.webmonitor.PipelineErrorHandler;
import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;
import java.io.File;
import java.net.InetSocketAddress;
......@@ -115,7 +118,7 @@ public class WebFrontendBootstrap {
public ServerBootstrap getBootstrap() {
return bootstrap;
}
public int getServerPort() {
Channel server = this.serverChannel;
if (server != null) {
......@@ -129,7 +132,7 @@ public class WebFrontendBootstrap {
return -1;
}
public void shutdown() {
if (this.serverChannel != null) {
this.serverChannel.close().awaitUninterruptibly();
......
......@@ -18,8 +18,6 @@
package org.apache.flink.runtime.webmonitor;
import akka.actor.ActorSystem;
import akka.testkit.JavaTestKit;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
......@@ -42,17 +40,21 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.util.TestLogger;
import akka.actor.ActorSystem;
import akka.testkit.JavaTestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.Option;
import scala.concurrent.duration.FiniteDuration;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import scala.Option;
import scala.concurrent.duration.FiniteDuration;
import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.AllVerticesRunning;
import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ExecutionGraphFound;
import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestExecutionGraph;
......
......@@ -29,6 +29,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
import org.apache.flink.runtime.executiongraph.ExecutionVertex;
import org.apache.flink.runtime.jobgraph.JobStatus;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.junit.Test;
import java.util.ArrayList;
......@@ -47,6 +48,9 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
/**
* Tests for the BackPressureStatsTracker.
*/
public class BackPressureStatsTrackerTest {
/** Tests simple statistics with fake stack traces. */
......@@ -148,7 +152,7 @@ public class BackPressureStatsTrackerTest {
assertEquals(sampleId, stats.getSampleId());
assertEquals(endTime, stats.getEndTimestamp());
assertEquals(taskVertices.length, stats.getNumberOfSubTasks());
for (int i = 0; i < taskVertices.length; i++) {
double ratio = stats.getBackPressureRatio(i);
// Traces until sub task index are back pressured
......
......@@ -18,8 +18,6 @@
package org.apache.flink.runtime.webmonitor;
import akka.actor.ActorSystem;
import akka.testkit.JavaTestKit;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
......@@ -40,15 +38,17 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
import org.apache.flink.util.TestLogger;
import akka.actor.ActorSystem;
import akka.testkit.JavaTestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.concurrent.TimeUnit;
import scala.concurrent.Await;
import scala.concurrent.duration.FiniteDuration;
import java.util.concurrent.TimeUnit;
import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.AllVerticesRunning;
import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ExecutionGraphFound;
import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestExecutionGraph;
......
......@@ -18,7 +18,6 @@
package org.apache.flink.runtime.webmonitor;
import akka.actor.ActorSystem;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.akka.AkkaUtils;
......@@ -32,6 +31,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionVertex;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.messages.StackTraceSampleMessages.TriggerStackTraceSample;
import org.apache.flink.runtime.messages.StackTraceSampleResponse;
import akka.actor.ActorSystem;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
......@@ -251,7 +252,6 @@ public class StackTraceSampleCoordinatorTest {
assertTrue(e.getCause().getCause().getMessage().contains("Timeout"));
}
// Collect after the timeout (should be ignored)
ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId();
coord.collectStackTraces(0, executionId, new ArrayList<StackTraceElement[]>());
......
......@@ -15,12 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.junit.Assert;
import org.junit.Test;
/**
* Tests for the WebMonitorUtils.
*/
public class WebMonitorUtilsTest {
@Test
......@@ -29,7 +34,7 @@ public class WebMonitorUtilsTest {
JsonArchivist[] reflected = WebMonitorUtils.getJsonArchivists();
Assert.assertEquals(direct.length, reflected.length);
for(int x = 0; x < direct.length; x++) {
for (int x = 0; x < direct.length; x++) {
Assert.assertSame(direct[x].getClass(), reflected[x].getClass());
}
}
......
......@@ -18,10 +18,6 @@
package org.apache.flink.runtime.webmonitor;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import io.netty.handler.codec.http.HttpResponseStatus;
import org.apache.curator.test.TestingServer;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.HighAvailabilityOptions;
......@@ -40,14 +36,15 @@ import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
import org.apache.flink.runtime.webmonitor.files.MimeTypes;
import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient;
import org.apache.flink.util.TestLogger;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import io.netty.handler.codec.http.HttpResponseStatus;
import org.apache.curator.test.TestingServer;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.powermock.reflect.Whitebox;
import scala.Some;
import scala.Tuple2;
import scala.concurrent.duration.Deadline;
import scala.concurrent.duration.FiniteDuration;
import java.io.File;
import java.nio.file.Files;
......@@ -56,19 +53,27 @@ import java.util.Scanner;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import scala.Some;
import scala.Tuple2;
import scala.concurrent.duration.Deadline;
import scala.concurrent.duration.FiniteDuration;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
/**
* Tests for the WebRuntimeMonitor.
*/
public class WebRuntimeMonitorITCase extends TestLogger {
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private final static FiniteDuration TestTimeout = new FiniteDuration(2, TimeUnit.MINUTES);
private static final FiniteDuration TestTimeout = new FiniteDuration(2, TimeUnit.MINUTES);
private final String MAIN_RESOURCES_PATH = getClass().getResource("/web").getPath();
private final String mainResourcesPath = getClass().getResource("/web").getPath();
/**
* Tests operation of the monitor in standalone operation.
......@@ -87,7 +92,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
webMonitor = startWebRuntimeMonitor(flink);
try (HttpTestClient client = new HttpTestClient("localhost", webMonitor.getServerPort())) {
String expected = new Scanner(new File(MAIN_RESOURCES_PATH + "/index.html"))
String expected = new Scanner(new File(mainResourcesPath + "/index.html"))
.useDelimiter("\\A").next();
// Request the file from the web server
......@@ -215,7 +220,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
HttpTestClient followingClient = new HttpTestClient(
"localhost", followerWebMonitor.getServerPort())) {
String expected = new Scanner(new File(MAIN_RESOURCES_PATH + "/index.html"))
String expected = new Scanner(new File(mainResourcesPath + "/index.html"))
.useDelimiter("\\A").next();
// Request the file from the leading web server
......@@ -349,7 +354,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
webMonitor = startWebRuntimeMonitor(flink);
try (HttpTestClient client = new HttpTestClient("localhost", webMonitor.getServerPort())) {
String expectedIndex = new Scanner(new File(MAIN_RESOURCES_PATH + "/index.html"))
String expectedIndex = new Scanner(new File(mainResourcesPath + "/index.html"))
.useDelimiter("\\A").next();
// 1) Request index.html from web server
......@@ -411,7 +416,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
webMonitor = startWebRuntimeMonitor(flink);
try (HttpTestClient client = new HttpTestClient("localhost", webMonitor.getServerPort())) {
String expectedIndex = new Scanner(new File(MAIN_RESOURCES_PATH + "/index.html"))
String expectedIndex = new Scanner(new File(mainResourcesPath + "/index.html"))
.useDelimiter("\\A").next();
// 1) Request index.html from web server
......
......@@ -20,13 +20,15 @@ package org.apache.flink.runtime.webmonitor.files;
import org.junit.Test;
import static org.junit.Assert.*;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.fail;
/**
* Tests for the MIME types map.
*/
public class MimeTypesTest {
@Test
public void testCompleteness() {
try {
......@@ -51,14 +53,14 @@ public class MimeTypesTest {
fail(e.getMessage());
}
}
@Test
public void testFileNameExtraction() {
try {
assertNotNull(MimeTypes.getMimeTypeForFileName("test.txt"));
assertNotNull(MimeTypes.getMimeTypeForFileName("t.txt"));
assertNotNull(MimeTypes.getMimeTypeForFileName("first.second.third.txt"));
assertNull(MimeTypes.getMimeTypeForFileName(".txt"));
assertNull(MimeTypes.getMimeTypeForFileName("txt"));
assertNull(MimeTypes.getMimeTypeForFileName("test."));
......
......@@ -15,14 +15,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.junit.Assert;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import java.util.concurrent.TimeUnit;
import scala.concurrent.duration.FiniteDuration;
/**
* Tests for the ClusterOverviewHandler.
*/
public class ClusterOverviewHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,14 +15,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.junit.Assert;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import java.util.concurrent.TimeUnit;
import scala.concurrent.duration.FiniteDuration;
/**
* Tests for the CurrentJobIdsHandler.
*/
public class CurrentJobIdsHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,11 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
......@@ -27,15 +25,23 @@ import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.junit.Assert;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Collection;
import java.util.concurrent.TimeUnit;
import scala.concurrent.duration.FiniteDuration;
/**
* Tests for the CurrentJobsOverviewHandler.
*/
public class CurrentJobsOverviewHandlerTest {
@Test
......@@ -50,7 +56,7 @@ public class CurrentJobsOverviewHandlerTest {
ArchivedJson archive = archives.iterator().next();
Assert.assertEquals("/joboverview", archive.getPath());
JsonNode result = ArchivedJobGenerationUtils.mapper.readTree(archive.getJson());
JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(archive.getJson());
ArrayNode running = (ArrayNode) result.get("running");
Assert.assertEquals(0, running.size());
......@@ -83,14 +89,14 @@ public class CurrentJobsOverviewHandlerTest {
AccessExecutionGraph originalJob = ArchivedJobGenerationUtils.getTestJob();
JobDetails expectedDetails = WebMonitorUtils.createDetailsForJob(originalJob);
StringWriter writer = new StringWriter();
try (JsonGenerator gen = ArchivedJobGenerationUtils.jacksonFactory.createGenerator(writer)) {
try (JsonGenerator gen = ArchivedJobGenerationUtils.JACKSON_FACTORY.createGenerator(writer)) {
CurrentJobsOverviewHandler.writeJobDetailOverviewAsJson(expectedDetails, gen, 0);
}
compareJobOverview(expectedDetails, writer.toString());
}
private static void compareJobOverview(JobDetails expectedDetails, String answer) throws IOException {
JsonNode result = ArchivedJobGenerationUtils.mapper.readTree(answer);
JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(answer);
Assert.assertEquals(expectedDetails.getJobId().toString(), result.get("jid").asText());
Assert.assertEquals(expectedDetails.getJobName(), result.get("name").asText());
......
......@@ -15,16 +15,21 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.databind.JsonNode;
import org.apache.flink.runtime.util.EnvironmentInformation;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import com.fasterxml.jackson.databind.JsonNode;
import org.junit.Assert;
import org.junit.Test;
import java.util.TimeZone;
/**
* Tests for the DashboardConfigHandler.
*/
public class DashboardConfigHandlerTest {
@Test
public void testGetPaths() {
......@@ -42,7 +47,7 @@ public class DashboardConfigHandlerTest {
String json = DashboardConfigHandler.createConfigJson(refreshInterval);
JsonNode result = ArchivedJobGenerationUtils.mapper.readTree(json);
JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
Assert.assertEquals(refreshInterval, result.get("refresh-interval").asLong());
Assert.assertEquals(timeZone.getDisplayName(), result.get("timezone-name").asText());
......
......@@ -20,15 +20,20 @@ package org.apache.flink.runtime.webmonitor.handlers;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.instance.DummyActorGateway;
import org.junit.Test;
import org.junit.Assert;
import org.junit.Test;
import scala.Tuple2;
/**
* Tests for the HandlerRedirectUtils.
*/
public class HandlerRedirectUtilsTest {
static final String localJobManagerAddress = "akka.tcp://flink@127.0.0.1:1234/user/foobar";
static final String remoteURL = "127.0.0.2:1235";
static final String remotePath = "akka.tcp://flink@" + remoteURL + "/user/jobmanager";
private static final String localJobManagerAddress = "akka.tcp://flink@127.0.0.1:1234/user/foobar";
private static final String remoteURL = "127.0.0.2:1235";
private static final String remotePath = "akka.tcp://flink@" + remoteURL + "/user/jobmanager";
@Test
public void testGetRedirectAddressWithLocalAkkaPath() throws Exception {
......@@ -36,7 +41,7 @@ public class HandlerRedirectUtilsTest {
Tuple2<ActorGateway, Integer> leader = new Tuple2<>(leaderGateway, 1235);
String redirectingAddress =HandlerRedirectUtils.getRedirectAddress(localJobManagerAddress, leader);
String redirectingAddress = HandlerRedirectUtils.getRedirectAddress(localJobManagerAddress, leader);
Assert.assertNull(redirectingAddress);
}
......@@ -47,7 +52,7 @@ public class HandlerRedirectUtilsTest {
Tuple2<ActorGateway, Integer> leader = new Tuple2<>(leaderGateway, 1235);
String redirectingAddress =HandlerRedirectUtils.getRedirectAddress(localJobManagerAddress, leader);
String redirectingAddress = HandlerRedirectUtils.getRedirectAddress(localJobManagerAddress, leader);
Assert.assertEquals(remoteURL, redirectingAddress);
}
......
......@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.google.common.collect.Lists;
......@@ -23,6 +24,9 @@ import org.junit.Test;
import java.util.List;
/**
* Tests for the JarAccessDeniedHandler.
*/
public class JarAccessDeniedHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -20,6 +20,7 @@ package org.apache.flink.runtime.webmonitor.handlers;
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
import org.apache.flink.runtime.webmonitor.handlers.JarActionHandler.JarActionHandlerConfig;
import org.junit.Test;
import java.util.HashMap;
......@@ -27,8 +28,10 @@ import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
/**
* Tests for the JarActionHandler.
*/
public class JarActionHandlerTest {
/**
......
......@@ -15,11 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.junit.Assert;
import org.junit.Test;
/**
* Tests for the JarDeleteHandler.
*/
public class JarDeleteHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,11 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.junit.Assert;
import org.junit.Test;
/**
* Tests for the JarListHandler.
*/
public class JarListHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,11 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.junit.Assert;
import org.junit.Test;
/**
* Tests for the JarPlanHandler.
*/
public class JarPlanHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,11 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.junit.Assert;
import org.junit.Test;
/**
* Tests for the JarRunHandler.
*/
public class JarRunHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,11 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.junit.Assert;
import org.junit.Test;
/**
* Tests for the JarUploadHandler.
*/
public class JarUploadHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,20 +15,25 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.Collection;
/**
* Tests for the JobAccumulatorsHandler.
*/
public class JobAccumulatorsHandlerTest {
@Test
......@@ -61,7 +66,7 @@ public class JobAccumulatorsHandlerTest {
}
private static void compareAccumulators(AccessExecutionGraph originalJob, String json) throws IOException {
JsonNode result = ArchivedJobGenerationUtils.mapper.readTree(json);
JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
ArrayNode accs = (ArrayNode) result.get("job-accumulators");
Assert.assertEquals(0, accs.size());
......
......@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.google.common.collect.Lists;
......@@ -23,6 +24,9 @@ import org.junit.Test;
import java.util.List;
/**
* Tests for the JobCancellationHandler.
*/
public class JobCancellationHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -18,13 +18,6 @@
package org.apache.flink.runtime.webmonitor.handlers;
import akka.dispatch.ExecutionContexts$;
import akka.dispatch.Futures;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.netty.handler.codec.http.FullHttpResponse;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpResponseStatus;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
......@@ -34,12 +27,16 @@ import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.messages.JobManagerMessages.CancelJobWithSavepoint;
import org.apache.flink.runtime.messages.JobManagerMessages.CancellationSuccess;
import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import akka.dispatch.ExecutionContexts$;
import akka.dispatch.Futures;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.netty.handler.codec.http.FullHttpResponse;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpResponseStatus;
import org.junit.Assert;
import org.junit.Test;
import scala.concurrent.ExecutionContext;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.concurrent.impl.Promise;
import java.nio.charset.Charset;
import java.util.Arrays;
......@@ -48,6 +45,11 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import scala.concurrent.ExecutionContext;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.concurrent.impl.Promise;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
......@@ -56,6 +58,9 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
/**
* Tests for the JobCancellationWithSavepointHandler.
*/
public class JobCancellationWithSavepointHandlersTest {
private static final ExecutionContext EC = ExecutionContexts$.MODULE$.fromExecutor(Executors.directExecutor());
......
......@@ -15,14 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.databind.JsonNode;
import org.apache.flink.api.common.ArchivedExecutionConfig;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import com.fasterxml.jackson.databind.JsonNode;
import org.junit.Assert;
import org.junit.Test;
......@@ -30,6 +32,9 @@ import java.io.IOException;
import java.util.Collection;
import java.util.Map;
/**
* Tests for the JobConfigHandler.
*/
public class JobConfigHandlerTest {
@Test
......@@ -60,7 +65,7 @@ public class JobConfigHandlerTest {
}
private static void compareJobConfig(AccessExecutionGraph originalJob, String answer) throws IOException {
JsonNode job = ArchivedJobGenerationUtils.mapper.readTree(answer);
JsonNode job = ArchivedJobGenerationUtils.MAPPER.readTree(answer);
Assert.assertEquals(originalJob.getJobID().toString(), job.get("jid").asText());
Assert.assertEquals(originalJob.getJobName(), job.get("name").asText());
......
......@@ -15,11 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.google.common.collect.Lists;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
......@@ -29,6 +27,10 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.google.common.collect.Lists;
import org.junit.Assert;
import org.junit.Test;
......@@ -37,6 +39,9 @@ import java.util.Collection;
import java.util.Iterator;
import java.util.List;
/**
* Tests for the JobDetailsHandler.
*/
public class JobDetailsHandlerTest {
@Test
......@@ -76,7 +81,7 @@ public class JobDetailsHandlerTest {
}
private static void compareJobDetails(AccessExecutionGraph originalJob, String json) throws IOException {
JsonNode result = ArchivedJobGenerationUtils.mapper.readTree(json);
JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
Assert.assertEquals(originalJob.getJobID().toString(), result.get("jid").asText());
Assert.assertEquals(originalJob.getJobName(), result.get("name").asText());
......@@ -154,6 +159,6 @@ public class JobDetailsHandlerTest {
Assert.assertEquals(0, statusCounts.get(ExecutionState.CANCELED.name()).asInt());
Assert.assertEquals(0, statusCounts.get(ExecutionState.FAILED.name()).asInt());
Assert.assertEquals(ArchivedJobGenerationUtils.mapper.readTree(originalJob.getJsonPlan()), result.get("plan"));
Assert.assertEquals(ArchivedJobGenerationUtils.MAPPER.readTree(originalJob.getJsonPlan()), result.get("plan"));
}
}
......@@ -15,10 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionVertex;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
......@@ -26,12 +25,18 @@ import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import org.apache.flink.util.ExceptionUtils;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.Collection;
/**
* Tests for the JobExceptionsHandler.
*/
public class JobExceptionsHandlerTest {
@Test
......@@ -64,7 +69,7 @@ public class JobExceptionsHandlerTest {
}
private static void compareExceptions(AccessExecutionGraph originalJob, String json) throws IOException {
JsonNode result = ArchivedJobGenerationUtils.mapper.readTree(json);
JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
Assert.assertEquals(originalJob.getFailureCauseAsString(), result.get("root-exception").asText());
......
......@@ -15,11 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.junit.Assert;
import org.junit.Test;
/**
* Tests for the JobManagerConfigHandler.
*/
public class JobManagerConfigHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,17 +15,22 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import org.junit.Assert;
import org.junit.Test;
import java.util.Collection;
/**
* Tests for the JobPlanHandler.
*/
public class JobPlanHandlerTest {
@Test
......
......@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.google.common.collect.Lists;
......@@ -23,6 +24,9 @@ import org.junit.Test;
import java.util.List;
/**
* Tests for the JobStoppingHandler.
*/
public class JobStoppingHandlerTest {
@Test
public void testGetPaths() {
......
......@@ -15,22 +15,27 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.Collection;
/**
* Tests for the JobVertexAccumulatorsHandler.
*/
public class JobVertexAccumulatorsHandlerTest {
@Test
......@@ -64,7 +69,7 @@ public class JobVertexAccumulatorsHandlerTest {
}
private static void compareAccumulators(AccessExecutionJobVertex originalTask, String json) throws IOException {
JsonNode result = ArchivedJobGenerationUtils.mapper.readTree(json);
JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
......
......@@ -18,18 +18,20 @@
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
import org.apache.flink.runtime.webmonitor.BackPressureStatsTracker;
import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder;
import org.apache.flink.runtime.webmonitor.OperatorBackPressureStats;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.junit.Assert;
import org.junit.Test;
import scala.Option;
import java.util.Collections;
import scala.Option;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.mockito.Matchers.any;
......@@ -50,7 +52,7 @@ public class JobVertexBackPressureHandlerTest {
Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/backpressure", paths[0]);
}
/** Tests the response when no stats are available */
/** Tests the response when no stats are available. */
@Test
public void testResponseNoStatsAvailable() throws Exception {
ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
......@@ -80,7 +82,7 @@ public class JobVertexBackPressureHandlerTest {
verify(statsTracker).triggerStackTraceSample(any(ExecutionJobVertex.class));
}
/** Tests the response when stats are available */
/** Tests the response when stats are available. */
@Test
public void testResponseStatsAvailable() throws Exception {
ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
......
......@@ -15,10 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.webmonitor.handlers;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex;
......@@ -27,12 +26,18 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.Collection;
/**
* Tests for the JobVertexDetailsHandler.
*/
public class JobVertexDetailsHandlerTest {
@Test
......@@ -67,7 +72,7 @@ public class JobVertexDetailsHandlerTest {
}
private static void compareVertexDetails(AccessExecutionJobVertex originalTask, String json) throws IOException {
JsonNode result = ArchivedJobGenerationUtils.mapper.readTree(json);
JsonNode result = ArchivedJobGenerationUtils.MAPPER.readTree(json);
Assert.assertEquals(originalTask.getJobVertexId().toString(), result.get("id").asText());
Assert.assertEquals(originalTask.getName(), result.get("name").asText());
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册