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.classification.InterfaceAudience;
35  import org.apache.hadoop.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        InetSocketAddress addr = new InetSocketAddress(uri.getHost(), uri.getPort());
89        addresses.add(addr);
90      }
91  
92      return addresses;
93    }
94  
95    /**
96     * @param conf the Configuration of HBase
97     * @param srcFs
98     * @param desFs
99     * @return Whether srcFs and desFs are on same hdfs or not
100    */
101   public static boolean isSameHdfs(Configuration conf, FileSystem srcFs, FileSystem desFs) {
102     // By getCanonicalServiceName, we could make sure both srcFs and desFs
103     // show a unified format which contains scheme, host and port.
104     String srcServiceName = srcFs.getCanonicalServiceName();
105     String desServiceName = desFs.getCanonicalServiceName();
106 
107     if (srcServiceName == null || desServiceName == null) {
108       return false;
109     }
110     if (srcServiceName.equals(desServiceName)) {
111       return true;
112     }
113     if (srcFs instanceof DistributedFileSystem && desFs instanceof DistributedFileSystem) {
114       //If one serviceName is an HA format while the other is a non-HA format,
115       // maybe they refer to the same FileSystem.
116       //For example, srcFs is "ha-hdfs://nameservices" and desFs is "hdfs://activeNamenode:port"
117       Set<InetSocketAddress> srcAddrs = getNNAddresses((DistributedFileSystem) srcFs, conf);
118       Set<InetSocketAddress> desAddrs = getNNAddresses((DistributedFileSystem) desFs, conf);
119       if (Sets.intersection(srcAddrs, desAddrs).size() > 0) {
120         return true;
121       }
122     }
123 
124     return false;
125   }
126 
127   /**
128    * Recover the lease from HDFS, retrying multiple times.
129    */
130   @Override
131   public void recoverFileLease(final FileSystem fs, final Path p,
132       Configuration conf, CancelableProgressable reporter)
133   throws IOException {
134     // lease recovery not needed for local file system case.
135     if (!(fs instanceof DistributedFileSystem)) return;
136     recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter);
137   }
138 
139   /*
140    * Run the dfs recover lease. recoverLease is asynchronous. It returns:
141    *    -false when it starts the lease recovery (i.e. lease recovery not *yet* done)
142    *    - true when the lease recovery has succeeded or the file is closed.
143    * But, we have to be careful.  Each time we call recoverLease, it starts the recover lease
144    * process over from the beginning.  We could put ourselves in a situation where we are
145    * doing nothing but starting a recovery, interrupting it to start again, and so on.
146    * The findings over in HBASE-8354 have it that the namenode will try to recover the lease
147    * on the file's primary node.  If all is well, it should return near immediately.  But,
148    * as is common, it is the very primary node that has crashed and so the namenode will be
149    * stuck waiting on a socket timeout before it will ask another datanode to start the
150    * recovery. It does not help if we call recoverLease in the meantime and in particular,
151    * subsequent to the socket timeout, a recoverLease invocation will cause us to start
152    * over from square one (possibly waiting on socket timeout against primary node).  So,
153    * in the below, we do the following:
154    * 1. Call recoverLease.
155    * 2. If it returns true, break.
156    * 3. If it returns false, wait a few seconds and then call it again.
157    * 4. If it returns true, break.
158    * 5. If it returns false, wait for what we think the datanode socket timeout is
159    * (configurable) and then try again.
160    * 6. If it returns true, break.
161    * 7. If it returns false, repeat starting at step 5. above.
162    *
163    * If HDFS-4525 is available, call it every second and we might be able to exit early.
164    */
165   boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
166       final Configuration conf, final CancelableProgressable reporter)
167   throws IOException {
168     LOG.info("Recovering lease on dfs file " + p);
169     long startWaiting = EnvironmentEdgeManager.currentTimeMillis();
170     // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
171     // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
172     // beyond that limit 'to be safe'.
173     long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
174     // This setting should be a little bit above what the cluster dfs heartbeat is set to.
175     long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
176     // This should be set to how long it'll take for us to timeout against primary datanode if it
177     // is dead.  We set it to 61 seconds, 1 second than the default READ_TIMEOUT in HDFS, the
178     // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY.
179     long subsequentPause = conf.getInt("hbase.lease.recovery.dfs.timeout", 61 * 1000);
180     
181     Method isFileClosedMeth = null;
182     // whether we need to look for isFileClosed method
183     boolean findIsFileClosedMeth = true;
184     boolean recovered = false;
185     // We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
186     for (int nbAttempt = 0; !recovered; nbAttempt++) {
187       recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
188       if (recovered) break;
189       checkIfCancelled(reporter);
190       if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
191       try {
192         // On the first time through wait the short 'firstPause'.
193         if (nbAttempt == 0) {
194           Thread.sleep(firstPause);
195         } else {
196           // Cycle here until subsequentPause elapses.  While spinning, check isFileClosed if
197           // available (should be in hadoop 2.0.5... not in hadoop 1 though.
198           long localStartWaiting = EnvironmentEdgeManager.currentTimeMillis();
199           while ((EnvironmentEdgeManager.currentTimeMillis() - localStartWaiting) <
200               subsequentPause) {
201             Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
202             if (findIsFileClosedMeth) {
203               try {
204                 isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
205                   new Class[]{ Path.class });
206               } catch (NoSuchMethodException nsme) {
207                 LOG.debug("isFileClosed not available");
208               } finally {
209                 findIsFileClosedMeth = false;
210               }
211             }
212             if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
213               recovered = true;
214               break;
215             }
216             checkIfCancelled(reporter);
217           }
218         }
219       } catch (InterruptedException ie) {
220         InterruptedIOException iioe = new InterruptedIOException();
221         iioe.initCause(ie);
222         throw iioe;
223       }
224     }
225     return recovered;
226   }
227 
228   boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
229       final int nbAttempt, final Path p, final long startWaiting) {
230     if (recoveryTimeout < EnvironmentEdgeManager.currentTimeMillis()) {
231       LOG.warn("Cannot recoverLease after trying for " +
232         conf.getInt("hbase.lease.recovery.timeout", 900000) +
233         "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
234         getLogMessageDetail(nbAttempt, p, startWaiting));
235       return true;
236     }
237     return false;
238   }
239 
240   /**
241    * Try to recover the lease.
242    * @param dfs
243    * @param nbAttempt
244    * @param p
245    * @param startWaiting
246    * @return True if dfs#recoverLease came by true.
247    * @throws FileNotFoundException
248    */
249   boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
250       final long startWaiting)
251   throws FileNotFoundException {
252     boolean recovered = false;
253     try {
254       recovered = dfs.recoverLease(p);
255       LOG.info("recoverLease=" + recovered + ", " +
256         getLogMessageDetail(nbAttempt, p, startWaiting));
257     } catch (IOException e) {
258       if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
259         // This exception comes out instead of FNFE, fix it
260         throw new FileNotFoundException("The given HLog wasn't found at " + p);
261       } else if (e instanceof FileNotFoundException) {
262         throw (FileNotFoundException)e;
263       }
264       LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
265     }
266     return recovered;
267   }
268 
269   /**
270    * @param nbAttempt
271    * @param p
272    * @param startWaiting
273    * @return Detail to append to any log message around lease recovering.
274    */
275   private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
276     return "attempt=" + nbAttempt + " on file=" + p + " after " +
277       (EnvironmentEdgeManager.currentTimeMillis() - startWaiting) + "ms";
278   }
279 
280   /**
281    * Call HDFS-4525 isFileClosed if it is available.
282    * @param dfs
283    * @param m
284    * @param p
285    * @return True if file is closed.
286    */
287   private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
288     try {
289       return (Boolean) m.invoke(dfs, p);
290     } catch (SecurityException e) {
291       LOG.warn("No access", e);
292     } catch (Exception e) {
293       LOG.warn("Failed invocation for " + p.toString(), e);
294     }
295     return false;
296   }
297 
298   void checkIfCancelled(final CancelableProgressable reporter)
299   throws InterruptedIOException {
300     if (reporter == null) return;
301     if (!reporter.progress()) throw new InterruptedIOException("Operation cancelled");
302   }
303 }