1
- /**
1
+ /*
2
2
*
3
3
* Licensed to the Apache Software Foundation (ASF) under one
4
4
* or more contributor license agreements. See the NOTICE file
17
17
* limitations under the License.
18
18
*/
19
19
package org .apache .hadoop .hbase .master ;
20
-
21
20
import java .io .IOException ;
21
+ import java .util .Optional ;
22
22
import java .util .concurrent .atomic .AtomicBoolean ;
23
-
24
- import org .apache .hadoop .hbase .zookeeper .MasterAddressTracker ;
25
- import org .apache .hadoop .hbase .zookeeper .ZKUtil ;
26
- import org .apache .hadoop .hbase .zookeeper .ZKWatcher ;
27
- import org .apache .hadoop .hbase .zookeeper .ZNodePaths ;
28
- import org .apache .yetus .audience .InterfaceAudience ;
29
23
import org .apache .hadoop .hbase .Server ;
30
24
import org .apache .hadoop .hbase .ServerName ;
31
25
import org .apache .hadoop .hbase .ZNodeClearer ;
32
26
import org .apache .hadoop .hbase .exceptions .DeserializationException ;
33
27
import org .apache .hadoop .hbase .monitoring .MonitoredTask ;
34
- import org .apache .hadoop .hbase .shaded . protobuf . ProtobufUtil ;
28
+ import org .apache .hadoop .hbase .zookeeper . MasterAddressTracker ;
35
29
import org .apache .hadoop .hbase .zookeeper .ZKListener ;
30
+ import org .apache .hadoop .hbase .zookeeper .ZKUtil ;
31
+ import org .apache .hadoop .hbase .zookeeper .ZKWatcher ;
32
+ import org .apache .hadoop .hbase .zookeeper .ZNodePaths ;
33
+ import org .apache .yetus .audience .InterfaceAudience ;
36
34
import org .apache .zookeeper .KeeperException ;
37
35
import org .slf4j .Logger ;
38
36
import org .slf4j .LoggerFactory ;
37
+ import org .apache .hadoop .hbase .shaded .protobuf .ProtobufUtil ;
39
38
40
39
/**
41
40
* Handles everything on master-side related to master election.
@@ -57,12 +56,18 @@ public class ActiveMasterManager extends ZKListener {
57
56
final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean (false );
58
57
final AtomicBoolean clusterShutDown = new AtomicBoolean (false );
59
58
59
+ // This server's information.
60
60
private final ServerName sn ;
61
61
private int infoPort ;
62
62
private final Server master ;
63
63
64
+ // Active master's server name. Invalidated anytime active master changes (based on ZK
65
+ // notifications) and lazily fetched on-demand.
66
+ // ServerName is immutable, so we don't need heavy synchronization around it.
67
+ private volatile ServerName activeMasterServerName ;
68
+
64
69
/**
65
- * @param watcher
70
+ * @param watcher ZK watcher
66
71
* @param sn ServerName
67
72
* @param master In an instance of a Master.
68
73
*/
@@ -106,6 +111,30 @@ void handle(final String path) {
106
111
}
107
112
}
108
113
114
+ /**
115
+ * Fetches the active master's ServerName from zookeeper.
116
+ */
117
+ private void fetchAndSetActiveMasterServerName () {
118
+ LOG .debug ("Attempting to fetch active master sn from zk" );
119
+ try {
120
+ activeMasterServerName = MasterAddressTracker .getMasterAddress (watcher );
121
+ } catch (IOException | KeeperException e ) {
122
+ // Log and ignore for now and re-fetch later if needed.
123
+ LOG .error ("Error fetching active master information" , e );
124
+ }
125
+ }
126
+
127
+ public Optional <ServerName > getActiveMasterServerName () {
128
+ if (!clusterHasActiveMaster .get ()) {
129
+ return Optional .empty ();
130
+ }
131
+ if (activeMasterServerName == null ) {
132
+ fetchAndSetActiveMasterServerName ();
133
+ }
134
+ // It could still be null, but return whatever we have.
135
+ return Optional .ofNullable (activeMasterServerName );
136
+ }
137
+
109
138
/**
110
139
* Handle a change in the master node. Doesn't matter whether this was called
111
140
* from a nodeCreated or nodeDeleted event because there are no guarantees
@@ -134,6 +163,9 @@ private void handleMasterNodeChange() {
134
163
// Notify any thread waiting to become the active master
135
164
clusterHasActiveMaster .notifyAll ();
136
165
}
166
+ // Reset the active master sn. Will be re-fetched later if needed.
167
+ // We don't want to make a synchronous RPC under a monitor.
168
+ activeMasterServerName = null ;
137
169
}
138
170
} catch (KeeperException ke ) {
139
171
master .abort ("Received an unexpected KeeperException, aborting" , ke );
@@ -151,8 +183,8 @@ private void handleMasterNodeChange() {
151
183
* @param checkInterval the interval to check if the master is stopped
152
184
* @param startupStatus the monitor status to track the progress
153
185
* @return True if no issue becoming active master else false if another
154
- * master was running or if some other problem (zookeeper, stop flag has been
155
- * set on this Master)
186
+ * master was running or if some other problem (zookeeper, stop flag has been
187
+ * set on this Master)
156
188
*/
157
189
boolean blockUntilBecomingActiveMaster (
158
190
int checkInterval , MonitoredTask startupStatus ) {
@@ -178,10 +210,14 @@ boolean blockUntilBecomingActiveMaster(
178
210
// We are the master, return
179
211
startupStatus .setStatus ("Successfully registered as active master." );
180
212
this .clusterHasActiveMaster .set (true );
213
+ activeMasterServerName = sn ;
181
214
LOG .info ("Registered as active master=" + this .sn );
182
215
return true ;
183
216
}
184
217
218
+ // Invalidate the active master name so that subsequent requests do not get any stale
219
+ // master information. Will be re-fetched if needed.
220
+ activeMasterServerName = null ;
185
221
// There is another active master running elsewhere or this is a restart
186
222
// and the master ephemeral node has not expired yet.
187
223
this .clusterHasActiveMaster .set (true );
@@ -208,7 +244,8 @@ boolean blockUntilBecomingActiveMaster(
208
244
ZKUtil .deleteNode (this .watcher , this .watcher .getZNodePaths ().masterAddressZNode );
209
245
210
246
// We may have failed to delete the znode at the previous step, but
211
- // we delete the file anyway: a second attempt to delete the znode is likely to fail again.
247
+ // we delete the file anyway: a second attempt to delete the znode is likely to fail
248
+ // again.
212
249
ZNodeClearer .deleteMyEphemeralNodeOnDisk ();
213
250
} else {
214
251
msg = "Another master is the active master, " + currentMaster +
0 commit comments