-
Notifications
You must be signed in to change notification settings - Fork 13k
/
NonHaServices.java
118 lines (100 loc) · 4.43 KB
/
NonHaServices.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
/*
* 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;
import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.blob.BlobStore;
import org.apache.flink.runtime.blob.VoidBlobStore;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
import org.apache.flink.runtime.highavailability.nonha.NonHaRegistry;
import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore;
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* An implementation of the {@link HighAvailabilityServices} for the non-high-availability case.
* This implementation can be used for testing, and for cluster setups that do not
* tolerate failures of the master processes (JobManager, ResourceManager).
*
* <p>This implementation has no dependencies on any external services. It returns fix
* pre-configured leaders, and stores checkpoints and metadata simply on the heap and therefore
* in volatile memory.
*/
public class NonHaServices implements HighAvailabilityServices {
/** The fix address of the ResourceManager */
private final String resourceManagerAddress;
private final ConcurrentHashMap<JobID, String> jobMastersAddress;
/**
* Creates a new services class for the fix pre-defined leaders.
*
* @param resourceManagerAddress The fix address of the ResourceManager
*/
public NonHaServices(String resourceManagerAddress) {
this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
this.jobMastersAddress = new ConcurrentHashMap<>(16);
}
/**
* Binds address of a specified job master
*
* @param jobID JobID for the specified job master
* @param jobMasterAddress address for the specified job master
*/
public void bindJobMasterLeaderAddress(JobID jobID, String jobMasterAddress) {
jobMastersAddress.put(jobID, jobMasterAddress);
}
// ------------------------------------------------------------------------
// Services
// ------------------------------------------------------------------------
@Override
public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0));
}
@Override
public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
return new StandaloneLeaderRetrievalService(jobMastersAddress.get(jobID), new UUID(0, 0));
}
@Override
public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
return new StandaloneLeaderElectionService();
}
@Override
public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
return new StandaloneLeaderElectionService();
}
@Override
public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
return new StandaloneCheckpointRecoveryFactory();
}
@Override
public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
return new StandaloneSubmittedJobGraphStore();
}
@Override
public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
return new NonHaRegistry();
}
@Override
public BlobStore createBlobStore() {
return new VoidBlobStore();
}
}