Skip to content

Commit

Permalink
[FLINK-4657] Implement HighAvailabilityServices based on ZooKeeper
Browse files Browse the repository at this point in the history
[FLINK-4657] Implement a few rpc calls for JobMaster

[FLINK-4657][cluster management] Address review comments

[FLINK-4657][cluster management] Throw exception when error occurred when request input split
  • Loading branch information
KurtYoung authored and StephanEwen committed Dec 23, 2016
1 parent 1020950 commit c2891c5
Show file tree
Hide file tree
Showing 14 changed files with 462 additions and 64 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
* <li>JobManager leader election and leader retrieval</li>
* <li>Persistence for checkpoint metadata</li>
* <li>Registering the latest completed checkpoint(s)</li>
* <li>Persistence for submitted job graph</li>
* </ul>
*/
public interface HighAvailabilityServices {
Expand All @@ -48,12 +49,10 @@ public interface HighAvailabilityServices {
* @return
* @throws Exception
*/
LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception;
LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception;

/**
* Gets the leader election service for the cluster's resource manager.
* @return
* @throws Exception
*/
LeaderElectionService getResourceManagerLeaderElectionService() throws Exception;

Expand All @@ -62,7 +61,7 @@ public interface HighAvailabilityServices {
*
* @param jobID The identifier of the job running the election.
*/
LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception;
LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception;

/**
* Gets the checkpoint recovery factory for the job manager
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Excepti
}

@Override
public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception {
public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
return new StandaloneLeaderRetrievalService(jobMastersAddress.get(jobID), new UUID(0, 0));
}

Expand All @@ -89,7 +89,7 @@ public LeaderElectionService getResourceManagerLeaderElectionService() throws Ex
}

@Override
public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
return new StandaloneLeaderElectionService();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
/*
* 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:https://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;

import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
import org.apache.flink.runtime.util.ZooKeeperUtils;

import java.util.concurrent.Executor;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* An implementation of the {@link HighAvailabilityServices} with zookeeper.
*/
public class ZookeeperHaServices implements HighAvailabilityServices {

private static final String DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX = "/resource-manager";

/** The ZooKeeper client to use */
private final CuratorFramework client;

/** The executor to run ZooKeeper callbacks on */
private final Executor executor;

/** The runtime configuration */
private final Configuration configuration;

public ZookeeperHaServices(CuratorFramework client, Executor executor, Configuration configuration) {
this.client = checkNotNull(client);
this.executor = checkNotNull(executor);
this.configuration = checkNotNull(configuration);
}

@Override
public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
}

@Override
public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathSuffixForJob(jobID));
}

@Override
public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
return ZooKeeperUtils.createLeaderElectionService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
}

@Override
public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathSuffixForJob(jobID));
}

@Override
public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
return new ZooKeeperCheckpointRecoveryFactory(client, configuration, executor);
}

@Override
public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
return ZooKeeperUtils.createSubmittedJobGraphs(client, configuration, executor);
}

private static String getPathSuffixForJob(final JobID jobID) {
return String.format("/job-managers/%s", jobID);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,5 @@ interface SubmittedJobGraphListener {
* @param jobId The {@link JobID} of the removed job graph
*/
void onRemovedJobGraph(JobID jobId);

}

}
Original file line number Diff line number Diff line change
Expand Up @@ -21,20 +21,18 @@
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobmanager.OnCompletionActions;
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
import org.apache.flink.runtime.leaderelection.LeaderContender;
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
import org.apache.flink.runtime.rpc.RpcService;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.UUID;
import java.util.concurrent.Executor;

/**
* The runner for the job manager. It deals with job level leader election and make underlying job manager
Expand All @@ -52,11 +50,8 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {

private final OnCompletionActions toNotify;

/** The execution context which is used to execute futures */
private final Executor executionContext;

// TODO: use this to decide whether the job is finished by other
private final CheckpointRecoveryFactory checkpointRecoveryFactory;
/** Used to check whether a job needs to be run */
private final SubmittedJobGraphStore submittedJobGraphStore;

/** Leader election for this job */
private final LeaderElectionService leaderElectionService;
Expand Down Expand Up @@ -87,9 +82,8 @@ public JobManagerRunner(
{
this.jobGraph = jobGraph;
this.toNotify = toNotify;
this.executionContext = rpcService.getExecutor();
this.checkpointRecoveryFactory = haServices.getCheckpointRecoveryFactory();
this.leaderElectionService = haServices.getJobMasterLeaderElectionService(jobGraph.getJobID());
this.submittedJobGraphStore = haServices.getSubmittedJobGraphStore();
this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID());

this.jobManager = new JobMaster(
jobGraph, configuration, rpcService, haServices,
Expand Down Expand Up @@ -271,7 +265,7 @@ public void handleError(Exception exception) {

@VisibleForTesting
boolean isJobFinishedByOthers() {
// TODO
// TODO: Fix
return false;
}

Expand Down
Loading

0 comments on commit c2891c5

Please sign in to comment.