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  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.classification.InterfaceAudience;
29  import org.apache.hadoop.classification.InterfaceStability;
30  import org.apache.hadoop.conf.Configuration;
31  import org.apache.hadoop.fs.FileSystem;
32  import org.apache.hadoop.fs.Path;
33  import org.apache.hadoop.hdfs.DistributedFileSystem;
34  import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
35  
36  
37  /**
38   * Implementation for hdfs
39   */
40  @InterfaceAudience.Private
41  @InterfaceStability.Evolving
42  public class FSHDFSUtils extends FSUtils {
43    private static final Log LOG = LogFactory.getLog(FSHDFSUtils.class);
44  
45    /**
46     * Recover the lease from HDFS, retrying multiple times.
47     */
48    @Override
49    public void recoverFileLease(final FileSystem fs, final Path p,
50        Configuration conf, CancelableProgressable reporter)
51    throws IOException {
52      // lease recovery not needed for local file system case.
53      if (!(fs instanceof DistributedFileSystem)) return;
54      recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter);
55    }
56  
57    /*
58     * Run the dfs recover lease. recoverLease is asynchronous. It returns:
59     *    -false when it starts the lease recovery (i.e. lease recovery not *yet* done)
60     *    - true when the lease recovery has succeeded or the file is closed.
61     * But, we have to be careful.  Each time we call recoverLease, it starts the recover lease
62     * process over from the beginning.  We could put ourselves in a situation where we are
63     * doing nothing but starting a recovery, interrupting it to start again, and so on.
64     * The findings over in HBASE-8354 have it that the namenode will try to recover the lease
65     * on the file's primary node.  If all is well, it should return near immediately.  But,
66     * as is common, it is the very primary node that has crashed and so the namenode will be
67     * stuck waiting on a socket timeout before it will ask another datanode to start the
68     * recovery. It does not help if we call recoverLease in the meantime and in particular,
69     * subsequent to the socket timeout, a recoverLease invocation will cause us to start
70     * over from square one (possibly waiting on socket timeout against primary node).  So,
71     * in the below, we do the following:
72     * 1. Call recoverLease.
73     * 2. If it returns true, break.
74     * 3. If it returns false, wait a few seconds and then call it again.
75     * 4. If it returns true, break.
76     * 5. If it returns false, wait for what we think the datanode socket timeout is
77     * (configurable) and then try again.
78     * 6. If it returns true, break.
79     * 7. If it returns false, repeat starting at step 5. above.
80     *
81     * If HDFS-4525 is available, call it every second and we might be able to exit early.
82     */
83    boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
84        final Configuration conf, final CancelableProgressable reporter)
85    throws IOException {
86      LOG.info("Recovering lease on dfs file " + p);
87      long startWaiting = EnvironmentEdgeManager.currentTimeMillis();
88      // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
89      // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
90      // beyond that limit 'to be safe'.
91      long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
92      // This setting should be a little bit above what the cluster dfs heartbeat is set to.
93      long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
94      // This should be set to how long it'll take for us to timeout against primary datanode if it
95      // is dead.  We set it to 61 seconds, 1 second than the default READ_TIMEOUT in HDFS, the
96      // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY.
97      long subsequentPause = conf.getInt("hbase.lease.recovery.dfs.timeout", 61 * 1000);
98      
99      Method isFileClosedMeth = null;
100     // whether we need to look for isFileClosed method
101     boolean findIsFileClosedMeth = true;
102     boolean recovered = false;
103     // We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
104     for (int nbAttempt = 0; !recovered; nbAttempt++) {
105       recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
106       if (recovered) break;
107       checkIfCancelled(reporter);
108       if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
109       try {
110         // On the first time through wait the short 'firstPause'.
111         if (nbAttempt == 0) {
112           Thread.sleep(firstPause);
113         } else {
114           // Cycle here until subsequentPause elapses.  While spinning, check isFileClosed if
115           // available (should be in hadoop 2.0.5... not in hadoop 1 though.
116           long localStartWaiting = EnvironmentEdgeManager.currentTimeMillis();
117           while ((EnvironmentEdgeManager.currentTimeMillis() - localStartWaiting) <
118               subsequentPause) {
119             Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
120             if (findIsFileClosedMeth) {
121               try {
122                 isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
123                   new Class[]{ Path.class });
124               } catch (NoSuchMethodException nsme) {
125                 LOG.debug("isFileClosed not available");
126               } finally {
127                 findIsFileClosedMeth = false;
128               }
129             }
130             if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
131               recovered = true;
132               break;
133             }
134             checkIfCancelled(reporter);
135           }
136         }
137       } catch (InterruptedException ie) {
138         InterruptedIOException iioe = new InterruptedIOException();
139         iioe.initCause(ie);
140         throw iioe;
141       }
142     }
143     return recovered;
144   }
145 
146   boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
147       final int nbAttempt, final Path p, final long startWaiting) {
148     if (recoveryTimeout < EnvironmentEdgeManager.currentTimeMillis()) {
149       LOG.warn("Cannot recoverLease after trying for " +
150         conf.getInt("hbase.lease.recovery.timeout", 900000) +
151         "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
152         getLogMessageDetail(nbAttempt, p, startWaiting));
153       return true;
154     }
155     return false;
156   }
157 
158   /**
159    * Try to recover the lease.
160    * @param dfs
161    * @param nbAttempt
162    * @param p
163    * @param startWaiting
164    * @return True if dfs#recoverLease came by true.
165    * @throws FileNotFoundException
166    */
167   boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
168       final long startWaiting)
169   throws FileNotFoundException {
170     boolean recovered = false;
171     try {
172       recovered = dfs.recoverLease(p);
173       LOG.info("recoverLease=" + recovered + ", " +
174         getLogMessageDetail(nbAttempt, p, startWaiting));
175     } catch (IOException e) {
176       if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
177         // This exception comes out instead of FNFE, fix it
178         throw new FileNotFoundException("The given HLog wasn't found at " + p);
179       } else if (e instanceof FileNotFoundException) {
180         throw (FileNotFoundException)e;
181       }
182       LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
183     }
184     return recovered;
185   }
186 
187   /**
188    * @param nbAttempt
189    * @param p
190    * @param startWaiting
191    * @return Detail to append to any log message around lease recovering.
192    */
193   private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
194     return "attempt=" + nbAttempt + " on file=" + p + " after " +
195       (EnvironmentEdgeManager.currentTimeMillis() - startWaiting) + "ms";
196   }
197 
198   /**
199    * Call HDFS-4525 isFileClosed if it is available.
200    * @param dfs
201    * @param m
202    * @param p
203    * @return True if file is closed.
204    */
205   private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
206     try {
207       return (Boolean) m.invoke(dfs, p);
208     } catch (SecurityException e) {
209       LOG.warn("No access", e);
210     } catch (Exception e) {
211       LOG.warn("Failed invocation for " + p.toString(), e);
212     }
213     return false;
214   }
215 
216   void checkIfCancelled(final CancelableProgressable reporter)
217   throws InterruptedIOException {
218     if (reporter == null) return;
219     if (!reporter.progress()) throw new InterruptedIOException("Operation cancelled");
220   }
221 }