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

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

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

    https://github.com/apache/flink/pull/4729#discussion_r148810409
  
    --- Diff: 
flink-yarn/src/main/java/org/apache/flink/yarn/YarnWorkerNode.java ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.yarn;
    +
    +import org.apache.flink.runtime.clusterframework.types.ResourceID;
    +import 
org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
    +
    +import org.apache.hadoop.yarn.api.records.Container;
    +
    +import java.io.Serializable;
    +
    +/**
    + * A stored YARN worker, which contains the YARN container.
    + */
    +public class YarnWorkerNode implements ResourceIDRetrievable, Serializable 
{
    --- End diff --
    
    Actually this class should not be `serializable` because `Container` is 
also not serializable.


> Implement container release to support dynamic scaling
> ------------------------------------------------------
>
>                 Key: FLINK-7076
>                 URL: https://issues.apache.org/jira/browse/FLINK-7076
>             Project: Flink
>          Issue Type: Sub-task
>          Components: ResourceManager
>            Reporter: Till Rohrmann
>            Assignee: Shuyi Chen
>            Priority: Major
>              Labels: flip-6
>
> In order to support dynamic scaling, the {{YarnResourceManager}} has to be 
> able to dynamically free containers. We have to implement the 
> {{YarnResourceManager#stopWorker}} method.



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

Reply via email to