[ https://issues.apache.org/jira/browse/FLINK-7072?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16192742#comment-16192742 ]
ASF GitHub Bot commented on FLINK-7072: --------------------------------------- Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/4742#discussion_r142897388 --- Diff: flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java --- @@ -0,0 +1,219 @@ +/* + * 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.client.program.rest; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.messages.BlobServerPortHeaders; +import org.apache.flink.runtime.rest.messages.BlobServerPortResponseBody; +import org.apache.flink.runtime.rest.messages.EmptyResponseBody; +import org.apache.flink.runtime.rest.messages.JobTerminationHeaders; +import org.apache.flink.runtime.rest.messages.JobTerminationMessageParameters; +import org.apache.flink.runtime.rest.messages.TerminationModeQueryParameter; +import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders; +import org.apache.flink.runtime.rest.messages.job.JobSubmitRequestBody; +import org.apache.flink.runtime.rest.messages.job.JobSubmitResponseBody; + +import javax.annotation.Nullable; + +import java.net.InetSocketAddress; +import java.net.URL; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * A {@link ClusterClient} implementation that communicates via HTTP REST requests. + */ +public class RestClusterClient extends ClusterClient { + + private final RestClusterClientConfiguration restClusterClientConfiguration; + private final RestClient restClient; + private final ExecutorService executorService = Executors.newFixedThreadPool(4); + + public RestClusterClient(Configuration config) throws Exception { + this(config, RestClusterClientConfiguration.fromConfiguration(config)); + } + + public RestClusterClient(Configuration config, RestClusterClientConfiguration configuration) throws Exception { + super(config); + this.restClusterClientConfiguration = configuration; + this.restClient = new RestClient(configuration.getRestEndpointConfiguration(), executorService); + } + + @Override + public void shutdown() { + try { + // we only call this for legacy reasons to shutdown components that are started in the ClusterClient constructor + super.shutdown(); + } catch (Exception e) { + log.error("An error occurred during the client shutdown.", e); + } + this.restClient.shutdown(Time.seconds(5)); + this.executorService.shutdown(); --- End diff -- Better to use `Executors.gracefulShutdown` here. > Create RESTful cluster endpoint > ------------------------------- > > Key: FLINK-7072 > URL: https://issues.apache.org/jira/browse/FLINK-7072 > Project: Flink > Issue Type: Sub-task > Components: Distributed Coordination > Reporter: Till Rohrmann > Assignee: Chesnay Schepler > Labels: flip-6 > Fix For: 1.4.0 > > > In order to communicate with the cluster from the RESTful client, we have to > implement a RESTful cluster endpoint. The endpoint shall support the > following operations: > * List jobs (GET): Get list of all running jobs on the cluster > * Submit job (POST): Submit a job to the cluster (only supported in session > mode) > * Get job status (GET): Get the status of an executed job (and maybe the > JobExecutionResult) > * Lookup job leader (GET): Gets the JM leader for the given job > This endpoint will run in session mode alongside the dispatcher/session > runner and forward calls to this component which maintains a view on all > currently executed jobs. > In the per-job mode, the endpoint will return only the single running job and > the address of the JobManager alongside which it is running. Furthermore, it > won't accept job submissions. -- This message was sent by Atlassian JIRA (v6.4.14#64029)