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.server.namenode.ha; 019 020import java.io.Closeable; 021import java.io.IOException; 022import java.net.InetSocketAddress; 023import java.net.URI; 024import java.util.ArrayList; 025import java.util.Collection; 026import java.util.List; 027import java.util.Map; 028 029import org.apache.commons.logging.Log; 030import org.apache.commons.logging.LogFactory; 031import org.apache.hadoop.conf.Configuration; 032import org.apache.hadoop.fs.CommonConfigurationKeysPublic; 033import org.apache.hadoop.hdfs.DFSConfigKeys; 034import org.apache.hadoop.hdfs.DFSUtil; 035import org.apache.hadoop.hdfs.HAUtil; 036import org.apache.hadoop.hdfs.NameNodeProxies; 037import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; 038import org.apache.hadoop.io.retry.FailoverProxyProvider; 039import org.apache.hadoop.ipc.RPC; 040import org.apache.hadoop.security.UserGroupInformation; 041 042import com.google.common.base.Preconditions; 043 044/** 045 * A FailoverProxyProvider implementation which allows one to configure two URIs 046 * to connect to during fail-over. The first configured address is tried first, 047 * and on a fail-over event the other address is tried. 048 */ 049public class ConfiguredFailoverProxyProvider<T> implements 050 FailoverProxyProvider<T> { 051 052 private static final Log LOG = 053 LogFactory.getLog(ConfiguredFailoverProxyProvider.class); 054 055 private final Configuration conf; 056 private final List<AddressRpcProxyPair<T>> proxies = 057 new ArrayList<AddressRpcProxyPair<T>>(); 058 private final UserGroupInformation ugi; 059 private final Class<T> xface; 060 061 private int currentProxyIndex = 0; 062 063 public ConfiguredFailoverProxyProvider(Configuration conf, URI uri, 064 Class<T> xface) { 065 Preconditions.checkArgument( 066 xface.isAssignableFrom(NamenodeProtocols.class), 067 "Interface class %s is not a valid NameNode protocol!"); 068 this.xface = xface; 069 070 this.conf = new Configuration(conf); 071 int maxRetries = this.conf.getInt( 072 DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY, 073 DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT); 074 this.conf.setInt( 075 CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 076 maxRetries); 077 078 int maxRetriesOnSocketTimeouts = this.conf.getInt( 079 DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 080 DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT); 081 this.conf.setInt( 082 CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 083 maxRetriesOnSocketTimeouts); 084 085 try { 086 ugi = UserGroupInformation.getCurrentUser(); 087 088 Map<String, Map<String, InetSocketAddress>> map = DFSUtil.getHaNnRpcAddresses( 089 conf); 090 Map<String, InetSocketAddress> addressesInNN = map.get(uri.getHost()); 091 092 if (addressesInNN == null || addressesInNN.size() == 0) { 093 throw new RuntimeException("Could not find any configured addresses " + 094 "for URI " + uri); 095 } 096 097 Collection<InetSocketAddress> addressesOfNns = addressesInNN.values(); 098 for (InetSocketAddress address : addressesOfNns) { 099 proxies.add(new AddressRpcProxyPair<T>(address)); 100 } 101 102 // The client may have a delegation token set for the logical 103 // URI of the cluster. Clone this token to apply to each of the 104 // underlying IPC addresses so that the IPC code can find it. 105 HAUtil.cloneDelegationTokenForLogicalUri(ugi, uri, addressesOfNns); 106 } catch (IOException e) { 107 throw new RuntimeException(e); 108 } 109 } 110 111 @Override 112 public Class<T> getInterface() { 113 return xface; 114 } 115 116 /** 117 * Lazily initialize the RPC proxy object. 118 */ 119 @SuppressWarnings("unchecked") 120 @Override 121 public synchronized T getProxy() { 122 AddressRpcProxyPair current = proxies.get(currentProxyIndex); 123 if (current.namenode == null) { 124 try { 125 current.namenode = NameNodeProxies.createNonHAProxy(conf, 126 current.address, xface, ugi, false).getProxy(); 127 } catch (IOException e) { 128 LOG.error("Failed to create RPC proxy to NameNode", e); 129 throw new RuntimeException(e); 130 } 131 } 132 return (T)current.namenode; 133 } 134 135 @Override 136 public synchronized void performFailover(T currentProxy) { 137 currentProxyIndex = (currentProxyIndex + 1) % proxies.size(); 138 } 139 140 /** 141 * A little pair object to store the address and connected RPC proxy object to 142 * an NN. Note that {@link AddressRpcProxyPair#namenode} may be null. 143 */ 144 private static class AddressRpcProxyPair<T> { 145 public InetSocketAddress address; 146 public T namenode; 147 148 public AddressRpcProxyPair(InetSocketAddress address) { 149 this.address = address; 150 } 151 } 152 153 /** 154 * Close all the proxy objects which have been opened over the lifetime of 155 * this proxy provider. 156 */ 157 @Override 158 public synchronized void close() throws IOException { 159 for (AddressRpcProxyPair<T> proxy : proxies) { 160 if (proxy.namenode != null) { 161 if (proxy.namenode instanceof Closeable) { 162 ((Closeable)proxy.namenode).close(); 163 } else { 164 RPC.stopProxy(proxy.namenode); 165 } 166 } 167 } 168 } 169}