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 */
018
019package org.apache.hadoop.hbase.util;
020
021import java.io.FileNotFoundException;
022import java.io.IOException;
023import java.io.InterruptedIOException;
024import java.lang.reflect.Method;
025import java.net.InetSocketAddress;
026import java.net.URI;
027import java.util.Collection;
028import java.util.HashSet;
029import java.util.Map;
030import java.util.Set;
031
032import org.apache.hadoop.conf.Configuration;
033import org.apache.hadoop.fs.FileSystem;
034import org.apache.hadoop.fs.Path;
035import org.apache.hadoop.hdfs.DistributedFileSystem;
036import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
037import org.apache.yetus.audience.InterfaceAudience;
038import org.apache.yetus.audience.InterfaceStability;
039import org.slf4j.Logger;
040import org.slf4j.LoggerFactory;
041
042import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
043
044/**
045 * Implementation for hdfs
046 */
047@InterfaceAudience.Private
048@InterfaceStability.Evolving
049public class FSHDFSUtils extends FSUtils {
050  private static final Logger LOG = LoggerFactory.getLogger(FSHDFSUtils.class);
051  private static Class dfsUtilClazz;
052  private static Method getNNAddressesMethod;
053
054  /**
055   * @param fs
056   * @param conf
057   * @return A set containing all namenode addresses of fs
058   */
059  private static Set<InetSocketAddress> getNNAddresses(DistributedFileSystem fs,
060                                                      Configuration conf) {
061    Set<InetSocketAddress> addresses = new HashSet<>();
062    String serviceName = fs.getCanonicalServiceName();
063
064    if (serviceName.startsWith("ha-hdfs")) {
065      try {
066        if (dfsUtilClazz == null) {
067          dfsUtilClazz = Class.forName("org.apache.hadoop.hdfs.DFSUtil");
068        }
069        if (getNNAddressesMethod == null) {
070          try {
071            // getNNServiceRpcAddressesForCluster is available only in version
072            // equal to or later than Hadoop 2.6
073            getNNAddressesMethod =
074                dfsUtilClazz.getMethod("getNNServiceRpcAddressesForCluster", Configuration.class);
075          } catch (NoSuchMethodException e) {
076            // If hadoop version is older than hadoop 2.6
077            getNNAddressesMethod =
078                dfsUtilClazz.getMethod("getNNServiceRpcAddresses", Configuration.class);
079          }
080
081        }
082
083        Map<String, Map<String, InetSocketAddress>> addressMap =
084                (Map<String, Map<String, InetSocketAddress>>) getNNAddressesMethod
085                        .invoke(null, conf);
086        String nameService = serviceName.substring(serviceName.indexOf(":") + 1);
087        if (addressMap.containsKey(nameService)) {
088          Map<String, InetSocketAddress> nnMap = addressMap.get(nameService);
089          for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) {
090            InetSocketAddress addr = e2.getValue();
091            addresses.add(addr);
092          }
093        }
094      } catch (Exception e) {
095        LOG.warn("DFSUtil.getNNServiceRpcAddresses failed. serviceName=" + serviceName, e);
096      }
097    } else {
098      URI uri = fs.getUri();
099      int port = uri.getPort();
100      if (port < 0) {
101        int idx = serviceName.indexOf(':');
102        port = Integer.parseInt(serviceName.substring(idx+1));
103      }
104      InetSocketAddress addr = new InetSocketAddress(uri.getHost(), port);
105      addresses.add(addr);
106    }
107
108    return addresses;
109  }
110
111  /**
112   * @param conf the Configuration of HBase
113   * @param srcFs
114   * @param desFs
115   * @return Whether srcFs and desFs are on same hdfs or not
116   */
117  public static boolean isSameHdfs(Configuration conf, FileSystem srcFs, FileSystem desFs) {
118    // By getCanonicalServiceName, we could make sure both srcFs and desFs
119    // show a unified format which contains scheme, host and port.
120    String srcServiceName = srcFs.getCanonicalServiceName();
121    String desServiceName = desFs.getCanonicalServiceName();
122
123    if (srcServiceName == null || desServiceName == null) {
124      return false;
125    }
126    if (srcServiceName.equals(desServiceName)) {
127      return true;
128    }
129    if (srcServiceName.startsWith("ha-hdfs") && desServiceName.startsWith("ha-hdfs")) {
130      Collection<String> internalNameServices =
131          conf.getTrimmedStringCollection("dfs.internal.nameservices");
132      if (!internalNameServices.isEmpty()) {
133        if (internalNameServices.contains(srcServiceName.split(":")[1])) {
134          return true;
135        } else {
136          return false;
137        }
138      }
139    }
140    if (srcFs instanceof DistributedFileSystem && desFs instanceof DistributedFileSystem) {
141      //If one serviceName is an HA format while the other is a non-HA format,
142      // maybe they refer to the same FileSystem.
143      //For example, srcFs is "ha-hdfs://nameservices" and desFs is "hdfs://activeNamenode:port"
144      Set<InetSocketAddress> srcAddrs = getNNAddresses((DistributedFileSystem) srcFs, conf);
145      Set<InetSocketAddress> desAddrs = getNNAddresses((DistributedFileSystem) desFs, conf);
146      if (Sets.intersection(srcAddrs, desAddrs).size() > 0) {
147        return true;
148      }
149    }
150
151    return false;
152  }
153
154  /**
155   * Recover the lease from HDFS, retrying multiple times.
156   */
157  @Override
158  public void recoverFileLease(final FileSystem fs, final Path p,
159      Configuration conf, CancelableProgressable reporter)
160  throws IOException {
161    // lease recovery not needed for local file system case.
162    if (!(fs instanceof DistributedFileSystem)) return;
163    recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter);
164  }
165
166  /*
167   * Run the dfs recover lease. recoverLease is asynchronous. It returns:
168   *    -false when it starts the lease recovery (i.e. lease recovery not *yet* done)
169   *    - true when the lease recovery has succeeded or the file is closed.
170   * But, we have to be careful.  Each time we call recoverLease, it starts the recover lease
171   * process over from the beginning.  We could put ourselves in a situation where we are
172   * doing nothing but starting a recovery, interrupting it to start again, and so on.
173   * The findings over in HBASE-8354 have it that the namenode will try to recover the lease
174   * on the file's primary node.  If all is well, it should return near immediately.  But,
175   * as is common, it is the very primary node that has crashed and so the namenode will be
176   * stuck waiting on a socket timeout before it will ask another datanode to start the
177   * recovery. It does not help if we call recoverLease in the meantime and in particular,
178   * subsequent to the socket timeout, a recoverLease invocation will cause us to start
179   * over from square one (possibly waiting on socket timeout against primary node).  So,
180   * in the below, we do the following:
181   * 1. Call recoverLease.
182   * 2. If it returns true, break.
183   * 3. If it returns false, wait a few seconds and then call it again.
184   * 4. If it returns true, break.
185   * 5. If it returns false, wait for what we think the datanode socket timeout is
186   * (configurable) and then try again.
187   * 6. If it returns true, break.
188   * 7. If it returns false, repeat starting at step 5. above.
189   *
190   * If HDFS-4525 is available, call it every second and we might be able to exit early.
191   */
192  boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
193      final Configuration conf, final CancelableProgressable reporter)
194  throws IOException {
195    LOG.info("Recover lease on dfs file " + p);
196    long startWaiting = EnvironmentEdgeManager.currentTime();
197    // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
198    // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
199    // beyond that limit 'to be safe'.
200    long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
201    // This setting should be a little bit above what the cluster dfs heartbeat is set to.
202    long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
203    // This should be set to how long it'll take for us to timeout against primary datanode if it
204    // is dead.  We set it to 64 seconds, 4 second than the default READ_TIMEOUT in HDFS, the
205    // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. If recovery is still failing after this
206    // timeout, then further recovery will take liner backoff with this base, to avoid endless
207    // preemptions when this value is not properly configured.
208    long subsequentPauseBase = conf.getLong("hbase.lease.recovery.dfs.timeout", 64 * 1000);
209
210    Method isFileClosedMeth = null;
211    // whether we need to look for isFileClosed method
212    boolean findIsFileClosedMeth = true;
213    boolean recovered = false;
214    // We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
215    for (int nbAttempt = 0; !recovered; nbAttempt++) {
216      recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
217      if (recovered) break;
218      checkIfCancelled(reporter);
219      if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
220      try {
221        // On the first time through wait the short 'firstPause'.
222        if (nbAttempt == 0) {
223          Thread.sleep(firstPause);
224        } else {
225          // Cycle here until (subsequentPause * nbAttempt) elapses.  While spinning, check
226          // isFileClosed if available (should be in hadoop 2.0.5... not in hadoop 1 though.
227          long localStartWaiting = EnvironmentEdgeManager.currentTime();
228          while ((EnvironmentEdgeManager.currentTime() - localStartWaiting) <
229              subsequentPauseBase * nbAttempt) {
230            Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
231            if (findIsFileClosedMeth) {
232              try {
233                isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
234                  new Class[]{ Path.class });
235              } catch (NoSuchMethodException nsme) {
236                LOG.debug("isFileClosed not available");
237              } finally {
238                findIsFileClosedMeth = false;
239              }
240            }
241            if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
242              recovered = true;
243              break;
244            }
245            checkIfCancelled(reporter);
246          }
247        }
248      } catch (InterruptedException ie) {
249        InterruptedIOException iioe = new InterruptedIOException();
250        iioe.initCause(ie);
251        throw iioe;
252      }
253    }
254    return recovered;
255  }
256
257  boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
258      final int nbAttempt, final Path p, final long startWaiting) {
259    if (recoveryTimeout < EnvironmentEdgeManager.currentTime()) {
260      LOG.warn("Cannot recoverLease after trying for " +
261        conf.getInt("hbase.lease.recovery.timeout", 900000) +
262        "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
263        getLogMessageDetail(nbAttempt, p, startWaiting));
264      return true;
265    }
266    return false;
267  }
268
269  /**
270   * Try to recover the lease.
271   * @param dfs
272   * @param nbAttempt
273   * @param p
274   * @param startWaiting
275   * @return True if dfs#recoverLease came by true.
276   * @throws FileNotFoundException
277   */
278  boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
279      final long startWaiting)
280  throws FileNotFoundException {
281    boolean recovered = false;
282    try {
283      recovered = dfs.recoverLease(p);
284      LOG.info((recovered? "Recovered lease, ": "Failed to recover lease, ") +
285        getLogMessageDetail(nbAttempt, p, startWaiting));
286    } catch (IOException e) {
287      if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
288        // This exception comes out instead of FNFE, fix it
289        throw new FileNotFoundException("The given WAL wasn't found at " + p);
290      } else if (e instanceof FileNotFoundException) {
291        throw (FileNotFoundException)e;
292      }
293      LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
294    }
295    return recovered;
296  }
297
298  /**
299   * @param nbAttempt
300   * @param p
301   * @param startWaiting
302   * @return Detail to append to any log message around lease recovering.
303   */
304  private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
305    return "attempt=" + nbAttempt + " on file=" + p + " after " +
306      (EnvironmentEdgeManager.currentTime() - startWaiting) + "ms";
307  }
308
309  /**
310   * Call HDFS-4525 isFileClosed if it is available.
311   * @param dfs
312   * @param m
313   * @param p
314   * @return True if file is closed.
315   */
316  private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
317    try {
318      return (Boolean) m.invoke(dfs, p);
319    } catch (SecurityException e) {
320      LOG.warn("No access", e);
321    } catch (Exception e) {
322      LOG.warn("Failed invocation for " + p.toString(), e);
323    }
324    return false;
325  }
326
327  void checkIfCancelled(final CancelableProgressable reporter)
328  throws InterruptedIOException {
329    if (reporter == null) return;
330    if (!reporter.progress()) throw new InterruptedIOException("Operation cancelled");
331  }
332}