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