Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-5870

Make handlers aware of their REST URLs

    Details

    • Type: Improvement
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.3.0
    • Component/s: Webfrontend
    • Labels:
      None

      Description

      The handlers in the WebRuntimeMonitor are currently unaware of the actual REST URL used. The handlers are simply registered under a given URL, without any guarantee that the handler can actually deal with that URL.

      I propose to let handlers themselves specify under which URL's they are supposed to be reachable. This provides are tighter coupling between URL and handler.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user zentol opened a pull request:

          https://github.com/apache/flink/pull/3376

          FLINK-5870 Handlers define REST URLs

          This PR allows ```RequestHandler```s to define the REST URLs under which they should be registered.

          For this purpose the following method was added to the ```RequestHandler``` interface: ```String[] getPaths();```

          Additionally, a utility class ```RestUtils``` was added that contains a number of often used REST URL components as constants (things like "job" or ":jobid"), and a utility method to concatenate these components into a REST URL (basically concat with "/"). The idea here is to prevent typos and such.

          Tests were added for every single handler to verify that the correct paths are returned. As a result if any URL should be changed, which isn't allowed since the REST API is considered stable, a test will now fail.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/zentol/flink 5870_handler_url

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3376.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3376


          commit 6f0987f46afc267023b9834a3d7abd4e5762f2e1
          Author: zentol <chesnay@apache.org>
          Date: 2017-02-20T15:29:27Z

          FLINK-5870 Handlers define REST URLs


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user zentol opened a pull request: https://github.com/apache/flink/pull/3376 FLINK-5870 Handlers define REST URLs This PR allows ```RequestHandler```s to define the REST URLs under which they should be registered. For this purpose the following method was added to the ```RequestHandler``` interface: ```String[] getPaths();``` Additionally, a utility class ```RestUtils``` was added that contains a number of often used REST URL components as constants (things like "job" or ":jobid"), and a utility method to concatenate these components into a REST URL (basically concat with "/"). The idea here is to prevent typos and such. Tests were added for every single handler to verify that the correct paths are returned. As a result if any URL should be changed, which isn't allowed since the REST API is considered stable, a test will now fail. You can merge this pull request into a Git repository by running: $ git pull https://github.com/zentol/flink 5870_handler_url Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3376.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3376 commit 6f0987f46afc267023b9834a3d7abd4e5762f2e1 Author: zentol <chesnay@apache.org> Date: 2017-02-20T15:29:27Z FLINK-5870 Handlers define REST URLs
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103456488

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/RestUtils.java —
          @@ -0,0 +1,64 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.runtime.webmonitor.utils;
          +
          +public class RestUtils {
          — End diff –

          I see why you added this, but I'm again skeptical :smile: The problem I have is that it really decreases readability in my opinion in the handlers. I would actually vote to remove it and keep the complete paths in the handlers.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103456488 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/RestUtils.java — @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.runtime.webmonitor.utils; + +public class RestUtils { — End diff – I see why you added this, but I'm again skeptical :smile: The problem I have is that it really decreases readability in my opinion in the handlers. I would actually vote to remove it and keep the complete paths in the handlers.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103453211

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java —
          @@ -435,6 +424,36 @@ protected void initChannel(SocketChannel ch)

          { LOG.info("Web frontend listening at " + address + ':' + port); }

          + private void GET(Router router, RequestHandler handler) {
          — End diff –

          Can we add a high level comment that says that we have these helpers in order to do the route path setup?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103453211 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java — @@ -435,6 +424,36 @@ protected void initChannel(SocketChannel ch) { LOG.info("Web frontend listening at " + address + ':' + port); } + private void GET(Router router, RequestHandler handler) { — End diff – Can we add a high level comment that says that we have these helpers in order to do the route path setup?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103455030

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java —
          @@ -141,6 +144,15 @@ public TaskManagerLogHandler(
          timeTimeout = Time.milliseconds(timeout.toMillis());
          }

          + @Override
          + public String[] getPaths() {
          + if (serveLogFile) {
          — End diff –

          I think the proper way to do this would be to have an abstract `TaskManagerLogHandler` and inherit it once for the logs and once for stdout.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103455030 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java — @@ -141,6 +144,15 @@ public TaskManagerLogHandler( timeTimeout = Time.milliseconds(timeout.toMillis()); } + @Override + public String[] getPaths() { + if (serveLogFile) { — End diff – I think the proper way to do this would be to have an abstract `TaskManagerLogHandler` and inherit it once for the logs and once for stdout.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103455441

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java —
          @@ -53,6 +59,11 @@ public TaskManagersHandler(FiniteDuration timeout, MetricFetcher fetcher) {
          }

          @Override
          + public String[] getPaths() {
          + return new String[]

          {TASKMANAGERS_REST_PATH, TASKMANAGER_DETAILS_REST_PATH}

          ;
          — End diff –

          Just to double check: The handler is OK to handle both requests, right? Before we created two instances of the handler. I think this is actually a case where this works better than before.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103455441 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java — @@ -53,6 +59,11 @@ public TaskManagersHandler(FiniteDuration timeout, MetricFetcher fetcher) { } @Override + public String[] getPaths() { + return new String[] {TASKMANAGERS_REST_PATH, TASKMANAGER_DETAILS_REST_PATH} ; — End diff – Just to double check: The handler is OK to handle both requests, right? Before we created two instances of the handler. I think this is actually a case where this works better than before.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103455491

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java —
          @@ -52,6 +63,12 @@ public CheckpointStatsDetailsSubtasksHandler(ExecutionGraphHolder executionGraph
          }

          @Override
          + public String[] getPaths() {
          + return new String[]

          {CHECKPOINT_STATS_DETAILS_SUBTASKS_REST_PATH}

          ;
          + }
          +
          — End diff –

          empty line

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103455491 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java — @@ -52,6 +63,12 @@ public CheckpointStatsDetailsSubtasksHandler(ExecutionGraphHolder executionGraph } @Override + public String[] getPaths() { + return new String[] {CHECKPOINT_STATS_DETAILS_SUBTASKS_REST_PATH} ; + } + — End diff – empty line
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103452837

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java —
          @@ -256,117 +256,106 @@ public WebRuntimeMonitor(
          RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler());
          RuntimeMonitorHandler inProgressHandler = handler(cancelWithSavepoint.getInProgressHandler());

          • 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, metricFetcher)))
          • .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs, metricFetcher)))
            -
          • .GET("/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/backpressure", handler(new JobVertexBackPressureHandler(
          • currentGraphs,
          • backPressureStatsTracker,
          • refreshInterval)))
          • .GET("/jobs/:jobid/vertices/:vertexid/metrics", handler(new JobVertexMetricsHandler(metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher)))
          • .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/metrics", handler(new JobMetricsHandler(metricFetcher)))
            -
          • .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log",
          • new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
          • TaskManagerLogHandler.FileMode.LOG, config, enableSSL))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout",
          • new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
          • TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagerMetricsHandler(metricFetcher)))
            + router = new Router();
            + // config how to interact with this web server
            + GET(router, new DashboardConfigHandler(cfg.getRefreshInterval()));
            +
            + // the overview - how many task managers, slots, free slots, ...
            + GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT));
            +
            + // job manager configuration
            + 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, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
            + TaskManagerLogHandler.FileMode.LOG, config, enableSSL));
            + GET(router, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
            + TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL));
            + GET(router, new TaskManagerMetricsHandler(metricFetcher));

          + router
          // log and stdout
          .GET("/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") :
          new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile,
          enableSSL))

          .GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") :
          new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile,

          • enableSSL))
            -
          • .GET("/jobmanager/metrics", handler(new JobManagerMetricsHandler(metricFetcher)))
            + enableSSL));
          • // 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()))
            + GET(router, new JobManagerMetricsHandler(metricFetcher));
          • // DELETE is the preferred way of canceling a job (Rest-conform)
          • .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler()))
            + // Cancel a job via GET (for proper integration with YARN this has to be performed via GET)
            + GET(router, new JobCancellationHandler());
            + // DELETE is the preferred way of canceling a job (Rest-conform)
            + DELETE(router, new JobCancellationHandler());
          • .GET("/jobs/:jobid/cancel-with-savepoint", triggerHandler)
          • .GET("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory", triggerHandler)
          • .GET(JobCancellationWithSavepointHandlers.IN_PROGRESS_URL, 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("/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()));
            + // stop a job via GET (for proper integration with YARN this has to be performed via GET)
            + GET(router, new JobStoppingHandler());
            + // DELETE is the preferred way of stopping a job (Rest-conform)
            + DELETE(router, new JobStoppingHandler());

          int maxCachedEntries = config.getInteger(
          ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
          ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
          CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries);

          // Register the checkpoint stats handlers

          • router
          • .GET("/jobs/:jobid/checkpoints", handler(new CheckpointStatsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/checkpoints/config", handler(new CheckpointConfigHandler(currentGraphs)))
          • .GET("/jobs/:jobid/checkpoints/details/:checkpointid", handler(new CheckpointStatsDetailsHandler(currentGraphs, cache)))
          • .GET("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", handler(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)

          { - router - // fetch the list of uploaded jars. - .GET("/jars", handler(new JarListHandler(uploadDir))) + // fetch the list of uploaded jars. + GET(router, new JarListHandler(uploadDir)); - // get plan for an uploaded jar - .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) + // get plan for an uploaded jar + GET(router, new JarPlanHandler(uploadDir)); - // run a jar - .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout, config))) + // run a jar + POST(router, new JarRunHandler(uploadDir, timeout, config)); - // upload a jar - .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) + // upload a jar + POST(router, new JarUploadHandler(uploadDir)); - // delete an uploaded jar from submission interface - .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); + // delete an uploaded jar from submission interface + DELETE(router, 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()));
            + // send an Access Denied message
            + JarAccessDeniedHandler jad = new JarAccessDeniedHandler();
            + GET(router, jad);
            + POST(router, jad);
              • End diff –

          Why do we add the POST and DELETE here?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103452837 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java — @@ -256,117 +256,106 @@ public WebRuntimeMonitor( RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler()); RuntimeMonitorHandler inProgressHandler = handler(cancelWithSavepoint.getInProgressHandler()); 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, metricFetcher))) .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs, metricFetcher))) - .GET("/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/backpressure", handler(new JobVertexBackPressureHandler( currentGraphs, backPressureStatsTracker, refreshInterval))) .GET("/jobs/:jobid/vertices/:vertexid/metrics", handler(new JobVertexMetricsHandler(metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher))) .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/metrics", handler(new JobMetricsHandler(metricFetcher))) - .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher))) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher))) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log", new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.LOG, config, enableSSL)) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout", new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL)) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagerMetricsHandler(metricFetcher))) + router = new Router(); + // config how to interact with this web server + GET(router, new DashboardConfigHandler(cfg.getRefreshInterval())); + + // the overview - how many task managers, slots, free slots, ... + GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT)); + + // job manager configuration + 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, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, + TaskManagerLogHandler.FileMode.LOG, config, enableSSL)); + GET(router, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, + TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL)); + GET(router, new TaskManagerMetricsHandler(metricFetcher)); + router // log and stdout .GET("/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") : new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile, enableSSL)) .GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") : new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile, enableSSL)) - .GET("/jobmanager/metrics", handler(new JobManagerMetricsHandler(metricFetcher))) + enableSSL)); // 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())) + GET(router, new JobManagerMetricsHandler(metricFetcher)); // DELETE is the preferred way of canceling a job (Rest-conform) .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler())) + // Cancel a job via GET (for proper integration with YARN this has to be performed via GET) + GET(router, new JobCancellationHandler()); + // DELETE is the preferred way of canceling a job (Rest-conform) + DELETE(router, new JobCancellationHandler()); .GET("/jobs/:jobid/cancel-with-savepoint", triggerHandler) .GET("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory", triggerHandler) .GET(JobCancellationWithSavepointHandlers.IN_PROGRESS_URL, 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("/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())); + // stop a job via GET (for proper integration with YARN this has to be performed via GET) + GET(router, new JobStoppingHandler()); + // DELETE is the preferred way of stopping a job (Rest-conform) + DELETE(router, new JobStoppingHandler()); int maxCachedEntries = config.getInteger( ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE); CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries); // Register the checkpoint stats handlers router .GET("/jobs/:jobid/checkpoints", handler(new CheckpointStatsHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints/config", handler(new CheckpointConfigHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints/details/:checkpointid", handler(new CheckpointStatsDetailsHandler(currentGraphs, cache))) .GET("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", handler(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) { - router - // fetch the list of uploaded jars. - .GET("/jars", handler(new JarListHandler(uploadDir))) + // fetch the list of uploaded jars. + GET(router, new JarListHandler(uploadDir)); - // get plan for an uploaded jar - .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) + // get plan for an uploaded jar + GET(router, new JarPlanHandler(uploadDir)); - // run a jar - .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout, config))) + // run a jar + POST(router, new JarRunHandler(uploadDir, timeout, config)); - // upload a jar - .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) + // upload a jar + POST(router, new JarUploadHandler(uploadDir)); - // delete an uploaded jar from submission interface - .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); + // delete an uploaded jar from submission interface + DELETE(router, 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())); + // send an Access Denied message + JarAccessDeniedHandler jad = new JarAccessDeniedHandler(); + GET(router, jad); + POST(router, jad); End diff – Why do we add the POST and DELETE here?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103452124

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandlerBase.java —
          @@ -67,6 +67,8 @@ public RuntimeMonitorHandlerBase(
          this.httpsEnabled = httpsEnabled;
          }

          + public abstract String[] getPaths();
          — End diff –

          I know the other methods are also not commented well, but could you add comments here?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103452124 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandlerBase.java — @@ -67,6 +67,8 @@ public RuntimeMonitorHandlerBase( this.httpsEnabled = httpsEnabled; } + public abstract String[] getPaths(); — End diff – I know the other methods are also not commented well, but could you add comments here?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103458807

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java —
          @@ -256,117 +256,106 @@ public WebRuntimeMonitor(
          RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler());
          RuntimeMonitorHandler inProgressHandler = handler(cancelWithSavepoint.getInProgressHandler());

          • 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, metricFetcher)))
          • .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs, metricFetcher)))
            -
          • .GET("/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/backpressure", handler(new JobVertexBackPressureHandler(
          • currentGraphs,
          • backPressureStatsTracker,
          • refreshInterval)))
          • .GET("/jobs/:jobid/vertices/:vertexid/metrics", handler(new JobVertexMetricsHandler(metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher)))
          • .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/metrics", handler(new JobMetricsHandler(metricFetcher)))
            -
          • .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log",
          • new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
          • TaskManagerLogHandler.FileMode.LOG, config, enableSSL))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout",
          • new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
          • TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagerMetricsHandler(metricFetcher)))
            + router = new Router();
            + // config how to interact with this web server
            + GET(router, new DashboardConfigHandler(cfg.getRefreshInterval()));
            +
            + // the overview - how many task managers, slots, free slots, ...
            + GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT));
            +
            + // job manager configuration
            + 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, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
            + TaskManagerLogHandler.FileMode.LOG, config, enableSSL));
            + GET(router, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
            + TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL));
            + GET(router, new TaskManagerMetricsHandler(metricFetcher));

          + router
          // log and stdout
          .GET("/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") :
          new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile,
          enableSSL))

          .GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") :
          new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile,

          • enableSSL))
            -
          • .GET("/jobmanager/metrics", handler(new JobManagerMetricsHandler(metricFetcher)))
            + enableSSL));
          • // 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()))
            + GET(router, new JobManagerMetricsHandler(metricFetcher));
          • // DELETE is the preferred way of canceling a job (Rest-conform)
          • .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler()))
            + // Cancel a job via GET (for proper integration with YARN this has to be performed via GET)
            + GET(router, new JobCancellationHandler());
            + // DELETE is the preferred way of canceling a job (Rest-conform)
            + DELETE(router, new JobCancellationHandler());
          • .GET("/jobs/:jobid/cancel-with-savepoint", triggerHandler)
          • .GET("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory", triggerHandler)
          • .GET(JobCancellationWithSavepointHandlers.IN_PROGRESS_URL, 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("/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()));
            + // stop a job via GET (for proper integration with YARN this has to be performed via GET)
            + GET(router, new JobStoppingHandler());
            + // DELETE is the preferred way of stopping a job (Rest-conform)
            + DELETE(router, new JobStoppingHandler());

          int maxCachedEntries = config.getInteger(
          ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
          ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
          CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries);

          // Register the checkpoint stats handlers

          • router
          • .GET("/jobs/:jobid/checkpoints", handler(new CheckpointStatsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/checkpoints/config", handler(new CheckpointConfigHandler(currentGraphs)))
          • .GET("/jobs/:jobid/checkpoints/details/:checkpointid", handler(new CheckpointStatsDetailsHandler(currentGraphs, cache)))
          • .GET("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", handler(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)

          { - router - // fetch the list of uploaded jars. - .GET("/jars", handler(new JarListHandler(uploadDir))) + // fetch the list of uploaded jars. + GET(router, new JarListHandler(uploadDir)); - // get plan for an uploaded jar - .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) + // get plan for an uploaded jar + GET(router, new JarPlanHandler(uploadDir)); - // run a jar - .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout, config))) + // run a jar + POST(router, new JarRunHandler(uploadDir, timeout, config)); - // upload a jar - .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) + // upload a jar + POST(router, new JarUploadHandler(uploadDir)); - // delete an uploaded jar from submission interface - .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); + // delete an uploaded jar from submission interface + DELETE(router, 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()));
            + // send an Access Denied message
            + JarAccessDeniedHandler jad = new JarAccessDeniedHandler();
            + GET(router, jad);
            + POST(router, jad);
              • End diff –

          Because there are Jar-related handlers that accept POST and DELETE requests. Having the JarAccessDeniedHandler reject these as well seemed more consistent.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103458807 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java — @@ -256,117 +256,106 @@ public WebRuntimeMonitor( RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler()); RuntimeMonitorHandler inProgressHandler = handler(cancelWithSavepoint.getInProgressHandler()); 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, metricFetcher))) .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs, metricFetcher))) - .GET("/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/backpressure", handler(new JobVertexBackPressureHandler( currentGraphs, backPressureStatsTracker, refreshInterval))) .GET("/jobs/:jobid/vertices/:vertexid/metrics", handler(new JobVertexMetricsHandler(metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher))) .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/metrics", handler(new JobMetricsHandler(metricFetcher))) - .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher))) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher))) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log", new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.LOG, config, enableSSL)) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout", new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL)) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagerMetricsHandler(metricFetcher))) + router = new Router(); + // config how to interact with this web server + GET(router, new DashboardConfigHandler(cfg.getRefreshInterval())); + + // the overview - how many task managers, slots, free slots, ... + GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT)); + + // job manager configuration + 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, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, + TaskManagerLogHandler.FileMode.LOG, config, enableSSL)); + GET(router, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, + TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL)); + GET(router, new TaskManagerMetricsHandler(metricFetcher)); + router // log and stdout .GET("/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") : new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile, enableSSL)) .GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") : new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile, enableSSL)) - .GET("/jobmanager/metrics", handler(new JobManagerMetricsHandler(metricFetcher))) + enableSSL)); // 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())) + GET(router, new JobManagerMetricsHandler(metricFetcher)); // DELETE is the preferred way of canceling a job (Rest-conform) .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler())) + // Cancel a job via GET (for proper integration with YARN this has to be performed via GET) + GET(router, new JobCancellationHandler()); + // DELETE is the preferred way of canceling a job (Rest-conform) + DELETE(router, new JobCancellationHandler()); .GET("/jobs/:jobid/cancel-with-savepoint", triggerHandler) .GET("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory", triggerHandler) .GET(JobCancellationWithSavepointHandlers.IN_PROGRESS_URL, 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("/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())); + // stop a job via GET (for proper integration with YARN this has to be performed via GET) + GET(router, new JobStoppingHandler()); + // DELETE is the preferred way of stopping a job (Rest-conform) + DELETE(router, new JobStoppingHandler()); int maxCachedEntries = config.getInteger( ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE); CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries); // Register the checkpoint stats handlers router .GET("/jobs/:jobid/checkpoints", handler(new CheckpointStatsHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints/config", handler(new CheckpointConfigHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints/details/:checkpointid", handler(new CheckpointStatsDetailsHandler(currentGraphs, cache))) .GET("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", handler(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) { - router - // fetch the list of uploaded jars. - .GET("/jars", handler(new JarListHandler(uploadDir))) + // fetch the list of uploaded jars. + GET(router, new JarListHandler(uploadDir)); - // get plan for an uploaded jar - .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) + // get plan for an uploaded jar + GET(router, new JarPlanHandler(uploadDir)); - // run a jar - .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout, config))) + // run a jar + POST(router, new JarRunHandler(uploadDir, timeout, config)); - // upload a jar - .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) + // upload a jar + POST(router, new JarUploadHandler(uploadDir)); - // delete an uploaded jar from submission interface - .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); + // delete an uploaded jar from submission interface + DELETE(router, 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())); + // send an Access Denied message + JarAccessDeniedHandler jad = new JarAccessDeniedHandler(); + GET(router, jad); + POST(router, jad); End diff – Because there are Jar-related handlers that accept POST and DELETE requests. Having the JarAccessDeniedHandler reject these as well seemed more consistent.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103458922

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java —
          @@ -141,6 +144,15 @@ public TaskManagerLogHandler(
          timeTimeout = Time.milliseconds(timeout.toMillis());
          }

          + @Override
          + public String[] getPaths() {
          + if (serveLogFile) {
          — End diff –

          I think adding 2 classes for a single flag is overkill.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103458922 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java — @@ -141,6 +144,15 @@ public TaskManagerLogHandler( timeTimeout = Time.milliseconds(timeout.toMillis()); } + @Override + public String[] getPaths() { + if (serveLogFile) { — End diff – I think adding 2 classes for a single flag is overkill.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103459049

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java —
          @@ -53,6 +59,11 @@ public TaskManagersHandler(FiniteDuration timeout, MetricFetcher fetcher) {
          }

          @Override
          + public String[] getPaths() {
          + return new String[]

          {TASKMANAGERS_REST_PATH, TASKMANAGER_DETAILS_REST_PATH}

          ;
          — End diff –

          yes, the same handler can handle both requests. The handlers that we created before were identical.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103459049 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java — @@ -53,6 +59,11 @@ public TaskManagersHandler(FiniteDuration timeout, MetricFetcher fetcher) { } @Override + public String[] getPaths() { + return new String[] {TASKMANAGERS_REST_PATH, TASKMANAGER_DETAILS_REST_PATH} ; — End diff – yes, the same handler can handle both requests. The handlers that we created before were identical.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103471161

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java —
          @@ -256,117 +256,106 @@ public WebRuntimeMonitor(
          RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler());
          RuntimeMonitorHandler inProgressHandler = handler(cancelWithSavepoint.getInProgressHandler());

          • 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, metricFetcher)))
          • .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs, metricFetcher)))
            -
          • .GET("/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/backpressure", handler(new JobVertexBackPressureHandler(
          • currentGraphs,
          • backPressureStatsTracker,
          • refreshInterval)))
          • .GET("/jobs/:jobid/vertices/:vertexid/metrics", handler(new JobVertexMetricsHandler(metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher)))
          • .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/metrics", handler(new JobMetricsHandler(metricFetcher)))
            -
          • .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log",
          • new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
          • TaskManagerLogHandler.FileMode.LOG, config, enableSSL))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout",
          • new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
          • TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagerMetricsHandler(metricFetcher)))
            + router = new Router();
            + // config how to interact with this web server
            + GET(router, new DashboardConfigHandler(cfg.getRefreshInterval()));
            +
            + // the overview - how many task managers, slots, free slots, ...
            + GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT));
            +
            + // job manager configuration
            + 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, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
            + TaskManagerLogHandler.FileMode.LOG, config, enableSSL));
            + GET(router, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
            + TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL));
            + GET(router, new TaskManagerMetricsHandler(metricFetcher));

          + router
          // log and stdout
          .GET("/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") :
          new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile,
          enableSSL))

          .GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") :
          new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile,

          • enableSSL))
            -
          • .GET("/jobmanager/metrics", handler(new JobManagerMetricsHandler(metricFetcher)))
            + enableSSL));
          • // 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()))
            + GET(router, new JobManagerMetricsHandler(metricFetcher));
          • // DELETE is the preferred way of canceling a job (Rest-conform)
          • .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler()))
            + // Cancel a job via GET (for proper integration with YARN this has to be performed via GET)
            + GET(router, new JobCancellationHandler());
            + // DELETE is the preferred way of canceling a job (Rest-conform)
            + DELETE(router, new JobCancellationHandler());
          • .GET("/jobs/:jobid/cancel-with-savepoint", triggerHandler)
          • .GET("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory", triggerHandler)
          • .GET(JobCancellationWithSavepointHandlers.IN_PROGRESS_URL, 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("/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()));
            + // stop a job via GET (for proper integration with YARN this has to be performed via GET)
            + GET(router, new JobStoppingHandler());
            + // DELETE is the preferred way of stopping a job (Rest-conform)
            + DELETE(router, new JobStoppingHandler());

          int maxCachedEntries = config.getInteger(
          ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
          ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
          CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries);

          // Register the checkpoint stats handlers

          • router
          • .GET("/jobs/:jobid/checkpoints", handler(new CheckpointStatsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/checkpoints/config", handler(new CheckpointConfigHandler(currentGraphs)))
          • .GET("/jobs/:jobid/checkpoints/details/:checkpointid", handler(new CheckpointStatsDetailsHandler(currentGraphs, cache)))
          • .GET("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", handler(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)

          { - router - // fetch the list of uploaded jars. - .GET("/jars", handler(new JarListHandler(uploadDir))) + // fetch the list of uploaded jars. + GET(router, new JarListHandler(uploadDir)); - // get plan for an uploaded jar - .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) + // get plan for an uploaded jar + GET(router, new JarPlanHandler(uploadDir)); - // run a jar - .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout, config))) + // run a jar + POST(router, new JarRunHandler(uploadDir, timeout, config)); - // upload a jar - .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) + // upload a jar + POST(router, new JarUploadHandler(uploadDir)); - // delete an uploaded jar from submission interface - .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); + // delete an uploaded jar from submission interface + DELETE(router, 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()));
            + // send an Access Denied message
            + JarAccessDeniedHandler jad = new JarAccessDeniedHandler();
            + GET(router, jad);
            + POST(router, jad);
              • End diff –

          OK I thought that they are automatically rejected because we did not register a route for them (only for GET before)

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103471161 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java — @@ -256,117 +256,106 @@ public WebRuntimeMonitor( RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler()); RuntimeMonitorHandler inProgressHandler = handler(cancelWithSavepoint.getInProgressHandler()); 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, metricFetcher))) .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs, metricFetcher))) - .GET("/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/backpressure", handler(new JobVertexBackPressureHandler( currentGraphs, backPressureStatsTracker, refreshInterval))) .GET("/jobs/:jobid/vertices/:vertexid/metrics", handler(new JobVertexMetricsHandler(metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher))) .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/metrics", handler(new JobMetricsHandler(metricFetcher))) - .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher))) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher))) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log", new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.LOG, config, enableSSL)) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout", new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL)) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagerMetricsHandler(metricFetcher))) + router = new Router(); + // config how to interact with this web server + GET(router, new DashboardConfigHandler(cfg.getRefreshInterval())); + + // the overview - how many task managers, slots, free slots, ... + GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT)); + + // job manager configuration + 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, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, + TaskManagerLogHandler.FileMode.LOG, config, enableSSL)); + GET(router, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, + TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL)); + GET(router, new TaskManagerMetricsHandler(metricFetcher)); + router // log and stdout .GET("/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") : new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile, enableSSL)) .GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") : new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile, enableSSL)) - .GET("/jobmanager/metrics", handler(new JobManagerMetricsHandler(metricFetcher))) + enableSSL)); // 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())) + GET(router, new JobManagerMetricsHandler(metricFetcher)); // DELETE is the preferred way of canceling a job (Rest-conform) .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler())) + // Cancel a job via GET (for proper integration with YARN this has to be performed via GET) + GET(router, new JobCancellationHandler()); + // DELETE is the preferred way of canceling a job (Rest-conform) + DELETE(router, new JobCancellationHandler()); .GET("/jobs/:jobid/cancel-with-savepoint", triggerHandler) .GET("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory", triggerHandler) .GET(JobCancellationWithSavepointHandlers.IN_PROGRESS_URL, 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("/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())); + // stop a job via GET (for proper integration with YARN this has to be performed via GET) + GET(router, new JobStoppingHandler()); + // DELETE is the preferred way of stopping a job (Rest-conform) + DELETE(router, new JobStoppingHandler()); int maxCachedEntries = config.getInteger( ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE); CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries); // Register the checkpoint stats handlers router .GET("/jobs/:jobid/checkpoints", handler(new CheckpointStatsHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints/config", handler(new CheckpointConfigHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints/details/:checkpointid", handler(new CheckpointStatsDetailsHandler(currentGraphs, cache))) .GET("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", handler(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) { - router - // fetch the list of uploaded jars. - .GET("/jars", handler(new JarListHandler(uploadDir))) + // fetch the list of uploaded jars. + GET(router, new JarListHandler(uploadDir)); - // get plan for an uploaded jar - .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) + // get plan for an uploaded jar + GET(router, new JarPlanHandler(uploadDir)); - // run a jar - .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout, config))) + // run a jar + POST(router, new JarRunHandler(uploadDir, timeout, config)); - // upload a jar - .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) + // upload a jar + POST(router, new JarUploadHandler(uploadDir)); - // delete an uploaded jar from submission interface - .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); + // delete an uploaded jar from submission interface + DELETE(router, 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())); + // send an Access Denied message + JarAccessDeniedHandler jad = new JarAccessDeniedHandler(); + GET(router, jad); + POST(router, jad); End diff – OK I thought that they are automatically rejected because we did not register a route for them (only for GET before)
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103480870

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/RestUtils.java —
          @@ -0,0 +1,64 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.runtime.webmonitor.utils;
          +
          +public class RestUtils {
          — End diff –

          The point of this class will be more apparent when the Archiver pattern is integrated. see https://github.com/zentol/flink/commit/83eff62a0cb1741317e645f35c9f47d39a023f45

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103480870 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/RestUtils.java — @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.runtime.webmonitor.utils; + +public class RestUtils { — End diff – The point of this class will be more apparent when the Archiver pattern is integrated. see https://github.com/zentol/flink/commit/83eff62a0cb1741317e645f35c9f47d39a023f45
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103485781

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java —
          @@ -256,117 +256,106 @@ public WebRuntimeMonitor(
          RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler());
          RuntimeMonitorHandler inProgressHandler = handler(cancelWithSavepoint.getInProgressHandler());

          • 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, metricFetcher)))
          • .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs, metricFetcher)))
            -
          • .GET("/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/backpressure", handler(new JobVertexBackPressureHandler(
          • currentGraphs,
          • backPressureStatsTracker,
          • refreshInterval)))
          • .GET("/jobs/:jobid/vertices/:vertexid/metrics", handler(new JobVertexMetricsHandler(metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher)))
          • .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher)))
          • .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/metrics", handler(new JobMetricsHandler(metricFetcher)))
            -
          • .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log",
          • new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
          • TaskManagerLogHandler.FileMode.LOG, config, enableSSL))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout",
          • new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
          • TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL))
          • .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagerMetricsHandler(metricFetcher)))
            + router = new Router();
            + // config how to interact with this web server
            + GET(router, new DashboardConfigHandler(cfg.getRefreshInterval()));
            +
            + // the overview - how many task managers, slots, free slots, ...
            + GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT));
            +
            + // job manager configuration
            + 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, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
            + TaskManagerLogHandler.FileMode.LOG, config, enableSSL));
            + GET(router, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout,
            + TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL));
            + GET(router, new TaskManagerMetricsHandler(metricFetcher));

          + router
          // log and stdout
          .GET("/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") :
          new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile,
          enableSSL))

          .GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") :
          new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile,

          • enableSSL))
            -
          • .GET("/jobmanager/metrics", handler(new JobManagerMetricsHandler(metricFetcher)))
            + enableSSL));
          • // 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()))
            + GET(router, new JobManagerMetricsHandler(metricFetcher));
          • // DELETE is the preferred way of canceling a job (Rest-conform)
          • .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler()))
            + // Cancel a job via GET (for proper integration with YARN this has to be performed via GET)
            + GET(router, new JobCancellationHandler());
            + // DELETE is the preferred way of canceling a job (Rest-conform)
            + DELETE(router, new JobCancellationHandler());
          • .GET("/jobs/:jobid/cancel-with-savepoint", triggerHandler)
          • .GET("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory", triggerHandler)
          • .GET(JobCancellationWithSavepointHandlers.IN_PROGRESS_URL, 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("/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()));
            + // stop a job via GET (for proper integration with YARN this has to be performed via GET)
            + GET(router, new JobStoppingHandler());
            + // DELETE is the preferred way of stopping a job (Rest-conform)
            + DELETE(router, new JobStoppingHandler());

          int maxCachedEntries = config.getInteger(
          ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
          ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
          CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries);

          // Register the checkpoint stats handlers

          • router
          • .GET("/jobs/:jobid/checkpoints", handler(new CheckpointStatsHandler(currentGraphs)))
          • .GET("/jobs/:jobid/checkpoints/config", handler(new CheckpointConfigHandler(currentGraphs)))
          • .GET("/jobs/:jobid/checkpoints/details/:checkpointid", handler(new CheckpointStatsDetailsHandler(currentGraphs, cache)))
          • .GET("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", handler(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)

          { - router - // fetch the list of uploaded jars. - .GET("/jars", handler(new JarListHandler(uploadDir))) + // fetch the list of uploaded jars. + GET(router, new JarListHandler(uploadDir)); - // get plan for an uploaded jar - .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) + // get plan for an uploaded jar + GET(router, new JarPlanHandler(uploadDir)); - // run a jar - .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout, config))) + // run a jar + POST(router, new JarRunHandler(uploadDir, timeout, config)); - // upload a jar - .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) + // upload a jar + POST(router, new JarUploadHandler(uploadDir)); - // delete an uploaded jar from submission interface - .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); + // delete an uploaded jar from submission interface + DELETE(router, 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()));
            + // send an Access Denied message
            + JarAccessDeniedHandler jad = new JarAccessDeniedHandler();
            + GET(router, jad);
            + POST(router, jad);
              • End diff –

          I think they got a 404 previously as unmatched paths are handled by the StaticFileServer.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103485781 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java — @@ -256,117 +256,106 @@ public WebRuntimeMonitor( RuntimeMonitorHandler triggerHandler = handler(cancelWithSavepoint.getTriggerHandler()); RuntimeMonitorHandler inProgressHandler = handler(cancelWithSavepoint.getInProgressHandler()); 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, metricFetcher))) .GET("/jobs/:jobid/vertices", handler(new JobDetailsHandler(currentGraphs, metricFetcher))) - .GET("/jobs/:jobid/vertices/:vertexid", handler(new JobVertexDetailsHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasktimes", handler(new SubtasksTimesHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/taskmanagers", handler(new JobVertexTaskManagersHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/accumulators", handler(new JobVertexAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/backpressure", handler(new JobVertexBackPressureHandler( currentGraphs, backPressureStatsTracker, refreshInterval))) .GET("/jobs/:jobid/vertices/:vertexid/metrics", handler(new JobVertexMetricsHandler(metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", handler(new SubtasksAllAccumulatorsHandler(currentGraphs))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", handler(new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher))) .GET("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", handler(new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher))) .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/metrics", handler(new JobMetricsHandler(metricFetcher))) - .GET("/taskmanagers", handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher))) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher))) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/log", new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.LOG, config, enableSSL)) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/stdout", new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL)) .GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY + "/metrics", handler(new TaskManagerMetricsHandler(metricFetcher))) + router = new Router(); + // config how to interact with this web server + GET(router, new DashboardConfigHandler(cfg.getRefreshInterval())); + + // the overview - how many task managers, slots, free slots, ... + GET(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT)); + + // job manager configuration + 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, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, + TaskManagerLogHandler.FileMode.LOG, config, enableSSL)); + GET(router, new TaskManagerLogHandler(retriever, context, jobManagerAddressPromise.future(), timeout, + TaskManagerLogHandler.FileMode.STDOUT, config, enableSSL)); + GET(router, new TaskManagerMetricsHandler(metricFetcher)); + router // log and stdout .GET("/jobmanager/log", logFiles.logFile == null ? new ConstantTextHandler("(log file unavailable)") : new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.logFile, enableSSL)) .GET("/jobmanager/stdout", logFiles.stdOutFile == null ? new ConstantTextHandler("(stdout file unavailable)") : new StaticFileServerHandler(retriever, jobManagerAddressPromise.future(), timeout, logFiles.stdOutFile, enableSSL)) - .GET("/jobmanager/metrics", handler(new JobManagerMetricsHandler(metricFetcher))) + enableSSL)); // 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())) + GET(router, new JobManagerMetricsHandler(metricFetcher)); // DELETE is the preferred way of canceling a job (Rest-conform) .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler())) + // Cancel a job via GET (for proper integration with YARN this has to be performed via GET) + GET(router, new JobCancellationHandler()); + // DELETE is the preferred way of canceling a job (Rest-conform) + DELETE(router, new JobCancellationHandler()); .GET("/jobs/:jobid/cancel-with-savepoint", triggerHandler) .GET("/jobs/:jobid/cancel-with-savepoint/target-directory/:targetDirectory", triggerHandler) .GET(JobCancellationWithSavepointHandlers.IN_PROGRESS_URL, 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("/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())); + // stop a job via GET (for proper integration with YARN this has to be performed via GET) + GET(router, new JobStoppingHandler()); + // DELETE is the preferred way of stopping a job (Rest-conform) + DELETE(router, new JobStoppingHandler()); int maxCachedEntries = config.getInteger( ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE); CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries); // Register the checkpoint stats handlers router .GET("/jobs/:jobid/checkpoints", handler(new CheckpointStatsHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints/config", handler(new CheckpointConfigHandler(currentGraphs))) .GET("/jobs/:jobid/checkpoints/details/:checkpointid", handler(new CheckpointStatsDetailsHandler(currentGraphs, cache))) .GET("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", handler(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) { - router - // fetch the list of uploaded jars. - .GET("/jars", handler(new JarListHandler(uploadDir))) + // fetch the list of uploaded jars. + GET(router, new JarListHandler(uploadDir)); - // get plan for an uploaded jar - .GET("/jars/:jarid/plan", handler(new JarPlanHandler(uploadDir))) + // get plan for an uploaded jar + GET(router, new JarPlanHandler(uploadDir)); - // run a jar - .POST("/jars/:jarid/run", handler(new JarRunHandler(uploadDir, timeout, config))) + // run a jar + POST(router, new JarRunHandler(uploadDir, timeout, config)); - // upload a jar - .POST("/jars/upload", handler(new JarUploadHandler(uploadDir))) + // upload a jar + POST(router, new JarUploadHandler(uploadDir)); - // delete an uploaded jar from submission interface - .DELETE("/jars/:jarid", handler(new JarDeleteHandler(uploadDir))); + // delete an uploaded jar from submission interface + DELETE(router, 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())); + // send an Access Denied message + JarAccessDeniedHandler jad = new JarAccessDeniedHandler(); + GET(router, jad); + POST(router, jad); End diff – I think they got a 404 previously as unmatched paths are handled by the StaticFileServer.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3376#discussion_r103662852

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/RestUtils.java —
          @@ -0,0 +1,64 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.runtime.webmonitor.utils;
          +
          +public class RestUtils {
          — End diff –

          I'm removing it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3376#discussion_r103662852 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/RestUtils.java — @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.runtime.webmonitor.utils; + +public class RestUtils { — End diff – I'm removing it.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on the issue:

          https://github.com/apache/flink/pull/3376

          @uce I've update the PR.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on the issue: https://github.com/apache/flink/pull/3376 @uce I've update the PR.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on the issue:

          https://github.com/apache/flink/pull/3376

          Thanks for addressing my comments. +1 to merge. Really looking forward to the next PRs with the history server.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on the issue: https://github.com/apache/flink/pull/3376 Thanks for addressing my comments. +1 to merge. Really looking forward to the next PRs with the history server.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on the issue:

          https://github.com/apache/flink/pull/3376

          merging.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on the issue: https://github.com/apache/flink/pull/3376 merging.
          Hide
          Zentol Chesnay Schepler added a comment -

          Implemented in 999baceff36165d950a61dd9cc4342f252e64837

          Show
          Zentol Chesnay Schepler added a comment - Implemented in 999baceff36165d950a61dd9cc4342f252e64837
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/3376

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3376

            People

            • Assignee:
              Zentol Chesnay Schepler
              Reporter:
              Zentol Chesnay Schepler
            • Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development