View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.util;
20  
21  import java.io.FileNotFoundException;
22  import java.io.IOException;
23  import java.io.InterruptedIOException;
24  import java.lang.reflect.Method;
25  import java.net.InetSocketAddress;
26  import java.net.URI;
27  import java.util.HashSet;
28  import java.util.Map;
29  import java.util.Set;
30  
31  import com.google.common.collect.Sets;
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.hbase.classification.InterfaceStability;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FileSystem;
38  import org.apache.hadoop.fs.Path;
39  import org.apache.hadoop.hdfs.DistributedFileSystem;
40  import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
41  
42  
43  /**
44   * Implementation for hdfs
45   */
46  @InterfaceAudience.Private
47  @InterfaceStability.Evolving
48  public class FSHDFSUtils extends FSUtils {
49    private static final Log LOG = LogFactory.getLog(FSHDFSUtils.class);
50    private static Class dfsUtilClazz;
51    private static Method getNNAddressesMethod;
52  
53    /**
54     * @param fs
55     * @param conf
56     * @return A set containing all namenode addresses of fs
57     */
58    private static Set<InetSocketAddress> getNNAddresses(DistributedFileSystem fs,
59                                                        Configuration conf) {
60      Set<InetSocketAddress> addresses = new HashSet<InetSocketAddress>();
61      String serviceName = fs.getCanonicalServiceName();
62  
63      if (serviceName.startsWith("ha-hdfs")) {
64        try {
65          if (dfsUtilClazz == null) {
66            dfsUtilClazz = Class.forName("org.apache.hadoop.hdfs.DFSUtil");
67          }
68          if (getNNAddressesMethod == null) {
69            getNNAddressesMethod =
70                    dfsUtilClazz.getMethod("getNNServiceRpcAddresses", Configuration.class);
71          }
72  
73          Map<String, Map<String, InetSocketAddress>> addressMap =
74                  (Map<String, Map<String, InetSocketAddress>>) getNNAddressesMethod
75                          .invoke(null, conf);
76          for (Map.Entry<String, Map<String, InetSocketAddress>> entry : addressMap.entrySet()) {
77            Map<String, InetSocketAddress> nnMap = entry.getValue();
78            for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) {
79              InetSocketAddress addr = e2.getValue();
80              addresses.add(addr);
81            }
82          }
83        } catch (Exception e) {
84          LOG.warn("DFSUtil.getNNServiceRpcAddresses failed. serviceName=" + serviceName, e);
85        }
86      } else {
87        URI uri = fs.getUri();
88        int port = uri.getPort();
89        if (port < 0) {
90          int idx = serviceName.indexOf(':');
91          port = Integer.parseInt(serviceName.substring(idx+1));
92        }
93        InetSocketAddress addr = new InetSocketAddress(uri.getHost(), port);
94        addresses.add(addr);
95      }
96  
97      return addresses;
98    }
99  
100   /**
101    * @param conf the Configuration of HBase
102    * @param srcFs
103    * @param desFs
104    * @return Whether srcFs and desFs are on same hdfs or not
105    */
106   public static boolean isSameHdfs(Configuration conf, FileSystem srcFs, FileSystem desFs) {
107     // By getCanonicalServiceName, we could make sure both srcFs and desFs
108     // show a unified format which contains scheme, host and port.
109     String srcServiceName = srcFs.getCanonicalServiceName();
110     String desServiceName = desFs.getCanonicalServiceName();
111 
112     if (srcServiceName == null || desServiceName == null) {
113       return false;
114     }
115     if (srcServiceName.equals(desServiceName)) {
116       return true;
117     }
118     if (srcFs instanceof DistributedFileSystem && desFs instanceof DistributedFileSystem) {
119       //If one serviceName is an HA format while the other is a non-HA format,
120       // maybe they refer to the same FileSystem.
121       //For example, srcFs is "ha-hdfs://nameservices" and desFs is "hdfs://activeNamenode:port"
122       Set<InetSocketAddress> srcAddrs = getNNAddresses((DistributedFileSystem) srcFs, conf);
123       Set<InetSocketAddress> desAddrs = getNNAddresses((DistributedFileSystem) desFs, conf);
124       if (Sets.intersection(srcAddrs, desAddrs).size() > 0) {
125         return true;
126       }
127     }
128 
129     return false;
130   }
131 
132   /**
133    * Recover the lease from HDFS, retrying multiple times.
134    */
135   @Override
136   public void recoverFileLease(final FileSystem fs, final Path p,
137       Configuration conf, CancelableProgressable reporter)
138   throws IOException {
139     // lease recovery not needed for local file system case.
140     if (!(fs instanceof DistributedFileSystem)) return;
141     recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter);
142   }
143 
144   /*
145    * Run the dfs recover lease. recoverLease is asynchronous. It returns:
146    *    -false when it starts the lease recovery (i.e. lease recovery not *yet* done)
147    *    - true when the lease recovery has succeeded or the file is closed.
148    * But, we have to be careful.  Each time we call recoverLease, it starts the recover lease
149    * process over from the beginning.  We could put ourselves in a situation where we are
150    * doing nothing but starting a recovery, interrupting it to start again, and so on.
151    * The findings over in HBASE-8354 have it that the namenode will try to recover the lease
152    * on the file's primary node.  If all is well, it should return near immediately.  But,
153    * as is common, it is the very primary node that has crashed and so the namenode will be
154    * stuck waiting on a socket timeout before it will ask another datanode to start the
155    * recovery. It does not help if we call recoverLease in the meantime and in particular,
156    * subsequent to the socket timeout, a recoverLease invocation will cause us to start
157    * over from square one (possibly waiting on socket timeout against primary node).  So,
158    * in the below, we do the following:
159    * 1. Call recoverLease.
160    * 2. If it returns true, break.
161    * 3. If it returns false, wait a few seconds and then call it again.
162    * 4. If it returns true, break.
163    * 5. If it returns false, wait for what we think the datanode socket timeout is
164    * (configurable) and then try again.
165    * 6. If it returns true, break.
166    * 7. If it returns false, repeat starting at step 5. above.
167    *
168    * If HDFS-4525 is available, call it every second and we might be able to exit early.
169    */
170   boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
171       final Configuration conf, final CancelableProgressable reporter)
172   throws IOException {
173     LOG.info("Recovering lease on dfs file " + p);
174     long startWaiting = EnvironmentEdgeManager.currentTime();
175     // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
176     // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
177     // beyond that limit 'to be safe'.
178     long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
179     // This setting should be a little bit above what the cluster dfs heartbeat is set to.
180     long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
181     // This should be set to how long it'll take for us to timeout against primary datanode if it
182     // is dead.  We set it to 61 seconds, 1 second than the default READ_TIMEOUT in HDFS, the
183     // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. If recovery is still failing after this
184     // timeout, then further recovery will take liner backoff with this base, to avoid endless
185     // preemptions when this value is not properly configured.
186     long subsequentPauseBase = conf.getLong("hbase.lease.recovery.dfs.timeout", 61 * 1000);
187 
188     Method isFileClosedMeth = null;
189     // whether we need to look for isFileClosed method
190     boolean findIsFileClosedMeth = true;
191     boolean recovered = false;
192     // We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
193     for (int nbAttempt = 0; !recovered; nbAttempt++) {
194       recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
195       if (recovered) break;
196       checkIfCancelled(reporter);
197       if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
198       try {
199         // On the first time through wait the short 'firstPause'.
200         if (nbAttempt == 0) {
201           Thread.sleep(firstPause);
202         } else {
203           // Cycle here until (subsequentPause * nbAttempt) elapses.  While spinning, check
204           // isFileClosed if available (should be in hadoop 2.0.5... not in hadoop 1 though.
205           long localStartWaiting = EnvironmentEdgeManager.currentTime();
206           while ((EnvironmentEdgeManager.currentTime() - localStartWaiting) <
207               subsequentPauseBase * nbAttempt) {
208             Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
209             if (findIsFileClosedMeth) {
210               try {
211                 isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
212                   new Class[]{ Path.class });
213               } catch (NoSuchMethodException nsme) {
214                 LOG.debug("isFileClosed not available");
215               } finally {
216                 findIsFileClosedMeth = false;
217               }
218             }
219             if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
220               recovered = true;
221               break;
222             }
223             checkIfCancelled(reporter);
224           }
225         }
226       } catch (InterruptedException ie) {
227         InterruptedIOException iioe = new InterruptedIOException();
228         iioe.initCause(ie);
229         throw iioe;
230       }
231     }
232     return recovered;
233   }
234 
235   boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
236       final int nbAttempt, final Path p, final long startWaiting) {
237     if (recoveryTimeout < EnvironmentEdgeManager.currentTime()) {
238       LOG.warn("Cannot recoverLease after trying for " +
239         conf.getInt("hbase.lease.recovery.timeout", 900000) +
240         "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
241         getLogMessageDetail(nbAttempt, p, startWaiting));
242       return true;
243     }
244     return false;
245   }
246 
247   /**
248    * Try to recover the lease.
249    * @param dfs
250    * @param nbAttempt
251    * @param p
252    * @param startWaiting
253    * @return True if dfs#recoverLease came by true.
254    * @throws FileNotFoundException
255    */
256   boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
257       final long startWaiting)
258   throws FileNotFoundException {
259     boolean recovered = false;
260     try {
261       recovered = dfs.recoverLease(p);
262       LOG.info("recoverLease=" + recovered + ", " +
263         getLogMessageDetail(nbAttempt, p, startWaiting));
264     } catch (IOException e) {
265       if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
266         // This exception comes out instead of FNFE, fix it
267         throw new FileNotFoundException("The given WAL wasn't found at " + p);
268       } else if (e instanceof FileNotFoundException) {
269         throw (FileNotFoundException)e;
270       }
271       LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
272     }
273     return recovered;
274   }
275 
276   /**
277    * @param nbAttempt
278    * @param p
279    * @param startWaiting
280    * @return Detail to append to any log message around lease recovering.
281    */
282   private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
283     return "attempt=" + nbAttempt + " on file=" + p + " after " +
284       (EnvironmentEdgeManager.currentTime() - startWaiting) + "ms";
285   }
286 
287   /**
288    * Call HDFS-4525 isFileClosed if it is available.
289    * @param dfs
290    * @param m
291    * @param p
292    * @return True if file is closed.
293    */
294   private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
295     try {
296       return (Boolean) m.invoke(dfs, p);
297     } catch (SecurityException e) {
298       LOG.warn("No access", e);
299     } catch (Exception e) {
300       LOG.warn("Failed invocation for " + p.toString(), e);
301     }
302     return false;
303   }
304 
305   void checkIfCancelled(final CancelableProgressable reporter)
306   throws InterruptedIOException {
307     if (reporter == null) return;
308     if (!reporter.progress()) throw new InterruptedIOException("Operation cancelled");
309   }
310 }