public WebRuntimeMonitor( Configuration config, LeaderRetrievalService leaderRetrievalService, ActorSystem actorSystem) throws IOException, InterruptedException { this.leaderRetrievalService = checkNotNull(leaderRetrievalService); this.timeout = AkkaUtils.getTimeout(config); this.retriever = new JobManagerRetriever(this, actorSystem, AkkaUtils.getTimeout(config), timeout); final WebMonitorConfig cfg = new WebMonitorConfig(config); final int configuredPort = cfg.getWebFrontendPort(); 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); LOG.info("Using directory {} for the web interface files", webRootDir); final boolean webSubmitAllow = cfg.isProgramSubmitEnabled(); if (webSubmitAllow) { // create storage for uploads String uploadDirName = "flink-web-upload-" + UUID.randomUUID(); this.uploadDir = new File(getBaseDir(config), uploadDirName); if (!uploadDir.mkdir() || !uploadDir.canWrite()) { throw new IOException("Unable to create temporary directory to support jar uploads."); } LOG.info("Using directory {} for web frontend JAR file uploads", uploadDir); } else { this.uploadDir = null; } ExecutionGraphHolder currentGraphs = new ExecutionGraphHolder(); // - Back pressure stats ---------------------------------------------- stackTraceSamples = new StackTraceSampleCoordinator(actorSystem, 60000); // Back pressure stats tracker config int cleanUpInterval = config.getInteger( ConfigConstants.JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_CLEAN_UP_INTERVAL); int refreshInterval = config.getInteger( ConfigConstants.JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL); int numSamples = config.getInteger( ConfigConstants.JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES); int delay = config.getInteger( ConfigConstants.JOB_MANAGER_WEB_BACK_PRESSURE_DELAY, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_BACK_PRESSURE_DELAY); FiniteDuration delayBetweenSamples = new FiniteDuration(delay, TimeUnit.MILLISECONDS); backPressureStatsTracker = new BackPressureStatsTracker( stackTraceSamples, cleanUpInterval, numSamples, delayBetweenSamples); // -------------------------------------------------------------------- executorService = new ForkJoinPool(); ExecutionContextExecutor context = ExecutionContext$.MODULE$.fromExecutor(executorService); router = new Router() // config how to interact with this web server .GET("/config", handler(new DashboardConfigHandler(cfg.getRefreshInterval()))) // the overview - how many task managers, slots, free slots, ... .GET("/overview", handler(new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT))) // job manager configuration .GET("/jobmanager/config", handler(new JobManagerConfigHandler(config))) // overview over jobs .GET( "/joboverview", handler(new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, true))) .GET( "/joboverview/running", handler(new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, false))) .GET( "/joboverview/completed", handler(new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, false, true))) .GET("/jobs", handler(new CurrentJobIdsHandler(DEFAULT_REQUEST_TIMEOUT))) .GET("/jobs/:jobid", handler(new JobDetailsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid/checkpoints", handler(new JobVertexCheckpointsHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid/backpressure", handler( new JobVertexBackPressureHandler( currentGraphs, backPressureStatsTracker, refreshInterval))) .GET( "/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs))) .GET( "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt/accumulators", handler(new SubtaskExecutionAttemptAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/plan", handler(new JobPlanHandler(currentGraphs))) .GET("/jobs/:jobid/config", handler(new JobConfigHandler(currentGraphs))) .GET("/jobs/:jobid/exceptions", handler(new JobExceptionsHandler(currentGraphs))) .GET("/jobs/:jobid/accumulators", handler(new JobAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints", handler(new JobCheckpointsHandler(currentGraphs))) .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT))) .GET( "/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT))) .GET( "/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log", new TaskManagerLogHandler( retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.LOG, config)) .GET( "/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout", new TaskManagerLogHandler( retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.STDOUT, config)) // log and stdout .GET( "/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") : new StaticFileServerHandler( retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile)) .GET( "/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") : new StaticFileServerHandler( retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile)) // Cancel a job via GET (for proper integration with YARN this has to be performed via // GET) .GET("/jobs/:jobid/yarn-cancel", handler(new JobCancellationHandler())) // DELETE is the preferred way of canceling a job (Rest-conform) .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler())) // stop a job via GET (for proper integration with YARN this has to be performed via // GET) .GET("/jobs/:jobid/yarn-stop", handler(new JobStoppingHandler())) // DELETE is the preferred way of stopping a job (Rest-conform) .DELETE("/jobs/:jobid/stop", handler(new JobStoppingHandler())); if (webSubmitAllow) { router // fetch the list of uploaded jars. .GET("/jars", handler(new JarListHandler(uploadDir))) // get plan for an uploaded jar .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) // run a jar .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout))) // upload a jar .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) // delete an uploaded jar from submission interface .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); } else { router // send an Access Denied message (sort of) // Every other GET request will go to the File Server, which will not provide // access to the jar directory anyway, because it doesn't exist in webRootDir. .GET("/jars", handler(new JarAccessDeniedHandler())); } // this handler serves all the static contents router.GET( "/:*", new StaticFileServerHandler( retriever, jobManagerAddressPromise.future(), timeout, webRootDir)); // add shutdown hook for deleting the directories and remaining temp files on shutdown try { Runtime.getRuntime() .addShutdownHook( new Thread() { @Override public void run() { cleanup(); } }); } catch (IllegalStateException e) { // race, JVM is in shutdown already, we can safely ignore this LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); } catch (Throwable t) { // these errors usually happen when the shutdown is already in progress LOG.warn("Error while adding shutdown hook", t); } ChannelInitializer<SocketChannel> initializer = new ChannelInitializer<SocketChannel>() { @Override protected void initChannel(SocketChannel ch) { Handler handler = new Handler(router); ch.pipeline() .addLast(new HttpServerCodec()) .addLast(new HttpRequestHandler(uploadDir)) .addLast(handler.name(), handler) .addLast(new PipelineErrorHandler(LOG)); } }; NioEventLoopGroup bossGroup = new NioEventLoopGroup(1); NioEventLoopGroup workerGroup = new NioEventLoopGroup(); this.bootstrap = new ServerBootstrap(); this.bootstrap .group(bossGroup, workerGroup) .channel(NioServerSocketChannel.class) .childHandler(initializer); Channel ch = this.bootstrap.bind(configuredPort).sync().channel(); this.serverChannel = ch; InetSocketAddress bindAddress = (InetSocketAddress) ch.localAddress(); String address = bindAddress.getAddress().getHostAddress(); int port = bindAddress.getPort(); LOG.info("Web frontend listening at " + address + ':' + port); }