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.Method;
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.FileSystem;
32 import org.apache.hadoop.fs.Path;
33 import org.apache.hadoop.hdfs.DistributedFileSystem;
34 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
35
36
37
38
39
40 @InterfaceAudience.Private
41 @InterfaceStability.Evolving
42 public class FSHDFSUtils extends FSUtils {
43 private static final Log LOG = LogFactory.getLog(FSHDFSUtils.class);
44
45
46
47
48 @Override
49 public void recoverFileLease(final FileSystem fs, final Path p,
50 Configuration conf, CancelableProgressable reporter)
51 throws IOException {
52
53 if (!(fs instanceof DistributedFileSystem)) return;
54 recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter);
55 }
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83 boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
84 final Configuration conf, final CancelableProgressable reporter)
85 throws IOException {
86 LOG.info("Recovering lease on dfs file " + p);
87 long startWaiting = EnvironmentEdgeManager.currentTimeMillis();
88
89
90
91 long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
92
93 long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
94
95
96
97 long subsequentPause = conf.getInt("hbase.lease.recovery.dfs.timeout", 61 * 1000);
98
99 Method isFileClosedMeth = null;
100
101 boolean findIsFileClosedMeth = true;
102 boolean recovered = false;
103
104 for (int nbAttempt = 0; !recovered; nbAttempt++) {
105 recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
106 if (recovered) break;
107 checkIfCancelled(reporter);
108 if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
109 try {
110
111 if (nbAttempt == 0) {
112 Thread.sleep(firstPause);
113 } else {
114
115
116 long localStartWaiting = EnvironmentEdgeManager.currentTimeMillis();
117 while ((EnvironmentEdgeManager.currentTimeMillis() - localStartWaiting) <
118 subsequentPause) {
119 Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
120 if (findIsFileClosedMeth) {
121 try {
122 isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
123 new Class[]{ Path.class });
124 } catch (NoSuchMethodException nsme) {
125 LOG.debug("isFileClosed not available");
126 } finally {
127 findIsFileClosedMeth = false;
128 }
129 }
130 if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
131 recovered = true;
132 break;
133 }
134 checkIfCancelled(reporter);
135 }
136 }
137 } catch (InterruptedException ie) {
138 InterruptedIOException iioe = new InterruptedIOException();
139 iioe.initCause(ie);
140 throw iioe;
141 }
142 }
143 return recovered;
144 }
145
146 boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
147 final int nbAttempt, final Path p, final long startWaiting) {
148 if (recoveryTimeout < EnvironmentEdgeManager.currentTimeMillis()) {
149 LOG.warn("Cannot recoverLease after trying for " +
150 conf.getInt("hbase.lease.recovery.timeout", 900000) +
151 "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
152 getLogMessageDetail(nbAttempt, p, startWaiting));
153 return true;
154 }
155 return false;
156 }
157
158
159
160
161
162
163
164
165
166
167 boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
168 final long startWaiting)
169 throws FileNotFoundException {
170 boolean recovered = false;
171 try {
172 recovered = dfs.recoverLease(p);
173 LOG.info("recoverLease=" + recovered + ", " +
174 getLogMessageDetail(nbAttempt, p, startWaiting));
175 } catch (IOException e) {
176 if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
177
178 throw new FileNotFoundException("The given HLog wasn't found at " + p);
179 } else if (e instanceof FileNotFoundException) {
180 throw (FileNotFoundException)e;
181 }
182 LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
183 }
184 return recovered;
185 }
186
187
188
189
190
191
192
193 private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
194 return "attempt=" + nbAttempt + " on file=" + p + " after " +
195 (EnvironmentEdgeManager.currentTimeMillis() - startWaiting) + "ms";
196 }
197
198
199
200
201
202
203
204
205 private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
206 try {
207 return (Boolean) m.invoke(dfs, p);
208 } catch (SecurityException e) {
209 LOG.warn("No access", e);
210 } catch (Exception e) {
211 LOG.warn("Failed invocation for " + p.toString(), e);
212 }
213 return false;
214 }
215
216 void checkIfCancelled(final CancelableProgressable reporter)
217 throws InterruptedIOException {
218 if (reporter == null) return;
219 if (!reporter.progress()) throw new InterruptedIOException("Operation cancelled");
220 }
221 }