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 ;
22
21
import java .util .concurrent .atomic .AtomicBoolean ;
23
22
@@ -57,12 +56,18 @@ public class ActiveMasterManager extends ZooKeeperListener {
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,33 @@ 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
+ /**
128
+ * @return the currently active master as seen by us or null if one does not exist.
129
+ */
130
+ public ServerName getActiveMasterServerName () {
131
+ if (!clusterHasActiveMaster .get ()) {
132
+ return null ;
133
+ }
134
+ if (activeMasterServerName == null ) {
135
+ fetchAndSetActiveMasterServerName ();
136
+ }
137
+ // It could still be null, but return whatever we have.
138
+ return activeMasterServerName ;
139
+ }
140
+
109
141
/**
110
142
* Handle a change in the master node. Doesn't matter whether this was called
111
143
* from a nodeCreated or nodeDeleted event because there are no guarantees
@@ -134,6 +166,9 @@ private void handleMasterNodeChange() {
134
166
// Notify any thread waiting to become the active master
135
167
clusterHasActiveMaster .notifyAll ();
136
168
}
169
+ // Reset the active master sn. Will be re-fetched later if needed.
170
+ // We don't want to make a synchronous RPC under a monitor.
171
+ activeMasterServerName = null ;
137
172
}
138
173
} catch (KeeperException ke ) {
139
174
master .abort ("Received an unexpected KeeperException, aborting" , ke );
@@ -151,8 +186,8 @@ private void handleMasterNodeChange() {
151
186
* @param checkInterval the interval to check if the master is stopped
152
187
* @param startupStatus the monitor status to track the progress
153
188
* @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)
189
+ * master was running or if some other problem (zookeeper, stop flag has been
190
+ * set on this Master)
156
191
*/
157
192
boolean blockUntilBecomingActiveMaster (
158
193
int checkInterval , MonitoredTask startupStatus ) {
@@ -179,9 +214,13 @@ boolean blockUntilBecomingActiveMaster(
179
214
startupStatus .setStatus ("Successfully registered as active master." );
180
215
this .clusterHasActiveMaster .set (true );
181
216
LOG .info ("Registered Active Master=" + this .sn );
217
+ activeMasterServerName = sn ;
182
218
return true ;
183
219
}
184
220
221
+ // Invalidate the active master name so that subsequent requests do not get any stale
222
+ // master information. Will be re-fetched if needed.
223
+ activeMasterServerName = null ;
185
224
// There is another active master running elsewhere or this is a restart
186
225
// and the master ephemeral node has not expired yet.
187
226
this .clusterHasActiveMaster .set (true );
@@ -208,7 +247,8 @@ boolean blockUntilBecomingActiveMaster(
208
247
ZKUtil .deleteNode (this .watcher , this .watcher .getMasterAddressZNode ());
209
248
210
249
// 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.
250
+ // we delete the file anyway: a second attempt to delete the znode is likely to fail
251
+ // again.
212
252
ZNodeClearer .deleteMyEphemeralNodeOnDisk ();
213
253
} else {
214
254
msg = "Another master is the active master, " + currentMaster +
0 commit comments