public class ZooKeeperHaServices extends Object implements HighAvailabilityServices
HighAvailabilityServices
using Apache ZooKeeper.
The services store data in ZooKeeper's nodes as illustrated by the following tree structure:
/flink +/cluster_id_1/resource_manager_lock | | | +/job-id-1/job_manager_lock | | /checkpoints/latest | | /latest-1 | | /latest-2 | | | +/job-id-2/job_manager_lock | +/cluster_id_2/resource_manager_lock | +/job-id-1/job_manager_lock |/checkpoints/latest | /latest-1 |/persisted_job_graph
The root path "/flink" is configurable via the option HighAvailabilityOptions.HA_ZOOKEEPER_ROOT
.
This makes sure Flink stores its data under specific subtrees in ZooKeeper, for example to
accommodate specific permission.
The "cluster_id" part identifies the data stored for a specific Flink "cluster". This "cluster" can be either a standalone or containerized Flink cluster, or it can be job on a framework like YARN or Mesos (in a "per-job-cluster" mode).
In case of a "per-job-cluster" on YARN or Mesos, the cluster-id is generated and configured automatically by the client or dispatcher that submits the Job to YARN or Mesos.
In the case of a standalone cluster, that cluster-id needs to be configured via
HighAvailabilityOptions.HA_CLUSTER_ID
. All nodes with the same cluster id will join the same
cluster and participate in the execution of the same set of jobs.
DEFAULT_JOB_ID, DEFAULT_LEADER_ID
Constructor and Description |
---|
ZooKeeperHaServices(org.apache.curator.framework.CuratorFramework client,
Executor executor,
Configuration configuration,
BlobStoreService blobStoreService) |
Modifier and Type | Method and Description |
---|---|
void |
close()
Closes the high availability services, releasing all resources.
|
void |
closeAndCleanupAllData()
Closes the high availability services (releasing all resources) and deletes
all data stored by these services in external stores.
|
BlobStore |
createBlobStore()
Creates the BLOB store in which BLOBs are stored in a highly-available fashion.
|
CheckpointRecoveryFactory |
getCheckpointRecoveryFactory()
Gets the checkpoint recovery factory for the job manager.
|
LeaderElectionService |
getClusterRestEndpointLeaderElectionService()
Gets the leader election service for the cluster's rest endpoint.
|
LeaderRetrievalService |
getClusterRestEndpointLeaderRetriever()
Get the leader retriever for the cluster's rest endpoint.
|
LeaderElectionService |
getDispatcherLeaderElectionService()
Gets the leader election service for the cluster's dispatcher.
|
LeaderRetrievalService |
getDispatcherLeaderRetriever()
Gets the leader retriever for the dispatcher.
|
JobGraphStore |
getJobGraphStore()
Gets the submitted job graph store for the job manager.
|
LeaderElectionService |
getJobManagerLeaderElectionService(JobID jobID)
Gets the leader election service for the given job.
|
LeaderRetrievalService |
getJobManagerLeaderRetriever(JobID jobID)
Gets the leader retriever for the job JobMaster which is responsible for the given job.
|
LeaderRetrievalService |
getJobManagerLeaderRetriever(JobID jobID,
String defaultJobManagerAddress)
Gets the leader retriever for the job JobMaster which is responsible for the given job.
|
LeaderElectionService |
getResourceManagerLeaderElectionService()
Gets the leader election service for the cluster's resource manager.
|
LeaderRetrievalService |
getResourceManagerLeaderRetriever()
Gets the leader retriever for the cluster's resource manager.
|
RunningJobsRegistry |
getRunningJobsRegistry()
Gets the registry that holds information about whether jobs are currently running.
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
getWebMonitorLeaderElectionService, getWebMonitorLeaderRetriever
public ZooKeeperHaServices(org.apache.curator.framework.CuratorFramework client, Executor executor, Configuration configuration, BlobStoreService blobStoreService)
public LeaderRetrievalService getResourceManagerLeaderRetriever()
HighAvailabilityServices
getResourceManagerLeaderRetriever
in interface HighAvailabilityServices
public LeaderRetrievalService getDispatcherLeaderRetriever()
HighAvailabilityServices
getDispatcherLeaderRetriever
in interface HighAvailabilityServices
public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID)
HighAvailabilityServices
getJobManagerLeaderRetriever
in interface HighAvailabilityServices
jobID
- The identifier of the job.public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress)
HighAvailabilityServices
getJobManagerLeaderRetriever
in interface HighAvailabilityServices
jobID
- The identifier of the job.defaultJobManagerAddress
- JobManager address which will be returned by
a static leader retrieval service.public LeaderRetrievalService getClusterRestEndpointLeaderRetriever()
ClientHighAvailabilityServices
getClusterRestEndpointLeaderRetriever
in interface ClientHighAvailabilityServices
getClusterRestEndpointLeaderRetriever
in interface HighAvailabilityServices
public LeaderElectionService getResourceManagerLeaderElectionService()
HighAvailabilityServices
getResourceManagerLeaderElectionService
in interface HighAvailabilityServices
public LeaderElectionService getDispatcherLeaderElectionService()
HighAvailabilityServices
getDispatcherLeaderElectionService
in interface HighAvailabilityServices
public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID)
HighAvailabilityServices
getJobManagerLeaderElectionService
in interface HighAvailabilityServices
jobID
- The identifier of the job running the election.public LeaderElectionService getClusterRestEndpointLeaderElectionService()
HighAvailabilityServices
getClusterRestEndpointLeaderElectionService
in interface HighAvailabilityServices
public CheckpointRecoveryFactory getCheckpointRecoveryFactory()
HighAvailabilityServices
getCheckpointRecoveryFactory
in interface HighAvailabilityServices
public JobGraphStore getJobGraphStore() throws Exception
HighAvailabilityServices
getJobGraphStore
in interface HighAvailabilityServices
Exception
- if the submitted job graph store could not be createdpublic RunningJobsRegistry getRunningJobsRegistry()
HighAvailabilityServices
getRunningJobsRegistry
in interface HighAvailabilityServices
public BlobStore createBlobStore() throws IOException
HighAvailabilityServices
createBlobStore
in interface HighAvailabilityServices
IOException
- if the blob store could not be createdpublic void close() throws Exception
HighAvailabilityServices
This method does not delete or clean up any data stored in external stores (file systems, ZooKeeper, etc). Another instance of the high availability services will be able to recover the job.
If an exception occurs during closing services, this method will attempt to continue closing other services and report exceptions only after all services have been attempted to be closed.
close
in interface AutoCloseable
close
in interface HighAvailabilityServices
Exception
- Thrown, if an exception occurred while closing these services.public void closeAndCleanupAllData() throws Exception
HighAvailabilityServices
After this method was called, the any job or session that was managed by these high availability services will be unrecoverable.
If an exception occurs during cleanup, this method will attempt to continue the cleanup and report exceptions only after all cleanup steps have been attempted.
closeAndCleanupAllData
in interface HighAvailabilityServices
Exception
- Thrown, if an exception occurred while closing these services
or cleaning up data stored by them.Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.