You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tillrohrmann <gi...@git.apache.org> on 2017/09/26 14:55:56 UTC

[GitHub] flink pull request #4727: [FLINK-7667] [flip6] Use ArchivedExecutionGraph as...

GitHub user tillrohrmann opened a pull request:

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

    [FLINK-7667] [flip6] Use ArchivedExecutionGraph as serializable AccessExecutionGraph

    ## What is the purpose of the change
    
    This commit removes AccessExecutionGraph#getCheckpointCoordinator and changes the
    AccessExecutionGraph#getJobCheckpointSettings into #getJobCheckpointConfiguration.
    The JobCheckpointConfiguration only contains the CheckpointCoordinator relevant
    configuration settings and excludes the serialized state backend and the
    serialized master hooks. That way we don't send unnecessary information when
    the ArchivedExecutionGraph is requested.
    
    ## Verifying this change
    
    This change is a trivial rework / code cleanup without any test coverage.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (no)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (not applicable)
    


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

    $ git pull https://github.com/tillrohrmann/flink serializableAccessExecutionGraph

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

    https://github.com/apache/flink/pull/4727.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 #4727
    
----
commit ed2d5482c315d64216e4a645b68e7b48dfb89456
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-09-21T08:53:24Z

    [FLINK-7650] [flip6] Port JobCancellationHandler to new REST endpoint
    
    Let the JobCancellationHandler implement the LegacyRestHandler interface. Moreover,
    this commit adds the DELETE method to HttpMethodWrapper and the
    RestServerEndpoint#registerHandler method.

commit e83065fa8102c10a10cdc981f0c7608577c20c34
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-09-21T14:47:18Z

    [FLINK-7649] [flip6] Extend JobTerminationHandler to support stop
    
    Rename the JobCancellationHandler into JobTerminationHandler which is now responsible
    for terminating jobs. Moreover, this commits adds two termination modes, cancel and stop,
    which are specified by providing a query parameter.

commit 02e345b8f34890d67230d0a75a818a8e3b65cf2b
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-09-22T11:31:12Z

    [FLINK-7667] [flip6] Use ArchivedExecutionGraph as serializable AccessExecutionGraph
    
    This commit removes AccessExecutionGraph#getCheckpointCoordinator and changes the
    AccessExecutionGraph#getJobCheckpointSettings into #getJobCheckpointConfiguration.
    The JobCheckpointConfiguration only contains the CheckpointCoordinator relevant
    configuration settings and excludes the serialized state backend and the
    serialized master hooks. That way we don't send unnecessary information when
    the ArchivedExecutionGraph is requested.

----


---

[GitHub] flink issue #4727: [FLINK-7667] [flip6] Use ArchivedExecutionGraph as serial...

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

    https://github.com/apache/flink/pull/4727
  
    Thanks for the review @zentol. Rebasing this commit and waiting on Travis.


---

[GitHub] flink issue #4727: [FLINK-7667] [flip6] Use ArchivedExecutionGraph as serial...

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

    https://github.com/apache/flink/pull/4727
  
    Merging this PR.


---

[GitHub] flink pull request #4727: [FLINK-7667] [flip6] Use ArchivedExecutionGraph as...

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

    https://github.com/apache/flink/pull/4727#discussion_r141110165
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingConfiguration.java ---
    @@ -0,0 +1,128 @@
    +/*
    + * 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.jobgraph.tasks;
    +
    +import org.apache.flink.util.Preconditions;
    +
    +import java.io.Serializable;
    +import java.util.Objects;
    +
    +public class JobCheckpointingConfiguration implements Serializable {
    --- End diff --
    
    missing javadoc. Can we also find a better name, since "Settings" and "Configuration" are virtually interchangeable? (Like CHeckpointCoordinatorConfig?)


---

[GitHub] flink pull request #4727: [FLINK-7667] [flip6] Use ArchivedExecutionGraph as...

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

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


---

[GitHub] flink issue #4727: [FLINK-7667] [flip6] Use ArchivedExecutionGraph as serial...

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

    https://github.com/apache/flink/pull/4727
  
    I've addressed your comments @zentol. The PR should now build and I renamed `JobCheckpointingConfiguration` into `CheckpointCoordinatorConfiguration`.


---

[GitHub] flink issue #4727: [FLINK-7667] [flip6] Use ArchivedExecutionGraph as serial...

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

    https://github.com/apache/flink/pull/4727
  
    +1.


---

[GitHub] flink pull request #4727: [FLINK-7667] [flip6] Use ArchivedExecutionGraph as...

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

    https://github.com/apache/flink/pull/4727#discussion_r141314449
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingConfiguration.java ---
    @@ -0,0 +1,128 @@
    +/*
    + * 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.jobgraph.tasks;
    +
    +import org.apache.flink.util.Preconditions;
    +
    +import java.io.Serializable;
    +import java.util.Objects;
    +
    +public class JobCheckpointingConfiguration implements Serializable {
    --- End diff --
    
    Thanks for pointing this out. I will add it and think about a better name.


---