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

    https://github.com/apache/flink/pull/1153#discussion_r41535673
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java
 ---
    @@ -0,0 +1,310 @@
    +/*
    + * 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.checkpoint;
    +
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.curator.framework.api.BackgroundCallback;
    +import org.apache.curator.framework.api.CuratorEvent;
    +import org.apache.curator.framework.api.CuratorEventType;
    +import org.apache.curator.utils.ZKPaths;
    +import org.apache.flink.runtime.jobmanager.RecoveryMode;
    +import org.apache.flink.runtime.state.StateHandleProvider;
    +import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
    +import 
org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore.ZooKeeperStateHandle;
    +import org.apache.zookeeper.data.Stat;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayDeque;
    +import java.util.ArrayList;
    +import java.util.ConcurrentModificationException;
    +import java.util.List;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +/**
    + * {@link CompletedCheckpointStore} for JobManagers running in {@link 
RecoveryMode#ZOOKEEPER}.
    + *
    + * <p>Checkpoints are added under a ZNode per job:
    + * <pre>
    + * +----O /flink/checkpoints/&lt;job-id&gt;  [persistent]
    + * .    |
    + * .    +----O /flink/checkpoints/&lt;job-id&gt;/1 [persistent_sequential]
    + * .    .                                  .
    + * .    .                                  .
    + * .    .                                  .
    + * .    +----O /flink/checkpoints/&lt;job-id&gt;/N [persistent_sequential]
    + * </pre>
    + *
    + * <p>During recovery, the latest checkpoint is read from ZooKeeper. If 
there is more than one,
    + * only the latest one is used and older ones are discarded (even if the 
maximum number
    + * of retained checkpoints is greater than one). The sequential; cversion 
determines, which
    + * checkpoint is the latest one.
    + *
    + * <p>If there is a network partition and multiple JobManagers run 
concurrent checkpoints for the
    + * same program, it is OK to take any valid successful checkpoint as long 
as the "history" of
    + * checkpoints is consistent. Currently, after recovery we start out with 
only a single
    + * checkpoint in oder to guarantee this.
    + */
    +public class ZooKeeperCompletedCheckpointStore implements 
CompletedCheckpointStore {
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(ZooKeeperCompletedCheckpointStore.class);
    +
    +   /** Curator ZooKeeper client */
    +   private final CuratorFramework client;
    +
    +   /** Completed checkpoints in ZooKeeper */
    +   private final ZooKeeperStateHandleStore<CompletedCheckpoint> 
checkpointsInZooKeeper;
    +
    +   /** The maximum number of checkpoints to retain (at least 1). */
    +   private final int maxNumberOfCheckpointsToRetain;
    +
    +   /** User class loader for discarding {@link CompletedCheckpoint} 
instances. */
    +   private final ClassLoader userClassLoader;
    +
    +   /** Local completed checkpoints. */
    +   private final ArrayDeque<ZooKeeperStateHandle<CompletedCheckpoint>> 
checkpointStateHandles;
    +
    +   /**
    +    * Creates a {@link ZooKeeperCompletedCheckpointStore} instance.
    +    *
    +    * @param maxNumberOfCheckpointsToRetain The maximum number of 
checkpoints to retain (at
    +    *                                       least 1). Adding more 
checkpoints than this results
    +    *                                       in older checkpoints being 
discarded. On recovery,
    +    *                                       we will only start with a 
single checkpoint.
    +    * @param userClassLoader                The user class loader used to 
discard checkpoints
    +    * @param client                         The Curator ZooKeeper client
    +    * @param checkpointsPath                The ZooKeeper path for the 
checkpoints (needs to
    +    *                                       start with a '/')
    +    * @param stateHandleProvider            The state handle provider for 
checkpoints
    +    * @throws Exception
    +    */
    +   public ZooKeeperCompletedCheckpointStore(
    +                   int maxNumberOfCheckpointsToRetain,
    +                   ClassLoader userClassLoader,
    +                   CuratorFramework client,
    +                   String checkpointsPath,
    +                   StateHandleProvider<CompletedCheckpoint> 
stateHandleProvider) throws Exception {
    +
    +           checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain 
at least one checkpoint.");
    +
    +           this.maxNumberOfCheckpointsToRetain = 
maxNumberOfCheckpointsToRetain;
    +           this.userClassLoader = checkNotNull(userClassLoader, "User 
class loader");
    +
    +           checkNotNull(client, "Curator client");
    +           checkNotNull(checkpointsPath, "Checkpoints path");
    +           checkNotNull(stateHandleProvider, "State handle provider");
    +
    +           // Ensure that the checkpoints path exists
    +           client.newNamespaceAwareEnsurePath(checkpointsPath)
    +                           .ensure(client.getZookeeperClient());
    +
    +           // All operations will have the path as root
    +           this.client = client.usingNamespace(client.getNamespace() + 
checkpointsPath);
    +
    +           this.checkpointsInZooKeeper = new ZooKeeperStateHandleStore<>(
    +                           this.client, stateHandleProvider);
    +
    +           this.checkpointStateHandles = new 
ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1);
    +
    +           LOG.info("Initialized in '{}'.", checkpointsPath);
    +   }
    +
    +   /**
    +    * Gets the latest checkpoint from ZooKeeper and removes all others.
    +    *
    +    * <p><strong>Important</strong>: Even if there are more than one 
checkpoint in ZooKeeper,
    +    * this will only recover the latest and discard the others. Otherwise, 
there is no guarantee
    +    * that the history of checkpoints is consistent.
    +    */
    +   @Override
    +   public void recover() throws Exception {
    +           LOG.info("Recovering checkpoints from ZooKeeper.");
    +
    +           // Get all there is first
    +           List<ZooKeeperStateHandle<CompletedCheckpoint>> 
initialCheckpoints;
    +           while (true) {
    +                   try {
    +                           initialCheckpoints = 
checkpointsInZooKeeper.getAllSortedByName();
    +                           break;
    +                   }
    +                   catch (ConcurrentModificationException e) {
    +                           LOG.warn("Concurrent modification while reading 
from ZooKeeper. Retrying.");
    +                   }
    +           }
    +
    +           int numberOfInitialCheckpoints = initialCheckpoints.size();
    +
    +           LOG.info("Found {} checkpoints in ZooKeeper.", 
numberOfInitialCheckpoints);
    +
    +           if (numberOfInitialCheckpoints > 0) {
    +                   // Take the last one. This is the latest checkpoints, 
because path names are strictly
    +                   // increasing (checkpoint ID).
    +                   ZooKeeperStateHandle<CompletedCheckpoint> latest = 
initialCheckpoints
    +                                   .get(numberOfInitialCheckpoints - 1);
    +
    +                   CompletedCheckpoint latestCheckpoint = 
latest.getState(userClassLoader);
    +
    +                   checkpointStateHandles.add(latest);
    +
    +                   LOG.info("Initialized with {}. Removing all older 
checkpoints.", latestCheckpoint);
    +
    +                   for (int i = 0; i < numberOfInitialCheckpoints - 1; 
i++) {
    +                           try {
    +                                   
removeFromZooKeeperAndDiscardCheckpoint(initialCheckpoints.get(i));
    +                           }
    +                           catch (Exception e) {
    +                                   LOG.error("Failed to discard 
checkpoint", e);
    +                           }
    +                   }
    +           }
    +   }
    +
    +   @Override
    +   public int getNextCheckpointID() throws Exception {
    --- End diff --
    
    Sorry for my confusion. The desired behaviour is to have increasing 
checkpoints after failed checkpoints as well. E.g. the task managers should 
always see increasing IDs, even for non-completed checkpoints. I'm reverting 
the checkpoint ID counter removal.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to