public abstract class AbstractHaServices extends Object implements HighAvailabilityServices
getLeaderPathForResourceManager()
, getLeaderPathForDispatcher()
, getLeaderPathForJobManager(org.apache.flink.api.common.JobID)
, getLeaderPathForRestServer()
. The returned leader name is the ConfigMap name in Kubernetes and
child path in Zookeeper.
close()
and cleanupAllData()
should be implemented to destroy the resources.
The abstract class is also responsible for determining which component service should be
reused. For example, jobResultStore
is created once and could be reused many times.
Modifier and Type | Field and Description |
---|---|
protected Configuration |
configuration
The runtime configuration.
|
protected Executor |
ioExecutor
The executor to run external IO operations on.
|
protected org.slf4j.Logger |
logger |
DEFAULT_JOB_ID, DEFAULT_LEADER_ID
Modifier | Constructor and Description |
---|---|
protected |
AbstractHaServices(Configuration config,
LeaderElectionDriverFactory driverFactory,
Executor ioExecutor,
BlobStoreService blobStoreService,
JobResultStore jobResultStore) |
Modifier and Type | Method and Description |
---|---|
void |
cleanupAllData()
Deletes all data stored by high availability services in external stores.
|
void |
close()
Closes the high availability services, releasing all resources.
|
BlobStore |
createBlobStore()
Creates the BLOB store in which BLOBs are stored in a highly-available fashion.
|
protected abstract CheckpointRecoveryFactory |
createCheckpointRecoveryFactory()
Create the checkpoint recovery factory for the job manager.
|
protected abstract JobGraphStore |
createJobGraphStore()
Create the submitted job graph store for the job manager.
|
protected abstract LeaderRetrievalService |
createLeaderRetrievalService(String leaderName)
Create leader retrieval service with specified leaderName.
|
CheckpointRecoveryFactory |
getCheckpointRecoveryFactory()
Gets the checkpoint recovery factory for the job manager.
|
LeaderElection |
getClusterRestEndpointLeaderElection()
Gets the
LeaderElection for the cluster's rest endpoint. |
LeaderRetrievalService |
getClusterRestEndpointLeaderRetriever()
Get the leader retriever for the cluster's rest endpoint.
|
LeaderElection |
getDispatcherLeaderElection()
Gets the
LeaderElection 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.
|
LeaderElection |
getJobManagerLeaderElection(JobID jobID)
Gets the
LeaderElection for the job with the given JobID . |
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.
|
JobResultStore |
getJobResultStore()
Gets the store that holds information about the state of finished jobs.
|
protected abstract String |
getLeaderPathForDispatcher()
Get the leader path for Dispatcher.
|
protected abstract String |
getLeaderPathForJobManager(JobID jobID)
Get the leader path for specific JobManager.
|
protected abstract String |
getLeaderPathForResourceManager()
Get the leader path for ResourceManager.
|
protected abstract String |
getLeaderPathForRestServer()
Get the leader path for RestServer.
|
LeaderElection |
getResourceManagerLeaderElection()
Gets the
LeaderElection for the cluster's resource manager. |
LeaderRetrievalService |
getResourceManagerLeaderRetriever()
Gets the leader retriever for the cluster's resource manager.
|
CompletableFuture<Void> |
globalCleanupAsync(JobID jobID,
Executor executor)
globalCleanupAsync is expected to be called from the main thread. |
protected abstract void |
internalCleanup()
Clean up the meta data in the distributed system(e.g.
|
protected abstract void |
internalCleanupJobData(JobID jobID)
Clean up the meta data in the distributed system(e.g.
|
protected abstract void |
internalClose()
Closes the components which is used for external operations(e.g.
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
closeWithOptionalClean, getWebMonitorLeaderElection, getWebMonitorLeaderRetriever
protected final org.slf4j.Logger logger
protected final Executor ioExecutor
protected final Configuration configuration
protected AbstractHaServices(Configuration config, LeaderElectionDriverFactory driverFactory, Executor ioExecutor, BlobStoreService blobStoreService, JobResultStore jobResultStore)
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 LeaderElection getResourceManagerLeaderElection()
HighAvailabilityServices
LeaderElection
for the cluster's resource manager.getResourceManagerLeaderElection
in interface HighAvailabilityServices
public LeaderElection getDispatcherLeaderElection()
HighAvailabilityServices
LeaderElection
for the cluster's dispatcher.getDispatcherLeaderElection
in interface HighAvailabilityServices
public LeaderElection getJobManagerLeaderElection(JobID jobID)
HighAvailabilityServices
LeaderElection
for the job with the given JobID
.getJobManagerLeaderElection
in interface HighAvailabilityServices
public LeaderElection getClusterRestEndpointLeaderElection()
HighAvailabilityServices
LeaderElection
for the cluster's rest endpoint.getClusterRestEndpointLeaderElection
in interface HighAvailabilityServices
public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception
HighAvailabilityServices
getCheckpointRecoveryFactory
in interface HighAvailabilityServices
Exception
public JobGraphStore getJobGraphStore() throws Exception
HighAvailabilityServices
getJobGraphStore
in interface HighAvailabilityServices
Exception
- if the submitted job graph store could not be createdpublic JobResultStore getJobResultStore() throws Exception
HighAvailabilityServices
getJobResultStore
in interface HighAvailabilityServices
Exception
- if job result store could not be createdpublic BlobStore createBlobStore()
HighAvailabilityServices
createBlobStore
in interface HighAvailabilityServices
public 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 cleanupAllData() throws Exception
HighAvailabilityServices
After this method was called, 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.
cleanupAllData
in interface HighAvailabilityServices
Exception
- if an error occurred while cleaning up data stored by them.public CompletableFuture<Void> globalCleanupAsync(JobID jobID, Executor executor)
GloballyCleanableResource
globalCleanupAsync
is expected to be called from the main thread. Heavy IO tasks
should be outsourced into the passed cleanupExecutor
. Thread-safety must be ensured.globalCleanupAsync
in interface GloballyCleanableResource
globalCleanupAsync
in interface HighAvailabilityServices
jobID
- The JobID
of the job for which the local data should be cleaned up.executor
- The fallback executor for IO-heavy operations.protected abstract LeaderRetrievalService createLeaderRetrievalService(String leaderName)
leaderName
- ConfigMap name in Kubernetes or child node path in Zookeeper.protected abstract CheckpointRecoveryFactory createCheckpointRecoveryFactory() throws Exception
Exception
protected abstract JobGraphStore createJobGraphStore() throws Exception
Exception
- if the submitted job graph store could not be createdprotected abstract void internalClose() throws Exception
Exception
- if the close operation failedprotected abstract void internalCleanup() throws Exception
If an exception occurs during internal cleanup, we will continue the cleanup in cleanupAllData()
and report exceptions only after all cleanup steps have been attempted.
Exception
- when do the cleanup operation on external storage.protected abstract void internalCleanupJobData(JobID jobID) throws Exception
jobID
- The identifier of the job to cleanup.Exception
- when do the cleanup operation on external storage.protected abstract String getLeaderPathForResourceManager()
protected abstract String getLeaderPathForDispatcher()
protected abstract String getLeaderPathForJobManager(JobID jobID)
jobID
- job idprotected abstract String getLeaderPathForRestServer()
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.