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.InvocationTargetException;
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.FSDataOutputStream;
32  import org.apache.hadoop.fs.FileSystem;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.hbase.RemoteExceptionHandler;
35  import org.apache.hadoop.hdfs.DistributedFileSystem;
36  import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
37  import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
38  
39  
40  /**
41   * Implementation for hdfs
42   */
43  @InterfaceAudience.Public
44  @InterfaceStability.Evolving
45  public class FSHDFSUtils extends FSUtils{
46    private static final Log LOG = LogFactory.getLog(FSHDFSUtils.class);
47  
48    /**
49     * Lease timeout constant, sourced from HDFS upstream.
50     * The upstream constant is defined in a private interface, so we
51     * can't reuse for compatibility reasons.
52     * NOTE: On versions earlier than Hadoop 0.23, the constant is in
53     * o.a.h.hdfs.protocol.FSConstants, while for 0.23 and above it is
54     * in o.a.h.hdfs.protocol.HdfsConstants cause of HDFS-1620.
55     */
56    public static final long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
57    
58    public static final String TEST_TRIGGER_DFS_APPEND = "hbase.test.trigger.dfs.append";
59  
60    @Override
61    public void recoverFileLease(final FileSystem fs, final Path p, Configuration conf)
62    throws IOException{
63      if (!isAppendSupported(conf)) {
64        LOG.warn("Running on HDFS without append enabled may result in data loss");
65        return;
66      }
67      // lease recovery not needed for local file system case.
68      // currently, local file system doesn't implement append either.
69      if (!(fs instanceof DistributedFileSystem)) {
70        return;
71      }
72      LOG.info("Recovering file " + p);
73      long startWaiting = System.currentTimeMillis();
74  
75      // Trying recovery
76      boolean recovered = false;
77      long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 300000);
78      // conf parameter passed from unit test, indicating whether fs.append() should be triggered
79      boolean triggerAppend = conf.getBoolean(TEST_TRIGGER_DFS_APPEND, false);
80      Exception ex = null;
81      while (!recovered) {
82        try {
83          try {
84            DistributedFileSystem dfs = (DistributedFileSystem) fs;
85            if (triggerAppend) throw new IOException();
86            try {
87              recovered = (Boolean) DistributedFileSystem.class.getMethod(
88                "recoverLease", new Class[] { Path.class }).invoke(dfs, p);
89              if (!recovered) LOG.debug("recoverLease returned false");
90            } catch (InvocationTargetException ite) {
91              // function was properly called, but threw it's own exception
92              throw (IOException) ite.getCause();
93            }
94          } catch (Exception e) {
95            LOG.debug("Failed fs.recoverLease invocation, " + e.toString() +
96                ", trying fs.append instead");
97            ex = e;
98          }
99          if (ex != null || System.currentTimeMillis() - startWaiting > recoveryTimeout) {
100           LOG.debug("trying fs.append for " + p + " with " + ex);
101           ex = null; // assume the following append() call would succeed
102           FSDataOutputStream out = fs.append(p);
103           out.close();
104           recovered = true;
105           LOG.debug("fs.append passed");
106         }
107         if (recovered) break;
108       } catch (IOException e) {
109         e = RemoteExceptionHandler.checkIOException(e);
110         if (e instanceof AlreadyBeingCreatedException) {
111           // We expect that we'll get this message while the lease is still
112           // within its soft limit, but if we get it past that, it means
113           // that the RS is holding onto the file even though it lost its
114           // znode. We could potentially abort after some time here.
115           long waitedFor = System.currentTimeMillis() - startWaiting;
116           if (waitedFor > LEASE_SOFTLIMIT_PERIOD) {
117             LOG.warn("Waited " + waitedFor + "ms for lease recovery on " + p +
118               ":" + e.getMessage());
119           }
120         } else if (e instanceof LeaseExpiredException &&
121             e.getMessage().contains("File does not exist")) {
122           // This exception comes out instead of FNFE, fix it
123           throw new FileNotFoundException(
124               "The given HLog wasn't found at " + p.toString());
125         } else {
126           throw new IOException("Failed to open " + p + " for append", e);
127         }
128       }
129       try {
130         Thread.sleep(1000);
131       } catch (InterruptedException ie) {
132         InterruptedIOException iioe = new InterruptedIOException();
133         iioe.initCause(ie);
134         throw iioe;
135       }
136     }
137     LOG.info("Finished lease recover attempt for " + p);
138   }
139 }