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.hbase.client;
019
020import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
021
022import java.io.IOException;
023import java.io.InterruptedIOException;
024import java.lang.reflect.UndeclaredThrowableException;
025import java.net.SocketTimeoutException;
026import java.time.Instant;
027import java.util.ArrayList;
028import java.util.List;
029import java.util.concurrent.atomic.AtomicBoolean;
030import org.apache.hadoop.hbase.DoNotRetryIOException;
031import org.apache.hadoop.hbase.HBaseServerException;
032import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException;
033import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
034import org.apache.hadoop.hbase.util.ExceptionUtil;
035import org.apache.hadoop.ipc.RemoteException;
036import org.apache.hadoop.util.StringUtils;
037import org.apache.yetus.audience.InterfaceAudience;
038import org.slf4j.Logger;
039import org.slf4j.LoggerFactory;
040
041import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
042
043/**
044 * Runs an rpc'ing {@link RetryingCallable}. Sets into rpc client threadlocal outstanding timeouts
045 * as so we don't persist too much. Dynamic rather than static so can set the generic appropriately.
046 * This object has a state. It should not be used by in parallel by different threads. Reusing it is
047 * possible however, even between multiple threads. However, the user will have to manage the
048 * synchronization on its side: there is no synchronization inside the class.
049 */
050@InterfaceAudience.Private
051public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
052  // LOG is being used in TestMultiRowRangeFilter, hence leaving it public
053  public static final Logger LOG = LoggerFactory.getLogger(RpcRetryingCallerImpl.class);
054
055  /** How many retries are allowed before we start to log */
056  private final int startLogErrorsCnt;
057
058  private final long pause;
059  private final long pauseForServerOverloaded;
060  private final int maxAttempts;// how many times to try
061  private final int rpcTimeout;// timeout for each rpc request
062  private final AtomicBoolean cancelled = new AtomicBoolean(false);
063  private final RetryingCallerInterceptor interceptor;
064  private final RetryingCallerInterceptorContext context;
065  private final RetryingTimeTracker tracker;
066
067  public RpcRetryingCallerImpl(long pause, long pauseForServerOverloaded, int retries,
068    int startLogErrorsCnt) {
069    this(pause, pauseForServerOverloaded, retries,
070      RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, startLogErrorsCnt, 0);
071  }
072
073  public RpcRetryingCallerImpl(long pause, long pauseForServerOverloaded, int retries,
074    RetryingCallerInterceptor interceptor, int startLogErrorsCnt, int rpcTimeout) {
075    this.pause = pause;
076    this.pauseForServerOverloaded = pauseForServerOverloaded;
077    this.maxAttempts = retries2Attempts(retries);
078    this.interceptor = interceptor;
079    context = interceptor.createEmptyContext();
080    this.startLogErrorsCnt = startLogErrorsCnt;
081    this.tracker = new RetryingTimeTracker();
082    this.rpcTimeout = rpcTimeout;
083  }
084
085  @Override
086  public void cancel() {
087    cancelled.set(true);
088    synchronized (cancelled) {
089      cancelled.notifyAll();
090    }
091  }
092
093  @Override
094  public T callWithRetries(RetryingCallable<T> callable, int callTimeout)
095    throws IOException, RuntimeException {
096    List<RetriesExhaustedException.ThrowableWithExtraContext> exceptions = new ArrayList<>();
097    tracker.start();
098    context.clear();
099    for (int tries = 0;; tries++) {
100      long expectedSleep;
101      try {
102        // bad cache entries are cleared in the call to RetryingCallable#throwable() in catch block
103        callable.prepare(tries != 0);
104        interceptor.intercept(context.prepare(callable, tries));
105        return callable.call(getTimeout(callTimeout));
106      } catch (PreemptiveFastFailException e) {
107        throw e;
108      } catch (Throwable t) {
109        ExceptionUtil.rethrowIfInterrupt(t);
110        Throwable cause = t.getCause();
111        if (cause instanceof DoNotRetryIOException) {
112          // Fail fast
113          throw (DoNotRetryIOException) cause;
114        }
115        // translateException throws exception when should not retry: i.e. when request is bad.
116        interceptor.handleFailure(context, t);
117        t = translateException(t);
118
119        if (tries > startLogErrorsCnt) {
120          if (LOG.isInfoEnabled()) {
121            StringBuilder builder = new StringBuilder("Call exception, tries=").append(tries)
122              .append(", retries=").append(maxAttempts).append(", started=")
123              .append((EnvironmentEdgeManager.currentTime() - tracker.getStartTime()))
124              .append(" ms ago, ").append("cancelled=").append(cancelled.get()).append(", msg=")
125              .append(t.getMessage()).append(", details=")
126              .append(callable.getExceptionMessageAdditionalDetail())
127              .append(", see https://s.apache.org/timeout");
128            if (LOG.isDebugEnabled()) {
129              builder.append(", exception=").append(StringUtils.stringifyException(t));
130              LOG.debug(builder.toString());
131            } else {
132              LOG.info(builder.toString());
133            }
134          }
135        }
136
137        callable.throwable(t, maxAttempts != 1);
138        RetriesExhaustedException.ThrowableWithExtraContext qt =
139          new RetriesExhaustedException.ThrowableWithExtraContext(t,
140            EnvironmentEdgeManager.currentTime(), toString());
141        exceptions.add(qt);
142        if (tries >= maxAttempts - 1) {
143          throw new RetriesExhaustedException(tries, exceptions);
144        }
145        // If the server is dead, we need to wait a little before retrying, to give
146        // a chance to the regions to be moved
147        // get right pause time, start by RETRY_BACKOFF[0] * pauseBase, where pauseBase might be
148        // special when encountering an exception indicating the server is overloaded.
149        // see #HBASE-17114 and HBASE-26807
150        long pauseBase =
151          HBaseServerException.isServerOverloaded(t) ? pauseForServerOverloaded : pause;
152        expectedSleep = callable.sleep(pauseBase, tries);
153
154        // If, after the planned sleep, there won't be enough time left, we stop now.
155        long duration = singleCallDuration(expectedSleep);
156        if (duration > callTimeout) {
157          String msg = "callTimeout=" + callTimeout + ", callDuration=" + duration + ": "
158            + t.getMessage() + " " + callable.getExceptionMessageAdditionalDetail();
159          throw (SocketTimeoutException) new SocketTimeoutException(msg).initCause(t);
160        }
161      } finally {
162        interceptor.updateFailureInfo(context);
163      }
164      try {
165        if (expectedSleep > 0) {
166          synchronized (cancelled) {
167            if (cancelled.get()) return null;
168            cancelled.wait(expectedSleep);
169          }
170        }
171        if (cancelled.get()) return null;
172      } catch (InterruptedException e) {
173        throw new InterruptedIOException(
174          "Interrupted after " + tries + " tries while maxAttempts=" + maxAttempts);
175      }
176    }
177  }
178
179  /** Returns Calculate how long a single call took */
180  private long singleCallDuration(final long expectedSleep) {
181    return (EnvironmentEdgeManager.currentTime() - tracker.getStartTime()) + expectedSleep;
182  }
183
184  @Override
185  public T callWithoutRetries(RetryingCallable<T> callable, int callTimeout)
186    throws IOException, RuntimeException {
187    // The code of this method should be shared with withRetries.
188    try {
189      callable.prepare(false);
190      return callable.call(callTimeout);
191    } catch (Throwable t) {
192      Throwable t2 = translateException(t);
193      ExceptionUtil.rethrowIfInterrupt(t2);
194      // It would be nice to clear the location cache here.
195      if (t2 instanceof IOException) {
196        throw (IOException) t2;
197      } else {
198        throw new RuntimeException(t2);
199      }
200    }
201  }
202
203  /**
204   * Get the good or the remote exception if any, throws the DoNotRetryIOException.
205   * @param t the throwable to analyze
206   * @return the translated exception, if it's not a DoNotRetryIOException
207   * @throws DoNotRetryIOException - if we find it, we throw it instead of translating.
208   */
209  static Throwable translateException(Throwable t) throws DoNotRetryIOException {
210    if (t instanceof UndeclaredThrowableException) {
211      if (t.getCause() != null) {
212        t = t.getCause();
213      }
214    }
215    if (t instanceof RemoteException) {
216      t = ((RemoteException) t).unwrapRemoteException();
217    }
218    if (t instanceof LinkageError) {
219      throw new DoNotRetryIOException(t);
220    }
221    if (t instanceof ServiceException) {
222      ServiceException se = (ServiceException) t;
223      Throwable cause = se.getCause();
224      if (cause instanceof DoNotRetryIOException) {
225        throw (DoNotRetryIOException) cause;
226      }
227      // Don't let ServiceException out; its rpc specific.
228      // It also could be a RemoteException, so go around again.
229      t = translateException(cause);
230    } else if (t instanceof DoNotRetryIOException) {
231      throw (DoNotRetryIOException) t;
232    }
233    return t;
234  }
235
236  private int getTimeout(int callTimeout) {
237    int timeout = tracker.getRemainingTime(callTimeout);
238    if (timeout <= 0 || (rpcTimeout > 0 && rpcTimeout < timeout)) {
239      timeout = rpcTimeout;
240    }
241    return timeout;
242  }
243
244  @Override
245  public String toString() {
246    return "RpcRetryingCaller{" + "globalStartTime=" + Instant.ofEpochMilli(tracker.getStartTime())
247      + ", pause=" + pause + ", maxAttempts=" + maxAttempts + '}';
248  }
249}