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.util;
019
020import java.io.FileNotFoundException;
021import java.io.IOException;
022import java.io.InterruptedIOException;
023import java.lang.reflect.Method;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.fs.FileSystem;
026import org.apache.hadoop.fs.FilterFileSystem;
027import org.apache.hadoop.fs.Path;
028import org.apache.hadoop.hdfs.DistributedFileSystem;
029import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
030import org.apache.yetus.audience.InterfaceAudience;
031import org.slf4j.Logger;
032import org.slf4j.LoggerFactory;
033
034/**
035 * Utility methods for recovering file lease for hdfs.
036 */
037@InterfaceAudience.Private
038public final class RecoverLeaseFSUtils {
039
040  private static final Logger LOG = LoggerFactory.getLogger(RecoverLeaseFSUtils.class);
041
042  private RecoverLeaseFSUtils() {
043  }
044
045  public static void recoverFileLease(FileSystem fs, Path p, Configuration conf)
046    throws IOException {
047    recoverFileLease(fs, p, conf, null);
048  }
049
050  /**
051   * Recover the lease from HDFS, retrying multiple times.
052   */
053  public static void recoverFileLease(FileSystem fs, Path p, Configuration conf,
054    CancelableProgressable reporter) throws IOException {
055    if (fs instanceof FilterFileSystem) {
056      fs = ((FilterFileSystem) fs).getRawFileSystem();
057    }
058    // lease recovery not needed for local file system case.
059    if (!(fs instanceof DistributedFileSystem)) {
060      return;
061    }
062    recoverDFSFileLease((DistributedFileSystem) fs, p, conf, reporter);
063  }
064
065  /*
066   * Run the dfs recover lease. recoverLease is asynchronous. It returns: -false when it starts the
067   * lease recovery (i.e. lease recovery not *yet* done) - true when the lease recovery has
068   * succeeded or the file is closed. But, we have to be careful. Each time we call recoverLease, it
069   * starts the recover lease process over from the beginning. We could put ourselves in a situation
070   * where we are doing nothing but starting a recovery, interrupting it to start again, and so on.
071   * The findings over in HBASE-8354 have it that the namenode will try to recover the lease on the
072   * file's primary node. If all is well, it should return near immediately. But, as is common, it
073   * is the very primary node that has crashed and so the namenode will be stuck waiting on a socket
074   * timeout before it will ask another datanode to start the recovery. It does not help if we call
075   * recoverLease in the meantime and in particular, subsequent to the socket timeout, a
076   * recoverLease invocation will cause us to start over from square one (possibly waiting on socket
077   * timeout against primary node). So, in the below, we do the following: 1. Call recoverLease. 2.
078   * If it returns true, break. 3. If it returns false, wait a few seconds and then call it again.
079   * 4. If it returns true, break. 5. If it returns false, wait for what we think the datanode
080   * socket timeout is (configurable) and then try again. 6. If it returns true, break. 7. If it
081   * returns false, repeat starting at step 5. above. If HDFS-4525 is available, call it every
082   * second and we might be able to exit early.
083   */
084  private static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
085    final Configuration conf, final CancelableProgressable reporter) throws IOException {
086    LOG.info("Recover lease on dfs file " + p);
087    long startWaiting = EnvironmentEdgeManager.currentTime();
088    // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
089    // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
090    // beyond that limit 'to be safe'.
091    long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
092    // This setting should be a little bit above what the cluster dfs heartbeat is set to.
093    long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
094    // This should be set to how long it'll take for us to timeout against primary datanode if it
095    // is dead. We set it to 64 seconds, 4 second than the default READ_TIMEOUT in HDFS, the
096    // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. If recovery is still failing after this
097    // timeout, then further recovery will take liner backoff with this base, to avoid endless
098    // preemptions when this value is not properly configured.
099    long subsequentPauseBase = conf.getLong("hbase.lease.recovery.dfs.timeout", 64 * 1000);
100
101    Method isFileClosedMeth = null;
102    // whether we need to look for isFileClosed method
103    boolean findIsFileClosedMeth = true;
104    boolean recovered = false;
105    // We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
106    for (int nbAttempt = 0; !recovered; nbAttempt++) {
107      recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
108      if (recovered) {
109        break;
110      }
111      checkIfCancelled(reporter);
112      if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) {
113        break;
114      }
115      try {
116        // On the first time through wait the short 'firstPause'.
117        if (nbAttempt == 0) {
118          Thread.sleep(firstPause);
119        } else {
120          // Cycle here until (subsequentPause * nbAttempt) elapses. While spinning, check
121          // isFileClosed if available (should be in hadoop 2.0.5... not in hadoop 1 though.
122          long localStartWaiting = EnvironmentEdgeManager.currentTime();
123          while (
124            (EnvironmentEdgeManager.currentTime() - localStartWaiting)
125                < subsequentPauseBase * nbAttempt
126          ) {
127            Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
128            if (findIsFileClosedMeth) {
129              try {
130                isFileClosedMeth =
131                  dfs.getClass().getMethod("isFileClosed", new Class[] { Path.class });
132              } catch (NoSuchMethodException nsme) {
133                LOG.debug("isFileClosed not available");
134              } finally {
135                findIsFileClosedMeth = false;
136              }
137            }
138            if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
139              recovered = true;
140              break;
141            }
142            checkIfCancelled(reporter);
143          }
144        }
145      } catch (InterruptedException ie) {
146        InterruptedIOException iioe = new InterruptedIOException();
147        iioe.initCause(ie);
148        throw iioe;
149      }
150    }
151    return recovered;
152  }
153
154  private static boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
155    final int nbAttempt, final Path p, final long startWaiting) {
156    if (recoveryTimeout < EnvironmentEdgeManager.currentTime()) {
157      LOG.warn("Cannot recoverLease after trying for "
158        + conf.getInt("hbase.lease.recovery.timeout", 900000)
159        + "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; "
160        + getLogMessageDetail(nbAttempt, p, startWaiting));
161      return true;
162    }
163    return false;
164  }
165
166  /**
167   * Try to recover the lease.
168   * @return True if dfs#recoverLease came by true.
169   */
170  private static boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt,
171    final Path p, final long startWaiting) throws FileNotFoundException {
172    boolean recovered = false;
173    try {
174      recovered = dfs.recoverLease(p);
175      LOG.info((recovered ? "Recovered lease, " : "Failed to recover lease, ")
176        + getLogMessageDetail(nbAttempt, p, startWaiting));
177    } catch (IOException e) {
178      if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
179        // This exception comes out instead of FNFE, fix it
180        throw new FileNotFoundException("The given WAL wasn't found at " + p);
181      } else if (e instanceof FileNotFoundException) {
182        throw (FileNotFoundException) e;
183      }
184      LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
185    }
186    return recovered;
187  }
188
189  /** Returns Detail to append to any log message around lease recovering. */
190  private static String getLogMessageDetail(final int nbAttempt, final Path p,
191    final long startWaiting) {
192    return "attempt=" + nbAttempt + " on file=" + p + " after "
193      + (EnvironmentEdgeManager.currentTime() - startWaiting) + "ms";
194  }
195
196  /**
197   * Call HDFS-4525 isFileClosed if it is available.
198   * @return True if file is closed.
199   */
200  private static boolean isFileClosed(final DistributedFileSystem dfs, final Method m,
201    final Path p) {
202    try {
203      return (Boolean) m.invoke(dfs, p);
204    } catch (SecurityException e) {
205      LOG.warn("No access", e);
206    } catch (Exception e) {
207      LOG.warn("Failed invocation for " + p.toString(), e);
208    }
209    return false;
210  }
211
212  private static void checkIfCancelled(final CancelableProgressable reporter)
213    throws InterruptedIOException {
214    if (reporter == null) {
215      return;
216    }
217    if (!reporter.progress()) {
218      throw new InterruptedIOException("Operation cancelled");
219    }
220  }
221}