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 * <p>
010 * http://www.apache.org/licenses/LICENSE-2.0
011 * <p>
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.IOException;
021import java.lang.reflect.InvocationHandler;
022import java.lang.reflect.Method;
023import java.lang.reflect.Proxy;
024import java.net.URI;
025import java.util.HashMap;
026import java.util.Map;
027import java.util.concurrent.Callable;
028import java.util.concurrent.CompletionService;
029import java.util.concurrent.ExecutorCompletionService;
030import java.util.concurrent.ExecutorService;
031import java.util.concurrent.Executors;
032import java.util.concurrent.Future;
033
034import org.apache.hadoop.conf.Configuration;
035import org.apache.hadoop.ipc.RemoteException;
036import org.apache.hadoop.ipc.StandbyException;
037
038import com.google.common.annotations.VisibleForTesting;
039import org.apache.hadoop.io.retry.MultiException;
040import org.slf4j.Logger;
041import org.slf4j.LoggerFactory;
042
043/**
044 * A FailoverProxyProvider implementation that technically does not "failover"
045 * per-se. It constructs a wrapper proxy that sends the request to ALL
046 * underlying proxies simultaneously. It assumes the in an HA setup, there will
047 * be only one Active, and the active should respond faster than any configured
048 * standbys. Once it recieve a response from any one of the configred proxies,
049 * outstanding requests to other proxies are immediately cancelled.
050 */
051public class RequestHedgingProxyProvider<T> extends
052        ConfiguredFailoverProxyProvider<T> {
053
054  public static final Logger LOG =
055      LoggerFactory.getLogger(RequestHedgingProxyProvider.class);
056
057  class RequestHedgingInvocationHandler implements InvocationHandler {
058
059    final Map<String, ProxyInfo<T>> targetProxies;
060
061    public RequestHedgingInvocationHandler(
062            Map<String, ProxyInfo<T>> targetProxies) {
063      this.targetProxies = new HashMap<>(targetProxies);
064    }
065
066    /**
067     * Creates a Executor and invokes all proxies concurrently. This
068     * implementation assumes that Clients have configured proper socket
069     * timeouts, else the call can block forever.
070     *
071     * @param proxy
072     * @param method
073     * @param args
074     * @return
075     * @throws Throwable
076     */
077    @Override
078    public Object
079    invoke(Object proxy, final Method method, final Object[] args)
080            throws Throwable {
081      Map<Future<Object>, ProxyInfo<T>> proxyMap = new HashMap<>();
082      int numAttempts = 0;
083
084      ExecutorService executor = null;
085      CompletionService<Object> completionService;
086      try {
087        // Optimization : if only 2 proxies are configured and one had failed
088        // over, then we dont need to create a threadpool etc.
089        targetProxies.remove(toIgnore);
090        if (targetProxies.size() == 1) {
091          ProxyInfo<T> proxyInfo = targetProxies.values().iterator().next();
092          Object retVal = method.invoke(proxyInfo.proxy, args);
093          successfulProxy = proxyInfo;
094          return retVal;
095        }
096        executor = Executors.newFixedThreadPool(proxies.size());
097        completionService = new ExecutorCompletionService<>(executor);
098        for (final Map.Entry<String, ProxyInfo<T>> pEntry :
099                targetProxies.entrySet()) {
100          Callable<Object> c = new Callable<Object>() {
101            @Override
102            public Object call() throws Exception {
103              LOG.trace("Invoking method {} on proxy {}", method,
104                  pEntry.getValue().proxyInfo);
105              return method.invoke(pEntry.getValue().proxy, args);
106            }
107          };
108          proxyMap.put(completionService.submit(c), pEntry.getValue());
109          numAttempts++;
110        }
111
112        Map<String, Exception> badResults = new HashMap<>();
113        while (numAttempts > 0) {
114          Future<Object> callResultFuture = completionService.take();
115          Object retVal;
116          try {
117            retVal = callResultFuture.get();
118            successfulProxy = proxyMap.get(callResultFuture);
119            LOG.debug("Invocation successful on [{}]",
120                successfulProxy.proxyInfo);
121            return retVal;
122          } catch (Exception ex) {
123            ProxyInfo<T> tProxyInfo = proxyMap.get(callResultFuture);
124            logProxyException(ex, tProxyInfo.proxyInfo);
125            badResults.put(tProxyInfo.proxyInfo, ex);
126            LOG.trace("Unsuccessful invocation on [{}]", tProxyInfo.proxyInfo);
127            numAttempts--;
128          }
129        }
130
131        // At this point we should have All bad results (Exceptions)
132        // Or should have returned with successful result.
133        if (badResults.size() == 1) {
134          throw badResults.values().iterator().next();
135        } else {
136          throw new MultiException(badResults);
137        }
138      } finally {
139        if (executor != null) {
140          LOG.trace("Shutting down threadpool executor");
141          executor.shutdownNow();
142        }
143      }
144    }
145  }
146
147
148  private volatile ProxyInfo<T> successfulProxy = null;
149  private volatile String toIgnore = null;
150
151  public RequestHedgingProxyProvider(
152          Configuration conf, URI uri, Class<T> xface) {
153    this(conf, uri, xface, new DefaultProxyFactory<T>());
154  }
155
156  @VisibleForTesting
157  RequestHedgingProxyProvider(Configuration conf, URI uri,
158                              Class<T> xface, ProxyFactory<T> factory) {
159    super(conf, uri, xface, factory);
160  }
161
162  @SuppressWarnings("unchecked")
163  @Override
164  public synchronized ProxyInfo<T> getProxy() {
165    if (successfulProxy != null) {
166      return successfulProxy;
167    }
168    Map<String, ProxyInfo<T>> targetProxyInfos = new HashMap<>();
169    StringBuilder combinedInfo = new StringBuilder("[");
170    for (int i = 0; i < proxies.size(); i++) {
171      ProxyInfo<T> pInfo = super.getProxy();
172      incrementProxyIndex();
173      targetProxyInfos.put(pInfo.proxyInfo, pInfo);
174      combinedInfo.append(pInfo.proxyInfo).append(',');
175    }
176    combinedInfo.append(']');
177    T wrappedProxy = (T) Proxy.newProxyInstance(
178            RequestHedgingInvocationHandler.class.getClassLoader(),
179            new Class<?>[]{xface},
180            new RequestHedgingInvocationHandler(targetProxyInfos));
181    return new ProxyInfo<T>(wrappedProxy, combinedInfo.toString());
182  }
183
184  @Override
185  public synchronized void performFailover(T currentProxy) {
186    toIgnore = successfulProxy.proxyInfo;
187    successfulProxy = null;
188  }
189
190  /**
191   * Check the exception returned by the proxy log a warning message if it's
192   * not a StandbyException (expected exception).
193   * @param ex Exception to evaluate.
194   * @param proxyInfo Information of the proxy reporting the exception.
195   */
196  private void logProxyException(Exception ex, String proxyInfo) {
197    if (isStandbyException(ex)) {
198      LOG.debug("Invocation returned standby exception on [{}]", proxyInfo);
199    } else {
200      LOG.warn("Invocation returned exception on [{}]", proxyInfo);
201    }
202  }
203
204  /**
205   * Check if the returned exception is caused by an standby namenode.
206   * @param ex Exception to check.
207   * @return If the exception is caused by an standby namenode.
208   */
209  private boolean isStandbyException(Exception ex) {
210    Throwable cause = ex.getCause();
211    if (cause != null) {
212      Throwable cause2 = cause.getCause();
213      if (cause2 instanceof RemoteException) {
214        RemoteException remoteException = (RemoteException)cause2;
215        IOException unwrapRemoteException =
216            remoteException.unwrapRemoteException();
217        return unwrapRemoteException instanceof StandbyException;
218      }
219    }
220    return false;
221  }
222}