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.conf.Configuration;
29  import org.apache.hadoop.fs.FileSystem;
30  import org.apache.hadoop.fs.Path;
31  import org.apache.hadoop.hdfs.DistributedFileSystem;
32  import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
33  
34  
35  /**
36   * Implementation for hdfs
37   */
38  public class FSHDFSUtils extends FSUtils{
39    private static final Log LOG = LogFactory.getLog(FSHDFSUtils.class);
40  
41    /**
42     * Recover the lease from HDFS, retrying multiple times.
43     */
44    @Override
45    public void recoverFileLease(final FileSystem fs, final Path p,
46        Configuration conf)
47    throws IOException {
48      if (!isAppendSupported(conf)) {
49        LOG.warn("Running on HDFS without append enabled may result in data loss");
50        return;
51      }
52      // lease recovery not needed for local file system case.
53      if (!(fs instanceof DistributedFileSystem)) return;
54      recoverDFSFileLease((DistributedFileSystem)fs, p, conf);
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)
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 what the cluster dfs heartbeat is set to.
93      long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 3000);
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       if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
108       try {
109         // On the first time through wait the short 'firstPause'.
110         if (nbAttempt == 0) {
111           Thread.sleep(firstPause);
112         } else {
113           // Cycle here until subsequentPause elapses.  While spinning, check isFileClosed if
114           // available (should be in hadoop 2.0.5... not in hadoop 1 though.
115           long localStartWaiting = EnvironmentEdgeManager.currentTimeMillis();
116           while ((EnvironmentEdgeManager.currentTimeMillis() - localStartWaiting) <
117               subsequentPause) {
118             Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
119             if (findIsFileClosedMeth) {
120                try {
121                  isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
122                    new Class[]{ Path.class });
123                } catch (NoSuchMethodException nsme) {
124                  LOG.debug("isFileClosed not available");
125                } finally {
126                  findIsFileClosedMeth = false;
127                }
128              }
129              if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
130                recovered = true;
131                break;
132              }
133           }
134         }
135       } catch (InterruptedException ie) {
136         InterruptedIOException iioe = new InterruptedIOException();
137         iioe.initCause(ie);
138         throw iioe;
139       }
140     }
141     return recovered;
142   }
143 
144   boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
145       final int nbAttempt, final Path p, final long startWaiting) {
146     if (recoveryTimeout < EnvironmentEdgeManager.currentTimeMillis()) {
147       LOG.warn("Cannot recoverLease after trying for " +
148         conf.getInt("hbase.lease.recovery.timeout", 900000) +
149         "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
150         getLogMessageDetail(nbAttempt, p, startWaiting));
151       return true;
152     }
153     return false;
154   }
155 
156   /**
157    * Try to recover the lease.
158    * @param dfs
159    * @param nbAttempt
160    * @param p
161    * @param startWaiting
162    * @return True if dfs#recoverLease came by true.
163    * @throws FileNotFoundException
164    */
165   boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
166       final long startWaiting)
167   throws FileNotFoundException {
168     boolean recovered = false;
169     try {
170       recovered = dfs.recoverLease(p);
171       LOG.info("recoverLease=" + recovered + ", " +
172         getLogMessageDetail(nbAttempt, p, startWaiting));
173     } catch (IOException e) {
174       if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
175         // This exception comes out instead of FNFE, fix it
176         throw new FileNotFoundException("The given HLog wasn't found at " + p);
177       } else if (e instanceof FileNotFoundException) {
178         throw (FileNotFoundException)e;
179       }
180       LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
181     }
182     return recovered;
183   }
184 
185   /**
186    * @param nbAttempt
187    * @param p
188    * @param startWaiting
189    * @return Detail to append to any log message around lease recovering.
190    */
191   private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
192     return "attempt=" + nbAttempt + " on file=" + p + " after " +
193       (EnvironmentEdgeManager.currentTimeMillis() - startWaiting) + "ms";
194   }
195 
196   /**
197    * Call HDFS-4525 isFileClosed if it is available.
198    * @param dfs
199    * @param m
200    * @param p
201    * @return True if file is closed.
202    */
203   private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
204     try {
205       return (Boolean) m.invoke(dfs, p);
206     } catch (SecurityException e) {
207       LOG.warn("No access", e);
208     } catch (Exception e) {
209       LOG.warn("Failed invocation for " + p.toString(), e);
210     }
211     return false;
212   }
213 }