You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GJL <gi...@git.apache.org> on 2018/03/12 22:30:08 UTC

[GitHub] flink pull request #5685: [FLINK-7521][flip6]

GitHub user GJL opened a pull request:

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

    [FLINK-7521][flip6]

    ## What is the purpose of the change
    
    *Make HTTP request and response limits configurable. A relatively high default value is chosen (100 mb) because Netty does not allocate the upper limit at once.*
    
    
    ## Brief change log
    
      - *Make HTTP request and response limits configurable.*
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
      - *Added tests to `RestServerEndpointITCase`*
      - *Manually verified that client and server limits are respected.*
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / **no**)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
      - The serializers: (yes / **no** / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
      - The S3 file system connector: (yes / **no** / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / **no**)
      - If yes, how is the feature documented? (not applicable / docs / JavaDocs / **not documented**)


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

    $ git pull https://github.com/GJL/flink FLINK-7521-2

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

    https://github.com/apache/flink/pull/5685.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 #5685
    
----
commit aef8fa247a4c8b14f4dd7ce8f324ccd89bb2ce14
Author: zjureel <zj...@...>
Date:   2017-09-07T02:39:39Z

    [FLINK-7521] Add config option to set the content length limit of REST server and client

commit ff6c7eb1127ff1870f479c1b779379cc22c9dc87
Author: gyao <ga...@...>
Date:   2018-03-12T14:44:27Z

    [FLINK-7521][flip6] Remove RestServerEndpoint#MAX_REQUEST_SIZE_BYTES

commit a14e5935dd9132ddb43e55e357674d390ff9c597
Author: gyao <ga...@...>
Date:   2018-03-12T22:16:25Z

    [FLINK-7521][flip6] Return HTTP 413 if request limit is exceeded.
    
    Remove unnecessary PipelineErrorHandler from RestClient.
    Rename config keys for configuring request and response limits.
    Set response headers for all error responses.

----


---

[GitHub] flink issue #5685: [FLINK-7521][flip6] Remove the 10MB limit from the curren...

Posted by GJL <gi...@git.apache.org>.
Github user GJL commented on the issue:

    https://github.com/apache/flink/pull/5685
  
    Maybe docs should be updated.


---

[GitHub] flink pull request #5685: [FLINK-7521][flip6] Remove the 10MB limit from the...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5685#discussion_r174225988
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/FlinkHttpObjectAggregator.java ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.RestOptions;
    +import org.apache.flink.runtime.rest.handler.util.HandlerUtils;
    +import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
    +
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.handler.codec.TooLongFrameException;
    +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObject;
    +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Same as {@link org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectDecoder}
    + * but returns HTTP 413 to the client if the payload exceeds {@link #maxContentLength}.
    + */
    +public class FlinkHttpObjectAggregator extends org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectAggregator {
    +
    +	private final Map<String, String> responseHeaders;
    +
    +	public FlinkHttpObjectAggregator(final int maxContentLength, final Map<String, String> responseHeaders) {
    +		super(maxContentLength);
    +		this.responseHeaders = responseHeaders;
    --- End diff --
    
    `checkNotNull` or `@NonNull` missing


---

[GitHub] flink pull request #5685: [FLINK-7521][flip6] Remove the 10MB limit from the...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---