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;
028import java.util.concurrent.atomic.AtomicBoolean;
029
030import org.apache.commons.logging.Log;
031import org.apache.commons.logging.LogFactory;
032import org.apache.hadoop.conf.Configuration;
033import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
034import org.apache.hadoop.hdfs.DFSUtil;
035import org.apache.hadoop.hdfs.HAUtil;
036import org.apache.hadoop.hdfs.NameNodeProxies;
037import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
038import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
039import org.apache.hadoop.ipc.RPC;
040import org.apache.hadoop.security.UserGroupInformation;
041
042import com.google.common.annotations.VisibleForTesting;
043import com.google.common.base.Preconditions;
044
045/**
046 * A FailoverProxyProvider implementation which allows one to configure two URIs
047 * to connect to during fail-over. The first configured address is tried first,
048 * and on a fail-over event the other address is tried.
049 */
050public class ConfiguredFailoverProxyProvider<T> extends
051    AbstractNNFailoverProxyProvider<T> {
052  
053  private static final Log LOG =
054      LogFactory.getLog(ConfiguredFailoverProxyProvider.class);
055  
056  interface ProxyFactory<T> {
057    T createProxy(Configuration conf, InetSocketAddress nnAddr, Class<T> xface,
058        UserGroupInformation ugi, boolean withRetries,
059        AtomicBoolean fallbackToSimpleAuth) throws IOException;
060  }
061
062  static class DefaultProxyFactory<T> implements ProxyFactory<T> {
063    @Override
064    public T createProxy(Configuration conf, InetSocketAddress nnAddr,
065        Class<T> xface, UserGroupInformation ugi, boolean withRetries,
066        AtomicBoolean fallbackToSimpleAuth) throws IOException {
067      return NameNodeProxies.createNonHAProxy(conf,
068          nnAddr, xface, ugi, false, fallbackToSimpleAuth).getProxy();
069    }
070  }
071
072  protected final Configuration conf;
073  protected final List<AddressRpcProxyPair<T>> proxies =
074      new ArrayList<AddressRpcProxyPair<T>>();
075  private final UserGroupInformation ugi;
076  protected final Class<T> xface;
077
078  private int currentProxyIndex = 0;
079  private final ProxyFactory<T> factory;
080
081  public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
082      Class<T> xface) {
083    this(conf, uri, xface, new DefaultProxyFactory<T>());
084  }
085
086  @VisibleForTesting
087  ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
088      Class<T> xface, ProxyFactory<T> factory) {
089
090    Preconditions.checkArgument(
091        xface.isAssignableFrom(NamenodeProtocols.class),
092        "Interface class %s is not a valid NameNode protocol!");
093    this.xface = xface;
094    
095    this.conf = new Configuration(conf);
096    int maxRetries = this.conf.getInt(
097        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
098        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
099    this.conf.setInt(
100        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
101        maxRetries);
102    
103    int maxRetriesOnSocketTimeouts = this.conf.getInt(
104        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
105        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
106    this.conf.setInt(
107            CommonConfigurationKeysPublic
108                    .IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
109            maxRetriesOnSocketTimeouts);
110
111    try {
112      ugi = UserGroupInformation.getCurrentUser();
113      
114      Map<String, Map<String, InetSocketAddress>> map = DFSUtil.getHaNnRpcAddresses(
115          conf);
116      Map<String, InetSocketAddress> addressesInNN = map.get(uri.getHost());
117      
118      if (addressesInNN == null || addressesInNN.size() == 0) {
119        throw new RuntimeException("Could not find any configured addresses " +
120            "for URI " + uri);
121      }
122      
123      Collection<InetSocketAddress> addressesOfNns = addressesInNN.values();
124      for (InetSocketAddress address : addressesOfNns) {
125        proxies.add(new AddressRpcProxyPair<T>(address));
126      }
127
128      // The client may have a delegation token set for the logical
129      // URI of the cluster. Clone this token to apply to each of the
130      // underlying IPC addresses so that the IPC code can find it.
131      HAUtil.cloneDelegationTokenForLogicalUri(ugi, uri, addressesOfNns);
132      this.factory = factory;
133    } catch (IOException e) {
134      throw new RuntimeException(e);
135    }
136  }
137    
138  @Override
139  public Class<T> getInterface() {
140    return xface;
141  }
142
143  /**
144   * Lazily initialize the RPC proxy object.
145   */
146  @Override
147  public synchronized ProxyInfo<T> getProxy() {
148    AddressRpcProxyPair<T> current = proxies.get(currentProxyIndex);
149    if (current.namenode == null) {
150      try {
151        current.namenode = factory.createProxy(conf,
152            current.address, xface, ugi, false, getFallbackToSimpleAuth());
153      } catch (IOException e) {
154        LOG.error("Failed to create RPC proxy to NameNode", e);
155        throw new RuntimeException(e);
156      }
157    }
158    return new ProxyInfo<T>(current.namenode, current.address.toString());
159  }
160
161  @Override
162  public  void performFailover(T currentProxy) {
163    incrementProxyIndex();
164  }
165
166  synchronized void incrementProxyIndex() {
167    currentProxyIndex = (currentProxyIndex + 1) % proxies.size();
168  }
169
170  /**
171   * A little pair object to store the address and connected RPC proxy object to
172   * an NN. Note that {@link AddressRpcProxyPair#namenode} may be null.
173   */
174  private static class AddressRpcProxyPair<T> {
175    public final InetSocketAddress address;
176    public T namenode;
177    
178    public AddressRpcProxyPair(InetSocketAddress address) {
179      this.address = address;
180    }
181  }
182
183  /**
184   * Close all the proxy objects which have been opened over the lifetime of
185   * this proxy provider.
186   */
187  @Override
188  public synchronized void close() throws IOException {
189    for (AddressRpcProxyPair<T> proxy : proxies) {
190      if (proxy.namenode != null) {
191        if (proxy.namenode instanceof Closeable) {
192          ((Closeable)proxy.namenode).close();
193        } else {
194          RPC.stopProxy(proxy.namenode);
195        }
196      }
197    }
198  }
199
200  /**
201   * Logical URI is required for this failover proxy provider.
202   */
203  @Override
204  public boolean useLogicalURI() {
205    return true;
206  }
207}