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 */ 018 package org.apache.hadoop.hdfs; 019 020 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT; 021 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY; 022 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX; 023 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT; 024 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY; 025 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT; 026 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY; 027 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_RETRY_MAX_ATTEMPTS_KEY; 028 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_RETRY_MAX_ATTEMPTS_DEFAULT; 029 030 import java.io.IOException; 031 import java.lang.reflect.Constructor; 032 import java.lang.reflect.InvocationHandler; 033 import java.lang.reflect.Proxy; 034 import java.net.InetSocketAddress; 035 import java.net.URI; 036 import java.util.HashMap; 037 import java.util.Map; 038 import java.util.concurrent.TimeUnit; 039 040 import org.apache.commons.logging.Log; 041 import org.apache.commons.logging.LogFactory; 042 import org.apache.hadoop.conf.Configuration; 043 import org.apache.hadoop.hdfs.DFSClient.Conf; 044 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; 045 import org.apache.hadoop.hdfs.protocol.ClientProtocol; 046 import org.apache.hadoop.hdfs.protocol.HdfsConstants; 047 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB; 048 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB; 049 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB; 050 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB; 051 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB; 052 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB; 053 import org.apache.hadoop.hdfs.server.namenode.NameNode; 054 import org.apache.hadoop.hdfs.server.namenode.SafeModeException; 055 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol; 056 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; 057 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; 058 import org.apache.hadoop.io.Text; 059 import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider; 060 import org.apache.hadoop.io.retry.FailoverProxyProvider; 061 import org.apache.hadoop.io.retry.LossyRetryInvocationHandler; 062 import org.apache.hadoop.io.retry.RetryPolicies; 063 import org.apache.hadoop.io.retry.RetryPolicy; 064 import org.apache.hadoop.io.retry.RetryProxy; 065 import org.apache.hadoop.io.retry.RetryUtils; 066 import org.apache.hadoop.ipc.ProtobufRpcEngine; 067 import org.apache.hadoop.ipc.RPC; 068 import org.apache.hadoop.ipc.RemoteException; 069 import org.apache.hadoop.net.NetUtils; 070 import org.apache.hadoop.security.RefreshUserMappingsProtocol; 071 import org.apache.hadoop.security.SecurityUtil; 072 import org.apache.hadoop.security.UserGroupInformation; 073 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol; 074 import org.apache.hadoop.security.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB; 075 import org.apache.hadoop.security.protocolPB.RefreshAuthorizationPolicyProtocolPB; 076 import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB; 077 import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolPB; 078 import org.apache.hadoop.tools.GetUserMappingsProtocol; 079 import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB; 080 import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB; 081 082 import com.google.common.annotations.VisibleForTesting; 083 import com.google.common.base.Preconditions; 084 085 /** 086 * Create proxy objects to communicate with a remote NN. All remote access to an 087 * NN should be funneled through this class. Most of the time you'll want to use 088 * {@link NameNodeProxies#createProxy(Configuration, URI, Class)}, which will 089 * create either an HA- or non-HA-enabled client proxy as appropriate. 090 */ 091 public class NameNodeProxies { 092 093 private static final Log LOG = LogFactory.getLog(NameNodeProxies.class); 094 095 /** 096 * Wrapper for a client proxy as well as its associated service ID. 097 * This is simply used as a tuple-like return type for 098 * {@link NameNodeProxies#createProxy} and 099 * {@link NameNodeProxies#createNonHAProxy}. 100 */ 101 public static class ProxyAndInfo<PROXYTYPE> { 102 private final PROXYTYPE proxy; 103 private final Text dtService; 104 105 public ProxyAndInfo(PROXYTYPE proxy, Text dtService) { 106 this.proxy = proxy; 107 this.dtService = dtService; 108 } 109 110 public PROXYTYPE getProxy() { 111 return proxy; 112 } 113 114 public Text getDelegationTokenService() { 115 return dtService; 116 } 117 } 118 119 /** 120 * Creates the namenode proxy with the passed protocol. This will handle 121 * creation of either HA- or non-HA-enabled proxy objects, depending upon 122 * if the provided URI is a configured logical URI. 123 * 124 * @param conf the configuration containing the required IPC 125 * properties, client failover configurations, etc. 126 * @param nameNodeUri the URI pointing either to a specific NameNode 127 * or to a logical nameservice. 128 * @param xface the IPC interface which should be created 129 * @return an object containing both the proxy and the associated 130 * delegation token service it corresponds to 131 * @throws IOException if there is an error creating the proxy 132 **/ 133 @SuppressWarnings("unchecked") 134 public static <T> ProxyAndInfo<T> createProxy(Configuration conf, 135 URI nameNodeUri, Class<T> xface) throws IOException { 136 Class<FailoverProxyProvider<T>> failoverProxyProviderClass = 137 getFailoverProxyProviderClass(conf, nameNodeUri, xface); 138 139 if (failoverProxyProviderClass == null) { 140 // Non-HA case 141 return createNonHAProxy(conf, NameNode.getAddress(nameNodeUri), xface, 142 UserGroupInformation.getCurrentUser(), true); 143 } else { 144 // HA case 145 FailoverProxyProvider<T> failoverProxyProvider = NameNodeProxies 146 .createFailoverProxyProvider(conf, failoverProxyProviderClass, xface, 147 nameNodeUri); 148 Conf config = new Conf(conf); 149 T proxy = (T) RetryProxy.create(xface, failoverProxyProvider, 150 RetryPolicies.failoverOnNetworkException( 151 RetryPolicies.TRY_ONCE_THEN_FAIL, config.maxFailoverAttempts, 152 config.maxRetryAttempts, config.failoverSleepBaseMillis, 153 config.failoverSleepMaxMillis)); 154 155 Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri); 156 return new ProxyAndInfo<T>(proxy, dtService); 157 } 158 } 159 160 /** 161 * Generate a dummy namenode proxy instance that utilizes our hacked 162 * {@link LossyRetryInvocationHandler}. Proxy instance generated using this 163 * method will proactively drop RPC responses. Currently this method only 164 * support HA setup. null will be returned if the given configuration is not 165 * for HA. 166 * 167 * @param config the configuration containing the required IPC 168 * properties, client failover configurations, etc. 169 * @param nameNodeUri the URI pointing either to a specific NameNode 170 * or to a logical nameservice. 171 * @param xface the IPC interface which should be created 172 * @param numResponseToDrop The number of responses to drop for each RPC call 173 * @return an object containing both the proxy and the associated 174 * delegation token service it corresponds to. Will return null of the 175 * given configuration does not support HA. 176 * @throws IOException if there is an error creating the proxy 177 */ 178 @SuppressWarnings("unchecked") 179 public static <T> ProxyAndInfo<T> createProxyWithLossyRetryHandler( 180 Configuration config, URI nameNodeUri, Class<T> xface, 181 int numResponseToDrop) throws IOException { 182 Preconditions.checkArgument(numResponseToDrop > 0); 183 Class<FailoverProxyProvider<T>> failoverProxyProviderClass = 184 getFailoverProxyProviderClass(config, nameNodeUri, xface); 185 if (failoverProxyProviderClass != null) { // HA case 186 FailoverProxyProvider<T> failoverProxyProvider = 187 createFailoverProxyProvider(config, failoverProxyProviderClass, 188 xface, nameNodeUri); 189 int delay = config.getInt( 190 DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY, 191 DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT); 192 int maxCap = config.getInt( 193 DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY, 194 DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT); 195 int maxFailoverAttempts = config.getInt( 196 DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY, 197 DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT); 198 int maxRetryAttempts = config.getInt( 199 DFS_CLIENT_RETRY_MAX_ATTEMPTS_KEY, 200 DFS_CLIENT_RETRY_MAX_ATTEMPTS_DEFAULT); 201 InvocationHandler dummyHandler = new LossyRetryInvocationHandler<T>( 202 numResponseToDrop, failoverProxyProvider, 203 RetryPolicies.failoverOnNetworkException( 204 RetryPolicies.TRY_ONCE_THEN_FAIL, maxFailoverAttempts, 205 Math.max(numResponseToDrop + 1, maxRetryAttempts), delay, 206 maxCap)); 207 208 T proxy = (T) Proxy.newProxyInstance( 209 failoverProxyProvider.getInterface().getClassLoader(), 210 new Class[] { xface }, dummyHandler); 211 Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri); 212 return new ProxyAndInfo<T>(proxy, dtService); 213 } else { 214 LOG.warn("Currently creating proxy using " + 215 "LossyRetryInvocationHandler requires NN HA setup"); 216 return null; 217 } 218 } 219 220 /** 221 * Creates an explicitly non-HA-enabled proxy object. Most of the time you 222 * don't want to use this, and should instead use {@link NameNodeProxies#createProxy}. 223 * 224 * @param conf the configuration object 225 * @param nnAddr address of the remote NN to connect to 226 * @param xface the IPC interface which should be created 227 * @param ugi the user who is making the calls on the proxy object 228 * @param withRetries certain interfaces have a non-standard retry policy 229 * @return an object containing both the proxy and the associated 230 * delegation token service it corresponds to 231 * @throws IOException 232 */ 233 @SuppressWarnings("unchecked") 234 public static <T> ProxyAndInfo<T> createNonHAProxy( 235 Configuration conf, InetSocketAddress nnAddr, Class<T> xface, 236 UserGroupInformation ugi, boolean withRetries) throws IOException { 237 Text dtService = SecurityUtil.buildTokenService(nnAddr); 238 239 T proxy; 240 if (xface == ClientProtocol.class) { 241 proxy = (T) createNNProxyWithClientProtocol(nnAddr, conf, ugi, 242 withRetries); 243 } else if (xface == JournalProtocol.class) { 244 proxy = (T) createNNProxyWithJournalProtocol(nnAddr, conf, ugi); 245 } else if (xface == NamenodeProtocol.class) { 246 proxy = (T) createNNProxyWithNamenodeProtocol(nnAddr, conf, ugi, 247 withRetries); 248 } else if (xface == GetUserMappingsProtocol.class) { 249 proxy = (T) createNNProxyWithGetUserMappingsProtocol(nnAddr, conf, ugi); 250 } else if (xface == RefreshUserMappingsProtocol.class) { 251 proxy = (T) createNNProxyWithRefreshUserMappingsProtocol(nnAddr, conf, ugi); 252 } else if (xface == RefreshAuthorizationPolicyProtocol.class) { 253 proxy = (T) createNNProxyWithRefreshAuthorizationPolicyProtocol(nnAddr, 254 conf, ugi); 255 } else { 256 String message = "Upsupported protocol found when creating the proxy " + 257 "connection to NameNode: " + 258 ((xface != null) ? xface.getClass().getName() : "null"); 259 LOG.error(message); 260 throw new IllegalStateException(message); 261 } 262 return new ProxyAndInfo<T>(proxy, dtService); 263 } 264 265 private static JournalProtocol createNNProxyWithJournalProtocol( 266 InetSocketAddress address, Configuration conf, UserGroupInformation ugi) 267 throws IOException { 268 JournalProtocolPB proxy = (JournalProtocolPB) createNameNodeProxy(address, 269 conf, ugi, JournalProtocolPB.class); 270 return new JournalProtocolTranslatorPB(proxy); 271 } 272 273 private static RefreshAuthorizationPolicyProtocol 274 createNNProxyWithRefreshAuthorizationPolicyProtocol(InetSocketAddress address, 275 Configuration conf, UserGroupInformation ugi) throws IOException { 276 RefreshAuthorizationPolicyProtocolPB proxy = (RefreshAuthorizationPolicyProtocolPB) 277 createNameNodeProxy(address, conf, ugi, RefreshAuthorizationPolicyProtocolPB.class); 278 return new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(proxy); 279 } 280 281 private static RefreshUserMappingsProtocol 282 createNNProxyWithRefreshUserMappingsProtocol(InetSocketAddress address, 283 Configuration conf, UserGroupInformation ugi) throws IOException { 284 RefreshUserMappingsProtocolPB proxy = (RefreshUserMappingsProtocolPB) 285 createNameNodeProxy(address, conf, ugi, RefreshUserMappingsProtocolPB.class); 286 return new RefreshUserMappingsProtocolClientSideTranslatorPB(proxy); 287 } 288 289 private static GetUserMappingsProtocol createNNProxyWithGetUserMappingsProtocol( 290 InetSocketAddress address, Configuration conf, UserGroupInformation ugi) 291 throws IOException { 292 GetUserMappingsProtocolPB proxy = (GetUserMappingsProtocolPB) 293 createNameNodeProxy(address, conf, ugi, GetUserMappingsProtocolPB.class); 294 return new GetUserMappingsProtocolClientSideTranslatorPB(proxy); 295 } 296 297 private static NamenodeProtocol createNNProxyWithNamenodeProtocol( 298 InetSocketAddress address, Configuration conf, UserGroupInformation ugi, 299 boolean withRetries) throws IOException { 300 NamenodeProtocolPB proxy = (NamenodeProtocolPB) createNameNodeProxy( 301 address, conf, ugi, NamenodeProtocolPB.class); 302 if (withRetries) { // create the proxy with retries 303 RetryPolicy timeoutPolicy = RetryPolicies.exponentialBackoffRetry(5, 200, 304 TimeUnit.MILLISECONDS); 305 Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap 306 = new HashMap<Class<? extends Exception>, RetryPolicy>(); 307 RetryPolicy methodPolicy = RetryPolicies.retryByException(timeoutPolicy, 308 exceptionToPolicyMap); 309 Map<String, RetryPolicy> methodNameToPolicyMap 310 = new HashMap<String, RetryPolicy>(); 311 methodNameToPolicyMap.put("getBlocks", methodPolicy); 312 methodNameToPolicyMap.put("getAccessKeys", methodPolicy); 313 proxy = (NamenodeProtocolPB) RetryProxy.create(NamenodeProtocolPB.class, 314 proxy, methodNameToPolicyMap); 315 } 316 return new NamenodeProtocolTranslatorPB(proxy); 317 } 318 319 private static ClientProtocol createNNProxyWithClientProtocol( 320 InetSocketAddress address, Configuration conf, UserGroupInformation ugi, 321 boolean withRetries) throws IOException { 322 RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine.class); 323 324 final RetryPolicy defaultPolicy = 325 RetryUtils.getDefaultRetryPolicy( 326 conf, 327 DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_ENABLED_KEY, 328 DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_ENABLED_DEFAULT, 329 DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_SPEC_KEY, 330 DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_SPEC_DEFAULT, 331 SafeModeException.class); 332 333 final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class); 334 ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy( 335 ClientNamenodeProtocolPB.class, version, address, ugi, conf, 336 NetUtils.getDefaultSocketFactory(conf), 337 org.apache.hadoop.ipc.Client.getTimeout(conf), defaultPolicy) 338 .getProxy(); 339 340 if (withRetries) { // create the proxy with retries 341 342 RetryPolicy createPolicy = RetryPolicies 343 .retryUpToMaximumCountWithFixedSleep(5, 344 HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS); 345 346 Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap 347 = new HashMap<Class<? extends Exception>, RetryPolicy>(); 348 remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class, 349 createPolicy); 350 351 Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap 352 = new HashMap<Class<? extends Exception>, RetryPolicy>(); 353 exceptionToPolicyMap.put(RemoteException.class, RetryPolicies 354 .retryByRemoteException(defaultPolicy, 355 remoteExceptionToPolicyMap)); 356 RetryPolicy methodPolicy = RetryPolicies.retryByException( 357 defaultPolicy, exceptionToPolicyMap); 358 Map<String, RetryPolicy> methodNameToPolicyMap 359 = new HashMap<String, RetryPolicy>(); 360 361 methodNameToPolicyMap.put("create", methodPolicy); 362 363 proxy = (ClientNamenodeProtocolPB) RetryProxy.create( 364 ClientNamenodeProtocolPB.class, 365 new DefaultFailoverProxyProvider<ClientNamenodeProtocolPB>( 366 ClientNamenodeProtocolPB.class, proxy), 367 methodNameToPolicyMap, 368 defaultPolicy); 369 } 370 return new ClientNamenodeProtocolTranslatorPB(proxy); 371 } 372 373 private static Object createNameNodeProxy(InetSocketAddress address, 374 Configuration conf, UserGroupInformation ugi, Class<?> xface) 375 throws IOException { 376 RPC.setProtocolEngine(conf, xface, ProtobufRpcEngine.class); 377 Object proxy = RPC.getProxy(xface, RPC.getProtocolVersion(xface), address, 378 ugi, conf, NetUtils.getDefaultSocketFactory(conf)); 379 return proxy; 380 } 381 382 /** Gets the configured Failover proxy provider's class */ 383 @VisibleForTesting 384 public static <T> Class<FailoverProxyProvider<T>> getFailoverProxyProviderClass( 385 Configuration conf, URI nameNodeUri, Class<T> xface) throws IOException { 386 if (nameNodeUri == null) { 387 return null; 388 } 389 String host = nameNodeUri.getHost(); 390 391 String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." 392 + host; 393 try { 394 @SuppressWarnings("unchecked") 395 Class<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>) conf 396 .getClass(configKey, null, FailoverProxyProvider.class); 397 if (ret != null) { 398 // If we found a proxy provider, then this URI should be a logical NN. 399 // Given that, it shouldn't have a non-default port number. 400 int port = nameNodeUri.getPort(); 401 if (port > 0 && port != NameNode.DEFAULT_PORT) { 402 throw new IOException("Port " + port + " specified in URI " 403 + nameNodeUri + " but host '" + host 404 + "' is a logical (HA) namenode" 405 + " and does not use port information."); 406 } 407 } 408 return ret; 409 } catch (RuntimeException e) { 410 if (e.getCause() instanceof ClassNotFoundException) { 411 throw new IOException("Could not load failover proxy provider class " 412 + conf.get(configKey) + " which is configured for authority " 413 + nameNodeUri, e); 414 } else { 415 throw e; 416 } 417 } 418 } 419 420 /** Creates the Failover proxy provider instance*/ 421 @VisibleForTesting 422 public static <T> FailoverProxyProvider<T> createFailoverProxyProvider( 423 Configuration conf, Class<FailoverProxyProvider<T>> failoverProxyProviderClass, 424 Class<T> xface, URI nameNodeUri) throws IOException { 425 Preconditions.checkArgument( 426 xface.isAssignableFrom(NamenodeProtocols.class), 427 "Interface %s is not a NameNode protocol", xface); 428 try { 429 Constructor<FailoverProxyProvider<T>> ctor = failoverProxyProviderClass 430 .getConstructor(Configuration.class, URI.class, Class.class); 431 FailoverProxyProvider<T> provider = ctor.newInstance(conf, nameNodeUri, 432 xface); 433 return provider; 434 } catch (Exception e) { 435 String message = "Couldn't create proxy provider " + failoverProxyProviderClass; 436 if (LOG.isDebugEnabled()) { 437 LOG.debug(message, e); 438 } 439 if (e.getCause() instanceof IOException) { 440 throw (IOException) e.getCause(); 441 } else { 442 throw new IOException(message, e); 443 } 444 } 445 } 446 447 }