001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hdfs;
019
020import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
021import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY;
022import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
023import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
024import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HA_DT_SERVICE_PREFIX;
025
026import java.io.IOException;
027import java.net.InetSocketAddress;
028import java.net.URI;
029import java.util.ArrayList;
030import java.util.Collection;
031import java.util.List;
032import java.util.Map;
033
034import org.apache.commons.logging.Log;
035import org.apache.commons.logging.LogFactory;
036import org.apache.hadoop.HadoopIllegalArgumentException;
037import org.apache.hadoop.conf.Configuration;
038import org.apache.hadoop.fs.FileSystem;
039import org.apache.hadoop.fs.Path;
040import org.apache.hadoop.hdfs.NameNodeProxies;
041import org.apache.hadoop.hdfs.NameNodeProxies.ProxyAndInfo;
042import org.apache.hadoop.hdfs.protocol.ClientProtocol;
043import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
044import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
045import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
046import org.apache.hadoop.hdfs.server.namenode.NameNode;
047import org.apache.hadoop.io.Text;
048import org.apache.hadoop.ipc.RPC;
049import org.apache.hadoop.ipc.RemoteException;
050import org.apache.hadoop.ipc.StandbyException;
051import org.apache.hadoop.security.SecurityUtil;
052import org.apache.hadoop.security.UserGroupInformation;
053import org.apache.hadoop.security.token.Token;
054
055import com.google.common.base.Joiner;
056import com.google.common.base.Preconditions;
057import com.google.common.collect.Lists;
058
059public class HAUtil {
060  
061  private static final Log LOG = 
062    LogFactory.getLog(HAUtil.class);
063  
064  private static final DelegationTokenSelector tokenSelector =
065      new DelegationTokenSelector();
066
067  private HAUtil() { /* Hidden constructor */ }
068
069  /**
070   * Returns true if HA for namenode is configured for the given nameservice
071   * 
072   * @param conf Configuration
073   * @param nsId nameservice, or null if no federated NS is configured
074   * @return true if HA is configured in the configuration; else false.
075   */
076  public static boolean isHAEnabled(Configuration conf, String nsId) {
077    Map<String, Map<String, InetSocketAddress>> addresses =
078      DFSUtil.getHaNnRpcAddresses(conf);
079    if (addresses == null) return false;
080    Map<String, InetSocketAddress> nnMap = addresses.get(nsId);
081    return nnMap != null && nnMap.size() > 1;
082  }
083
084  /**
085   * Returns true if HA is using a shared edits directory.
086   *
087   * @param conf Configuration
088   * @return true if HA config is using a shared edits dir, false otherwise.
089   */
090  public static boolean usesSharedEditsDir(Configuration conf) {
091    return null != conf.get(DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
092  }
093
094  /**
095   * Get the namenode Id by matching the {@code addressKey}
096   * with the the address of the local node.
097   * 
098   * If {@link DFSConfigKeys#DFS_HA_NAMENODE_ID_KEY} is not specifically
099   * configured, this method determines the namenode Id by matching the local
100   * node's address with the configured addresses. When a match is found, it
101   * returns the namenode Id from the corresponding configuration key.
102   * 
103   * @param conf Configuration
104   * @return namenode Id on success, null on failure.
105   * @throws HadoopIllegalArgumentException on error
106   */
107  public static String getNameNodeId(Configuration conf, String nsId) {
108    String namenodeId = conf.getTrimmed(DFS_HA_NAMENODE_ID_KEY);
109    if (namenodeId != null) {
110      return namenodeId;
111    }
112    
113    String suffixes[] = DFSUtil.getSuffixIDs(conf, DFS_NAMENODE_RPC_ADDRESS_KEY,
114        nsId, null, DFSUtil.LOCAL_ADDRESS_MATCHER);
115    if (suffixes == null) {
116      String msg = "Configuration " + DFS_NAMENODE_RPC_ADDRESS_KEY + 
117          " must be suffixed with nameservice and namenode ID for HA " +
118          "configuration.";
119      throw new HadoopIllegalArgumentException(msg);
120    }
121    
122    return suffixes[1];
123  }
124
125  /**
126   * Similar to
127   * {@link DFSUtil#getNameServiceIdFromAddress(Configuration, 
128   * InetSocketAddress, String...)}
129   */
130  public static String getNameNodeIdFromAddress(final Configuration conf, 
131      final InetSocketAddress address, String... keys) {
132    // Configuration with a single namenode and no nameserviceId
133    String[] ids = DFSUtil.getSuffixIDs(conf, address, keys);
134    if (ids != null && ids.length > 1) {
135      return ids[1];
136    }
137    return null;
138  }
139  
140  /**
141   * Get the NN ID of the other node in an HA setup.
142   * 
143   * @param conf the configuration of this node
144   * @return the NN ID of the other node in this nameservice
145   */
146  public static String getNameNodeIdOfOtherNode(Configuration conf, String nsId) {
147    Preconditions.checkArgument(nsId != null,
148        "Could not determine namespace id. Please ensure that this " +
149        "machine is one of the machines listed as a NN RPC address, " +
150        "or configure " + DFSConfigKeys.DFS_NAMESERVICE_ID);
151    
152    Collection<String> nnIds = DFSUtil.getNameNodeIds(conf, nsId);
153    String myNNId = conf.get(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY);
154    Preconditions.checkArgument(nnIds != null,
155        "Could not determine namenode ids in namespace '%s'. " +
156        "Please configure " +
157        DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
158            nsId),
159        nsId);
160    Preconditions.checkArgument(nnIds.size() == 2,
161        "Expected exactly 2 NameNodes in namespace '%s'. " +
162        "Instead, got only %s (NN ids were '%s'",
163        nsId, nnIds.size(), Joiner.on("','").join(nnIds));
164    Preconditions.checkState(myNNId != null && !myNNId.isEmpty(),
165        "Could not determine own NN ID in namespace '%s'. Please " +
166        "ensure that this node is one of the machines listed as an " +
167        "NN RPC address, or configure " + DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY,
168        nsId);
169
170    ArrayList<String> nnSet = Lists.newArrayList(nnIds);
171    nnSet.remove(myNNId);
172    assert nnSet.size() == 1;
173    return nnSet.get(0);
174  }
175
176  /**
177   * Given the configuration for this node, return a Configuration object for
178   * the other node in an HA setup.
179   * 
180   * @param myConf the configuration of this node
181   * @return the configuration of the other node in an HA setup
182   */
183  public static Configuration getConfForOtherNode(
184      Configuration myConf) {
185    
186    String nsId = DFSUtil.getNamenodeNameServiceId(myConf);
187    String otherNn = getNameNodeIdOfOtherNode(myConf, nsId);
188    
189    // Look up the address of the active NN.
190    Configuration confForOtherNode = new Configuration(myConf);
191    NameNode.initializeGenericKeys(confForOtherNode, nsId, otherNn);
192    return confForOtherNode;
193  }
194
195  /**
196   * This is used only by tests at the moment.
197   * @return true if the NN should allow read operations while in standby mode.
198   */
199  public static boolean shouldAllowStandbyReads(Configuration conf) {
200    return conf.getBoolean("dfs.ha.allow.stale.reads", false);
201  }
202  
203  public static void setAllowStandbyReads(Configuration conf, boolean val) {
204    conf.setBoolean("dfs.ha.allow.stale.reads", val);
205  }
206 
207  /**
208   * @return true if the given nameNodeUri appears to be a logical URI.
209   */
210  public static boolean isLogicalUri(
211      Configuration conf, URI nameNodeUri) {
212    String host = nameNodeUri.getHost();
213    // A logical name must be one of the service IDs.
214    return DFSUtil.getNameServiceIds(conf).contains(host);
215  }
216
217  /**
218   * Check whether the client has a failover proxy provider configured
219   * for the namenode/nameservice.
220   *
221   * @param conf Configuration
222   * @param nameNodeUri The URI of namenode
223   * @return true if failover is configured.
224   */
225  public static boolean isClientFailoverConfigured(
226      Configuration conf, URI nameNodeUri) {
227    String host = nameNodeUri.getHost();
228    String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
229        + host;
230    return conf.get(configKey) != null;
231  }
232
233  /**
234   * Check whether logical URI is needed for the namenode and
235   * the corresponding failover proxy provider in the config.
236   *
237   * @param conf Configuration
238   * @param nameNodeUri The URI of namenode
239   * @return true if logical URI is needed. false, if not needed.
240   * @throws IOException most likely due to misconfiguration.
241   */
242  public static boolean useLogicalUri(Configuration conf, URI nameNodeUri) 
243      throws IOException {
244    // Create the proxy provider. Actual proxy is not created.
245    AbstractNNFailoverProxyProvider<ClientProtocol> provider = NameNodeProxies
246        .createFailoverProxyProvider(conf, nameNodeUri, ClientProtocol.class,
247        false);
248
249    // No need to use logical URI since failover is not configured.
250    if (provider == null) {
251      return false;
252    }
253    // Check whether the failover proxy provider uses logical URI.
254    return provider.useLogicalURI();
255  }
256
257  /**
258   * Parse the file system URI out of the provided token.
259   */
260  public static URI getServiceUriFromToken(final String scheme,
261                                           Token<?> token) {
262    String tokStr = token.getService().toString();
263
264    if (tokStr.startsWith(HA_DT_SERVICE_PREFIX)) {
265      tokStr = tokStr.replaceFirst(HA_DT_SERVICE_PREFIX, "");
266    }
267    return URI.create(scheme + "://" + tokStr);
268  }
269  
270  /**
271   * Get the service name used in the delegation token for the given logical
272   * HA service.
273   * @param uri the logical URI of the cluster
274   * @return the service name
275   */
276  public static Text buildTokenServiceForLogicalUri(URI uri) {
277    return new Text(HA_DT_SERVICE_PREFIX + uri.getHost());
278  }
279  
280  /**
281   * @return true if this token corresponds to a logical nameservice
282   * rather than a specific namenode.
283   */
284  public static boolean isTokenForLogicalUri(Token<?> token) {
285    return token.getService().toString().startsWith(HA_DT_SERVICE_PREFIX);
286  }
287  
288  /**
289   * Locate a delegation token associated with the given HA cluster URI, and if
290   * one is found, clone it to also represent the underlying namenode address.
291   * @param ugi the UGI to modify
292   * @param haUri the logical URI for the cluster
293   * @param nnAddrs collection of NNs in the cluster to which the token
294   * applies
295   */
296  public static void cloneDelegationTokenForLogicalUri(
297      UserGroupInformation ugi, URI haUri,
298      Collection<InetSocketAddress> nnAddrs) {
299    Text haService = HAUtil.buildTokenServiceForLogicalUri(haUri);
300    Token<DelegationTokenIdentifier> haToken =
301        tokenSelector.selectToken(haService, ugi.getTokens());
302    if (haToken != null) {
303      for (InetSocketAddress singleNNAddr : nnAddrs) {
304        // this is a minor hack to prevent physical HA tokens from being
305        // exposed to the user via UGI.getCredentials(), otherwise these
306        // cloned tokens may be inadvertently propagated to jobs
307        Token<DelegationTokenIdentifier> specificToken =
308            new Token.PrivateToken<DelegationTokenIdentifier>(haToken);
309        SecurityUtil.setTokenService(specificToken, singleNNAddr);
310        Text alias =
311            new Text(HA_DT_SERVICE_PREFIX + "//" + specificToken.getService());
312        ugi.addToken(alias, specificToken);
313        LOG.debug("Mapped HA service delegation token for logical URI " +
314            haUri + " to namenode " + singleNNAddr);
315      }
316    } else {
317      LOG.debug("No HA service delegation token found for logical URI " +
318          haUri);
319    }
320  }
321
322  /**
323   * Get the internet address of the currently-active NN. This should rarely be
324   * used, since callers of this method who connect directly to the NN using the
325   * resulting InetSocketAddress will not be able to connect to the active NN if
326   * a failover were to occur after this method has been called.
327   * 
328   * @param fs the file system to get the active address of.
329   * @return the internet address of the currently-active NN.
330   * @throws IOException if an error occurs while resolving the active NN.
331   */
332  public static InetSocketAddress getAddressOfActive(FileSystem fs)
333      throws IOException {
334    if (!(fs instanceof DistributedFileSystem)) {
335      throw new IllegalArgumentException("FileSystem " + fs + " is not a DFS.");
336    }
337    // force client address resolution.
338    fs.exists(new Path("/"));
339    DistributedFileSystem dfs = (DistributedFileSystem) fs;
340    DFSClient dfsClient = dfs.getClient();
341    return RPC.getServerAddress(dfsClient.getNamenode());
342  }
343  
344  /**
345   * Get an RPC proxy for each NN in an HA nameservice. Used when a given RPC
346   * call should be made on every NN in an HA nameservice, not just the active.
347   * 
348   * @param conf configuration
349   * @param nsId the nameservice to get all of the proxies for.
350   * @return a list of RPC proxies for each NN in the nameservice.
351   * @throws IOException in the event of error.
352   */
353  public static List<ClientProtocol> getProxiesForAllNameNodesInNameservice(
354      Configuration conf, String nsId) throws IOException {
355    List<ProxyAndInfo<ClientProtocol>> proxies =
356        getProxiesForAllNameNodesInNameservice(conf, nsId, ClientProtocol.class);
357
358    List<ClientProtocol> namenodes = new ArrayList<ClientProtocol>(
359        proxies.size());
360    for (ProxyAndInfo<ClientProtocol> proxy : proxies) {
361      namenodes.add(proxy.getProxy());
362    }
363    return namenodes;
364  }
365
366  /**
367   * Get an RPC proxy for each NN in an HA nameservice. Used when a given RPC
368   * call should be made on every NN in an HA nameservice, not just the active.
369   *
370   * @param conf configuration
371   * @param nsId the nameservice to get all of the proxies for.
372   * @param xface the protocol class.
373   * @return a list of RPC proxies for each NN in the nameservice.
374   * @throws IOException in the event of error.
375   */
376  public static <T> List<ProxyAndInfo<T>> getProxiesForAllNameNodesInNameservice(
377      Configuration conf, String nsId, Class<T> xface) throws IOException {
378    Map<String, InetSocketAddress> nnAddresses =
379        DFSUtil.getRpcAddressesForNameserviceId(conf, nsId, null);
380    
381    List<ProxyAndInfo<T>> proxies = new ArrayList<ProxyAndInfo<T>>(
382        nnAddresses.size());
383    for (InetSocketAddress nnAddress : nnAddresses.values()) {
384      NameNodeProxies.ProxyAndInfo<T> proxyInfo = null;
385      proxyInfo = NameNodeProxies.createNonHAProxy(conf,
386          nnAddress, xface,
387          UserGroupInformation.getCurrentUser(), false);
388      proxies.add(proxyInfo);
389    }
390    return proxies;
391  }
392  
393  /**
394   * Used to ensure that at least one of the given HA NNs is currently in the
395   * active state..
396   * 
397   * @param namenodes list of RPC proxies for each NN to check.
398   * @return true if at least one NN is active, false if all are in the standby state.
399   * @throws IOException in the event of error.
400   */
401  public static boolean isAtLeastOneActive(List<ClientProtocol> namenodes)
402      throws IOException {
403    for (ClientProtocol namenode : namenodes) {
404      try {
405        namenode.getFileInfo("/");
406        return true;
407      } catch (RemoteException re) {
408        IOException cause = re.unwrapRemoteException();
409        if (cause instanceof StandbyException) {
410          // This is expected to happen for a standby NN.
411        } else {
412          throw re;
413        }
414      }
415    }
416    return false;
417  }
418}