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 @Override
55 public void recoverFileLease(final FileSystem fs, final Path p, Configuration conf)
56 throws IOException{
57 if (!isAppendSupported(conf)) {
58 LOG.warn("Running on HDFS without append enabled may result in data loss");
59 return;
60 }
61
62
63 if (!(fs instanceof DistributedFileSystem)) {
64 return;
65 }
66 LOG.info("Recovering file " + p);
67 long startWaiting = System.currentTimeMillis();
68
69
70 boolean recovered = false;
71 while (!recovered) {
72 try {
73 try {
74 if (fs instanceof DistributedFileSystem) {
75 DistributedFileSystem dfs = (DistributedFileSystem)fs;
76 DistributedFileSystem.class.getMethod("recoverLease",
77 new Class[] {Path.class}).invoke(dfs, p);
78 } else {
79 throw new Exception("Not a DistributedFileSystem");
80 }
81 } catch (InvocationTargetException ite) {
82
83 throw (IOException) ite.getCause();
84 } catch (Exception e) {
85 LOG.debug("Failed fs.recoverLease invocation, " + e.toString() +
86 ", trying fs.append instead");
87 FSDataOutputStream out = fs.append(p);
88 out.close();
89 }
90 recovered = true;
91 } catch (IOException e) {
92 e = RemoteExceptionHandler.checkIOException(e);
93 if (e instanceof AlreadyBeingCreatedException) {
94
95
96
97
98 long waitedFor = System.currentTimeMillis() - startWaiting;
99 if (waitedFor > LEASE_SOFTLIMIT_PERIOD) {
100 LOG.warn("Waited " + waitedFor + "ms for lease recovery on " + p +
101 ":" + e.getMessage());
102 }
103 } else if (e instanceof LeaseExpiredException &&
104 e.getMessage().contains("File does not exist")) {
105
106 throw new FileNotFoundException(
107 "The given HLog wasn't found at " + p.toString());
108 } else {
109 throw new IOException("Failed to open " + p + " for append", e);
110 }
111 }
112 try {
113 Thread.sleep(1000);
114 } catch (InterruptedException ex) {
115 InterruptedIOException iioe = new InterruptedIOException();
116 iioe.initCause(ex);
117 throw iioe;
118 }
119 }
120 LOG.info("Finished lease recover attempt for " + p);
121 }
122 }