1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.fs;
22
23 import java.io.IOException;
24 import java.lang.reflect.Field;
25 import java.lang.reflect.InvocationHandler;
26 import java.lang.reflect.InvocationTargetException;
27 import java.lang.reflect.Method;
28 import java.lang.reflect.Modifier;
29 import java.lang.reflect.Proxy;
30 import java.lang.reflect.UndeclaredThrowableException;
31 import java.net.URI;
32
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.fs.FSDataOutputStream;
37 import org.apache.hadoop.fs.FileSystem;
38 import org.apache.hadoop.fs.FilterFileSystem;
39 import org.apache.hadoop.fs.LocalFileSystem;
40 import org.apache.hadoop.fs.Path;
41 import org.apache.hadoop.hbase.ServerName;
42 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
43 import org.apache.hadoop.hdfs.DFSClient;
44 import org.apache.hadoop.hdfs.DistributedFileSystem;
45 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
46 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
47 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
48 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
49 import org.apache.hadoop.io.Closeable;
50 import org.apache.hadoop.util.Progressable;
51 import org.apache.hadoop.util.ReflectionUtils;
52
53
54
55
56
57
58
59
60 public class HFileSystem extends FilterFileSystem {
61 public static final Log LOG = LogFactory.getLog(HFileSystem.class);
62
63 private final FileSystem noChecksumFs;
64 private final boolean useHBaseChecksum;
65
66
67
68
69
70
71
72
73 public HFileSystem(Configuration conf, boolean useHBaseChecksum)
74 throws IOException {
75
76
77
78
79 this.fs = FileSystem.get(conf);
80 this.useHBaseChecksum = useHBaseChecksum;
81
82 fs.initialize(getDefaultUri(conf), conf);
83 addLocationsOrderInterceptor(conf);
84
85
86
87
88
89
90
91
92
93 if (useHBaseChecksum && !(fs instanceof LocalFileSystem)) {
94 conf = new Configuration(conf);
95 conf.setBoolean("dfs.client.read.shortcircuit.skip.checksum", true);
96 this.noChecksumFs = newInstanceFileSystem(conf);
97 this.noChecksumFs.setVerifyChecksum(false);
98 } else {
99 this.noChecksumFs = fs;
100 }
101 }
102
103
104
105
106
107
108
109 public HFileSystem(FileSystem fs) {
110 this.fs = fs;
111 this.noChecksumFs = fs;
112 this.useHBaseChecksum = false;
113 }
114
115
116
117
118
119
120
121
122 public FileSystem getNoChecksumFs() {
123 return noChecksumFs;
124 }
125
126
127
128
129
130 public FileSystem getBackingFs() throws IOException {
131 return fs;
132 }
133
134
135
136
137
138
139 public boolean useHBaseChecksum() {
140 return useHBaseChecksum;
141 }
142
143
144
145
146 @Override
147 public void close() throws IOException {
148 super.close();
149 if (this.noChecksumFs != fs) {
150 this.noChecksumFs.close();
151 }
152 }
153
154
155
156
157
158
159
160
161
162 private static FileSystem newInstanceFileSystem(Configuration conf)
163 throws IOException {
164 URI uri = FileSystem.getDefaultUri(conf);
165 FileSystem fs = null;
166 Class<?> clazz = conf.getClass("fs." + uri.getScheme() + ".impl", null);
167 if (clazz != null) {
168
169 fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
170 fs.initialize(uri, conf);
171 } else {
172
173
174
175 Configuration clone = new Configuration(conf);
176 clone.setBoolean("fs." + uri.getScheme() + ".impl.disable.cache", true);
177 fs = FileSystem.get(uri, clone);
178 }
179 if (fs == null) {
180 throw new IOException("No FileSystem for scheme: " + uri.getScheme());
181 }
182
183 return fs;
184 }
185
186 public static boolean addLocationsOrderInterceptor(Configuration conf) throws IOException {
187 return addLocationsOrderInterceptor(conf, new ReorderWALBlocks());
188 }
189
190
191
192
193
194
195
196
197
198 static boolean addLocationsOrderInterceptor(Configuration conf, final ReorderBlocks lrb) {
199 LOG.debug("Starting addLocationsOrderInterceptor with class " + lrb.getClass());
200
201 if (!conf.getBoolean("hbase.filesystem.reorder.blocks", true)) {
202 LOG.debug("addLocationsOrderInterceptor configured to false");
203 return false;
204 }
205
206 FileSystem fs;
207 try {
208 fs = FileSystem.get(conf);
209 } catch (IOException e) {
210 LOG.warn("Can't get the file system from the conf.", e);
211 return false;
212 }
213
214 if (!(fs instanceof DistributedFileSystem)) {
215 LOG.warn("The file system is not a DistributedFileSystem." +
216 "Not adding block location reordering");
217 return false;
218 }
219
220 DistributedFileSystem dfs = (DistributedFileSystem) fs;
221 DFSClient dfsc = dfs.getClient();
222 if (dfsc == null) {
223 LOG.warn("The DistributedFileSystem does not contain a DFSClient. Can't add the location " +
224 "block reordering interceptor. Continuing, but this is unexpected."
225 );
226 return false;
227 }
228
229 try {
230 Field nf = DFSClient.class.getDeclaredField("namenode");
231 nf.setAccessible(true);
232 Field modifiersField = Field.class.getDeclaredField("modifiers");
233 modifiersField.setAccessible(true);
234 modifiersField.setInt(nf, nf.getModifiers() & ~Modifier.FINAL);
235
236 ClientProtocol namenode = (ClientProtocol) nf.get(dfsc);
237 if (namenode == null) {
238 LOG.warn("The DFSClient is not linked to a namenode. Can't add the location block" +
239 " reordering interceptor. Continuing, but this is unexpected."
240 );
241 return false;
242 }
243
244 ClientProtocol cp1 = createReorderingProxy(namenode, lrb, conf);
245 nf.set(dfsc, cp1);
246 LOG.info("Added intercepting call to namenode#getBlockLocations");
247 } catch (NoSuchFieldException e) {
248 LOG.warn("Can't modify the DFSClient#namenode field to add the location reorder.", e);
249 return false;
250 } catch (IllegalAccessException e) {
251 LOG.warn("Can't modify the DFSClient#namenode field to add the location reorder.", e);
252 return false;
253 }
254
255 return true;
256 }
257
258 private static ClientProtocol createReorderingProxy(final ClientProtocol cp,
259 final ReorderBlocks lrb, final Configuration conf) {
260 return (ClientProtocol) Proxy.newProxyInstance
261 (cp.getClass().getClassLoader(),
262 new Class[]{ClientProtocol.class, Closeable.class},
263 new InvocationHandler() {
264 public Object invoke(Object proxy, Method method,
265 Object[] args) throws Throwable {
266 try {
267 Object res = method.invoke(cp, args);
268 if (res != null && args != null && args.length == 3
269 && "getBlockLocations".equals(method.getName())
270 && res instanceof LocatedBlocks
271 && args[0] instanceof String
272 && args[0] != null) {
273 lrb.reorderBlocks(conf, (LocatedBlocks) res, (String) args[0]);
274 }
275 return res;
276 } catch (InvocationTargetException ite) {
277
278
279 Throwable cause = ite.getCause();
280 if (cause == null){
281 throw new RuntimeException(
282 "Proxy invocation failed and getCause is null", ite);
283 }
284 if (cause instanceof UndeclaredThrowableException) {
285 Throwable causeCause = cause.getCause();
286 if (causeCause == null) {
287 throw new RuntimeException("UndeclaredThrowableException had null cause!");
288 }
289 cause = cause.getCause();
290 }
291 throw cause;
292 }
293 }
294 });
295 }
296
297
298
299
300 static interface ReorderBlocks {
301
302
303
304
305
306
307
308 public void reorderBlocks(Configuration conf, LocatedBlocks lbs, String src) throws IOException;
309 }
310
311
312
313
314
315
316 static class ReorderWALBlocks implements ReorderBlocks {
317 public void reorderBlocks(Configuration conf, LocatedBlocks lbs, String src)
318 throws IOException {
319
320 ServerName sn = HLogUtil.getServerNameFromHLogDirectoryName(conf, src);
321 if (sn == null) {
322
323 return;
324 }
325
326
327 String hostName = sn.getHostname();
328 LOG.debug(src + " is an HLog file, so reordering blocks, last hostname will be:" + hostName);
329
330
331 for (LocatedBlock lb : lbs.getLocatedBlocks()) {
332 DatanodeInfo[] dnis = lb.getLocations();
333 if (dnis != null && dnis.length > 1) {
334 boolean found = false;
335 for (int i = 0; i < dnis.length - 1 && !found; i++) {
336 if (hostName.equals(dnis[i].getHostName())) {
337
338 DatanodeInfo toLast = dnis[i];
339 System.arraycopy(dnis, i + 1, dnis, i, dnis.length - i - 1);
340 dnis[dnis.length - 1] = toLast;
341 found = true;
342 }
343 }
344 }
345 }
346 }
347 }
348
349
350
351
352
353
354
355 static public FileSystem get(Configuration conf) throws IOException {
356 return new HFileSystem(conf, true);
357 }
358
359
360
361
362 static public FileSystem getLocalFs(Configuration conf) throws IOException {
363 return new HFileSystem(FileSystem.getLocal(conf));
364 }
365
366
367
368
369
370
371 public FSDataOutputStream createNonRecursive(Path f,
372 boolean overwrite,
373 int bufferSize, short replication, long blockSize,
374 Progressable progress) throws IOException {
375 return fs.createNonRecursive(f, overwrite, bufferSize, replication,
376 blockSize, progress);
377 }
378 }