1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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
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
50
51
52
53
54
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
68
69 if (!(fs instanceof DistributedFileSystem)) {
70 return;
71 }
72 LOG.info("Recovering file " + p);
73 long startWaiting = System.currentTimeMillis();
74
75
76 boolean recovered = false;
77 long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 300000);
78
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
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;
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
112
113
114
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
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 }