[ 
https://issues.apache.org/jira/browse/FLINK-7040?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16136714#comment-16136714
 ] 

ASF GitHub Bot commented on FLINK-7040:
---------------------------------------

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

    https://github.com/apache/flink/pull/4569#discussion_r134266899
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientEndpoint.java
 ---
    @@ -0,0 +1,263 @@
    +/*
    + * 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.rest;
    +
    +import org.apache.flink.configuration.ConfigConstants;
    +import org.apache.flink.runtime.concurrent.Executors;
    +import org.apache.flink.runtime.concurrent.FutureUtils;
    +import org.apache.flink.runtime.rest.handler.PipelineErrorHandler;
    +import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
    +import org.apache.flink.runtime.rest.messages.MessageHeaders;
    +import org.apache.flink.runtime.rest.messages.ParameterMapper;
    +import org.apache.flink.runtime.rest.messages.RequestBody;
    +import org.apache.flink.runtime.rest.messages.ResponseBody;
    +import org.apache.flink.runtime.rest.util.RestMapperUtils;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
    +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
    +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufInputStream;
    +import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
    +import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
    +import 
org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
    +import 
org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler;
    +import 
org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
    +import 
org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
    +import 
org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpRequest;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpRequest;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpClientCodec;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectAggregator;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion;
    +import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
    +
    +import com.fasterxml.jackson.core.JsonParseException;
    +import com.fasterxml.jackson.core.JsonProcessingException;
    +import com.fasterxml.jackson.databind.JsonMappingException;
    +import com.fasterxml.jackson.databind.JsonNode;
    +import com.fasterxml.jackson.databind.ObjectMapper;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.net.ssl.SSLEngine;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.StringWriter;
    +import java.util.concurrent.CompletableFuture;
    +import java.util.concurrent.Executor;
    +
    +/**
    + * This client is the counter-part to the {@link RestServerEndpoint}.
    + */
    +public class RestClientEndpoint {
    +   private static final Logger LOG = 
LoggerFactory.getLogger(RestClientEndpoint.class);
    +
    +   private static final ObjectMapper objectMapper = 
RestMapperUtils.getStrictObjectMapper();
    +
    +   private final String configuredTargetAddress;
    +   private final int configuredTargetPort;
    +   private final SSLEngine sslEngine;
    +
    +   private Bootstrap bootstrap;
    +
    +   private final ClientHandler handler = new ClientHandler();
    +
    +   private CompletableFuture<?> lastFuture = 
CompletableFuture.completedFuture(null);
    +
    +   private final Executor directExecutor = Executors.directExecutor();
    +
    +   public RestClientEndpoint(RestClientEndpointConfiguration 
configuration) {
    +           this.configuredTargetAddress = 
configuration.getTargetRestEndpointAddress();
    +           this.configuredTargetPort = 
configuration.getTargetRestEndpointPort();
    +           this.sslEngine = configuration.getSslEngine();
    +   }
    +
    +   public void start() {
    +           ChannelInitializer<SocketChannel> initializer = new 
ChannelInitializer<SocketChannel>() {
    +
    +                   @Override
    +                   protected void initChannel(SocketChannel ch) {
    +
    +                           // SSL should be the first handler in the 
pipeline
    +                           if (sslEngine != null) {
    +                                   ch.pipeline().addLast("ssl", new 
SslHandler(sslEngine));
    +                           }
    +
    +                           ch.pipeline()
    +                                   .addLast(new HttpClientCodec())
    +                                   .addLast(new HttpObjectAggregator(1024 
* 1024))
    +                                   .addLast(handler)
    +                                   .addLast(new PipelineErrorHandler(LOG));
    +                   }
    +           };
    +
    +           NioEventLoopGroup group = new NioEventLoopGroup(1);
    +
    +           bootstrap = new Bootstrap();
    +           bootstrap
    +                   .group(group)
    +                   .channel(NioSocketChannel.class)
    +                   .handler(initializer);
    +
    +           LOG.info("Rest client endpoint started.");
    +   }
    +
    +   public void shutdown() {
    +           if (bootstrap != null) {
    +                   if (bootstrap.group() != null) {
    +                           bootstrap.group().shutdownGracefully();
    +                   }
    +           }
    +   }
    +
    +   public <M extends MessageHeaders<R, P, U>, U extends ParameterMapper, R 
extends RequestBody, P extends ResponseBody> CompletableFuture<P> sendRequest(M 
messageHeaders, U urlResolver, R request) throws IOException {
    +           Preconditions.checkNotNull(messageHeaders);
    +           Preconditions.checkNotNull(request);
    +
    +           String targetUrl = ParameterMapper.resolveUrl(
    +                   messageHeaders.getTargetRestEndpointURL(),
    +                   
urlResolver.mapPathParameters(messageHeaders.getPathParameters()),
    +                   
urlResolver.mapQueryParameters(messageHeaders.getQueryParameters())
    +           );
    +
    +           LOG.debug("Sending request of class {} to {}", 
request.getClass(), targetUrl);
    +           // serialize payload
    +           StringWriter sw = new StringWriter();
    +           objectMapper.writeValue(sw, request);
    +           ByteBuf payload = 
Unpooled.wrappedBuffer(sw.toString().getBytes(ConfigConstants.DEFAULT_CHARSET));
    +
    +           // create request and set headers
    +           FullHttpRequest httpRequest = new 
DefaultFullHttpRequest(HttpVersion.HTTP_1_1, 
messageHeaders.getHttpMethod().getNettyHttpMethod(), targetUrl, payload);
    +           httpRequest.headers()
    +                   .add(HttpHeaders.Names.CONTENT_LENGTH, 
payload.capacity())
    +                   .add(HttpHeaders.Names.CONTENT_TYPE, "application/json; 
charset=" + ConfigConstants.DEFAULT_CHARSET.name())
    +                   .set(HttpHeaders.Names.HOST, configuredTargetAddress + 
":" + configuredTargetPort)
    +                   .set(HttpHeaders.Names.CONNECTION, 
HttpHeaders.Values.CLOSE);
    +
    +           synchronized (this) {
    +                   // This ensures strict sequential processing of 
requests.
    +                   // If we send new requests immediately we can no longer 
make assumptions about the order in which responses
    +                   // arrive, due to which the handler cannot know which 
future he should complete (not to mention what response
    +                   // type to read).
    +                   CompletableFuture<P> nextFuture = lastFuture
    +                           .handleAsync((f, e) -> 
submitRequest(httpRequest, messageHeaders), directExecutor)
    --- End diff --
    
    better to call `handle()`, then you don't have to specify the direct 
executor.


> Flip-6 client-cluster communication
> -----------------------------------
>
>                 Key: FLINK-7040
>                 URL: https://issues.apache.org/jira/browse/FLINK-7040
>             Project: Flink
>          Issue Type: New Feature
>          Components: Cluster Management, Mesos
>            Reporter: Till Rohrmann
>            Assignee: Chesnay Schepler
>            Priority: Critical
>              Labels: flip-6
>
> With the new Flip-6 architecture, the client will communicate with the 
> cluster in a RESTful manner.
> The cluster shall support the following REST calls:
> * 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)
> * Lookup job leader (GET): Gets the JM leader for the given job
> * Get job status (GET): Get the status of an executed job (and maybe the 
> JobExecutionResult)
> * Cancel job (PUT): Cancel the given job
> * Stop job (PUT): Stops the given job
> * Take savepoint (POST): Take savepoint for given job (How to return the 
> savepoint under which the savepoint was stored? Maybe always having to 
> specify a path)
> * Get KV state (GET): Gets the KV state for the given job and key (Queryable 
> state)
> * Poll/subscribe to notifications for job (GET, WebSocket): Polls new 
> notifications from the execution of the given job/Opens WebSocket to receive 
> notifications
> The first four REST calls will be served by the REST endpoint running in the 
> application master/cluster entrypoint. The other calls will be served by a 
> REST endpoint running along side to the JobManager.
> Detailed information about different implementations and their pros and cons 
> can be found in this document:
> https://docs.google.com/document/d/1eIX6FS9stwraRdSUgRSuLXC1sL7NAmxtuqIXe_jSi-k/edit?usp=sharing
> The implementation will most likely be Netty based.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to