tillrohrmann commented on a change in pull request #17485: URL: https://github.com/apache/flink/pull/17485#discussion_r791509668
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java ########## @@ -0,0 +1,152 @@ +/* + * 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.highavailability.zookeeper; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobStoreService; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory; +import org.apache.flink.runtime.highavailability.AbstractHaServices; +import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.util.ZooKeeperUtils; + +import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework; +import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths; +import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException; + +import javax.annotation.Nonnull; + +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Abstract ZooKeeper based HA services. */ +public abstract class AbstractZooKeeperHaServices extends AbstractHaServices { + /** The curator resource to use. */ + private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper; + + public AbstractZooKeeperHaServices( + CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper, + Executor executor, + Configuration configuration, + BlobStoreService blobStoreService) { + super(configuration, executor, blobStoreService); + this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper); + } + + protected final CuratorFramework getCuratorFramework() { + return curatorFrameworkWrapper.asCuratorFramework(); + } + + @Override + public CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception { + return new ZooKeeperCheckpointRecoveryFactory( + ZooKeeperUtils.useNamespaceAndEnsurePath( + curatorFrameworkWrapper.asCuratorFramework(), ZooKeeperUtils.getJobsPath()), + configuration, + ioExecutor); + } + + @Override + public JobGraphStore createJobGraphStore() throws Exception { + return ZooKeeperUtils.createJobGraphs( + curatorFrameworkWrapper.asCuratorFramework(), configuration); + } + + @Override + public RunningJobsRegistry createRunningJobsRegistry() { + return new ZooKeeperRunningJobsRegistry( + curatorFrameworkWrapper.asCuratorFramework(), configuration); + } + + @Override + protected void internalClose() throws Exception { + curatorFrameworkWrapper.close(); + } + + @Override + protected void internalCleanup() throws Exception { + cleanupZooKeeperPaths(); + } + + @Override + protected void internalCleanupJobData(JobID jobID) throws Exception { + deleteZNode(ZooKeeperUtils.getLeaderPathForJob(jobID)); + } + + /** Cleans up leftover ZooKeeper paths. */ + private void cleanupZooKeeperPaths() throws Exception { + deleteOwnedZNode(); + tryDeleteEmptyParentZNodes(); + } + + private void deleteOwnedZNode() throws Exception { + deleteZNode("/"); + } + + protected void deleteZNode(String path) throws Exception { + ZooKeeperUtils.deleteZNode(curatorFrameworkWrapper.asCuratorFramework(), path); + } + + /** + * Tries to delete empty parent znodes. + * + * <p>IMPORTANT: This method can be removed once all supported ZooKeeper versions support the + * container {@link org.apache.zookeeper.CreateMode}. + * + * @throws Exception if the deletion fails for other reason than {@link + * KeeperException.NotEmptyException} + */ + private void tryDeleteEmptyParentZNodes() throws Exception { + // try to delete the parent znodes if they are empty + String remainingPath = + getParentPath( + getNormalizedPath( + curatorFrameworkWrapper.asCuratorFramework().getNamespace())); + final CuratorFramework nonNamespaceClient = + curatorFrameworkWrapper.asCuratorFramework().usingNamespace(null); + + while (!isRootPath(remainingPath)) { + try { + nonNamespaceClient.delete().forPath(remainingPath); + } catch (KeeperException.NotEmptyException ignored) { + // We can only delete empty znodes + break; + } + + remainingPath = getParentPath(remainingPath); + } + } + + private static boolean isRootPath(String remainingPath) { + return ZKPaths.PATH_SEPARATOR.equals(remainingPath); + } + + @Nonnull Review comment: Will remove it. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java ########## @@ -0,0 +1,152 @@ +/* + * 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.highavailability.zookeeper; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobStoreService; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory; +import org.apache.flink.runtime.highavailability.AbstractHaServices; +import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.util.ZooKeeperUtils; + +import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework; +import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths; +import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException; + +import javax.annotation.Nonnull; + +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Abstract ZooKeeper based HA services. */ +public abstract class AbstractZooKeeperHaServices extends AbstractHaServices { + /** The curator resource to use. */ + private final CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper; + + public AbstractZooKeeperHaServices( + CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper, + Executor executor, + Configuration configuration, + BlobStoreService blobStoreService) { + super(configuration, executor, blobStoreService); + this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper); + } + + protected final CuratorFramework getCuratorFramework() { + return curatorFrameworkWrapper.asCuratorFramework(); + } + + @Override + public CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception { + return new ZooKeeperCheckpointRecoveryFactory( + ZooKeeperUtils.useNamespaceAndEnsurePath( + curatorFrameworkWrapper.asCuratorFramework(), ZooKeeperUtils.getJobsPath()), + configuration, + ioExecutor); + } + + @Override + public JobGraphStore createJobGraphStore() throws Exception { + return ZooKeeperUtils.createJobGraphs( + curatorFrameworkWrapper.asCuratorFramework(), configuration); + } + + @Override + public RunningJobsRegistry createRunningJobsRegistry() { + return new ZooKeeperRunningJobsRegistry( + curatorFrameworkWrapper.asCuratorFramework(), configuration); + } + + @Override + protected void internalClose() throws Exception { + curatorFrameworkWrapper.close(); + } + + @Override + protected void internalCleanup() throws Exception { + cleanupZooKeeperPaths(); + } + + @Override + protected void internalCleanupJobData(JobID jobID) throws Exception { + deleteZNode(ZooKeeperUtils.getLeaderPathForJob(jobID)); + } + + /** Cleans up leftover ZooKeeper paths. */ + private void cleanupZooKeeperPaths() throws Exception { + deleteOwnedZNode(); + tryDeleteEmptyParentZNodes(); + } + + private void deleteOwnedZNode() throws Exception { + deleteZNode("/"); + } + + protected void deleteZNode(String path) throws Exception { + ZooKeeperUtils.deleteZNode(curatorFrameworkWrapper.asCuratorFramework(), path); + } + + /** + * Tries to delete empty parent znodes. + * + * <p>IMPORTANT: This method can be removed once all supported ZooKeeper versions support the + * container {@link org.apache.zookeeper.CreateMode}. + * + * @throws Exception if the deletion fails for other reason than {@link + * KeeperException.NotEmptyException} + */ + private void tryDeleteEmptyParentZNodes() throws Exception { + // try to delete the parent znodes if they are empty + String remainingPath = + getParentPath( + getNormalizedPath( + curatorFrameworkWrapper.asCuratorFramework().getNamespace())); + final CuratorFramework nonNamespaceClient = + curatorFrameworkWrapper.asCuratorFramework().usingNamespace(null); + + while (!isRootPath(remainingPath)) { + try { + nonNamespaceClient.delete().forPath(remainingPath); + } catch (KeeperException.NotEmptyException ignored) { + // We can only delete empty znodes + break; + } + + remainingPath = getParentPath(remainingPath); + } + } + + private static boolean isRootPath(String remainingPath) { + return ZKPaths.PATH_SEPARATOR.equals(remainingPath); + } + + @Nonnull + private static String getNormalizedPath(String path) { + return ZKPaths.makePath(path, ""); + } + + @Nonnull Review comment: Will remove it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org