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.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
40
41 public class FSHDFSUtils extends FSUtils{
42 private static final Log LOG = LogFactory.getLog(FSHDFSUtils.class);
43
44
45
46
47
48
49
50
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
64
65 if (!(fs instanceof DistributedFileSystem)) {
66 return;
67 }
68 LOG.info("Recovering file " + p);
69 long startWaiting = System.currentTimeMillis();
70
71
72 boolean recovered = false;
73 long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 300000);
74
75 boolean triggerAppend = conf.getBoolean(TEST_TRIGGER_DFS_APPEND, false);
76
77
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
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;
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
111
112
113
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
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
136 break;
137 }
138 LOG.info("Finished lease recovery attempt for " + p);
139 }
140 }