1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.io;
20
21 import java.util.Collection;
22
23 import java.io.IOException;
24 import java.io.InputStream;
25 import java.io.FileNotFoundException;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.classification.InterfaceAudience;
30 import org.apache.hadoop.fs.FSDataInputStream;
31 import org.apache.hadoop.fs.FileSystem;
32 import org.apache.hadoop.fs.FileStatus;
33 import org.apache.hadoop.fs.Path;
34 import org.apache.hadoop.fs.PositionedReadable;
35 import org.apache.hadoop.fs.Seekable;
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
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
84
85
86
87 @InterfaceAudience.Private
88 public class FileLink {
89 private static final Log LOG = LogFactory.getLog(FileLink.class);
90
91
92 public static final String BACK_REFERENCES_DIRECTORY_PREFIX = ".links-";
93
94
95
96
97
98 private static class FileLinkInputStream extends InputStream
99 implements Seekable, PositionedReadable {
100 private FSDataInputStream in = null;
101 private Path currentPath = null;
102 private long pos = 0;
103
104 private final FileLink fileLink;
105 private final int bufferSize;
106 private final FileSystem fs;
107
108 public FileLinkInputStream(final FileSystem fs, final FileLink fileLink)
109 throws IOException {
110 this(fs, fileLink, fs.getConf().getInt("io.file.buffer.size", 4096));
111 }
112
113 public FileLinkInputStream(final FileSystem fs, final FileLink fileLink, int bufferSize)
114 throws IOException {
115 this.bufferSize = bufferSize;
116 this.fileLink = fileLink;
117 this.fs = fs;
118
119 this.in = tryOpen();
120 }
121
122 @Override
123 public int read() throws IOException {
124 int res;
125 try {
126 res = in.read();
127 } catch (FileNotFoundException e) {
128 res = tryOpen().read();
129 } catch (NullPointerException e) {
130 res = tryOpen().read();
131 } catch (AssertionError e) {
132 res = tryOpen().read();
133 }
134 if (res > 0) pos += 1;
135 return res;
136 }
137
138 @Override
139 public int read(byte b[]) throws IOException {
140 return read(b, 0, b.length);
141 }
142
143 @Override
144 public int read(byte b[], int off, int len) throws IOException {
145 int n;
146 try {
147 n = in.read(b, off, len);
148 } catch (FileNotFoundException e) {
149 n = tryOpen().read(b, off, len);
150 } catch (NullPointerException e) {
151 n = tryOpen().read(b, off, len);
152 } catch (AssertionError e) {
153 n = tryOpen().read(b, off, len);
154 }
155 if (n > 0) pos += n;
156 assert(in.getPos() == pos);
157 return n;
158 }
159
160 @Override
161 public int read(long position, byte[] buffer, int offset, int length) throws IOException {
162 int n;
163 try {
164 n = in.read(position, buffer, offset, length);
165 } catch (FileNotFoundException e) {
166 n = tryOpen().read(position, buffer, offset, length);
167 } catch (NullPointerException e) {
168 n = tryOpen().read(position, buffer, offset, length);
169 } catch (AssertionError e) {
170 n = tryOpen().read(position, buffer, offset, length);
171 }
172 return n;
173 }
174
175 @Override
176 public void readFully(long position, byte[] buffer) throws IOException {
177 readFully(position, buffer, 0, buffer.length);
178 }
179
180 @Override
181 public void readFully(long position, byte[] buffer, int offset, int length) throws IOException {
182 try {
183 in.readFully(position, buffer, offset, length);
184 } catch (FileNotFoundException e) {
185 tryOpen().readFully(position, buffer, offset, length);
186 } catch (NullPointerException e) {
187 tryOpen().readFully(position, buffer, offset, length);
188 } catch (AssertionError e) {
189 tryOpen().readFully(position, buffer, offset, length);
190 }
191 }
192
193 @Override
194 public long skip(long n) throws IOException {
195 long skipped;
196
197 try {
198 skipped = in.skip(n);
199 } catch (FileNotFoundException e) {
200 skipped = tryOpen().skip(n);
201 } catch (NullPointerException e) {
202 skipped = tryOpen().skip(n);
203 } catch (AssertionError e) {
204 skipped = tryOpen().skip(n);
205 }
206
207 if (skipped > 0) pos += skipped;
208 return skipped;
209 }
210
211 @Override
212 public int available() throws IOException {
213 try {
214 return in.available();
215 } catch (FileNotFoundException e) {
216 return tryOpen().available();
217 } catch (NullPointerException e) {
218 return tryOpen().available();
219 } catch (AssertionError e) {
220 return tryOpen().available();
221 }
222 }
223
224 @Override
225 public void seek(long pos) throws IOException {
226 try {
227 in.seek(pos);
228 } catch (FileNotFoundException e) {
229 tryOpen().seek(pos);
230 } catch (NullPointerException e) {
231 tryOpen().seek(pos);
232 } catch (AssertionError e) {
233 tryOpen().seek(pos);
234 }
235 this.pos = pos;
236 }
237
238 @Override
239 public long getPos() throws IOException {
240 return pos;
241 }
242
243 @Override
244 public boolean seekToNewSource(long targetPos) throws IOException {
245 boolean res;
246 try {
247 res = in.seekToNewSource(targetPos);
248 } catch (FileNotFoundException e) {
249 res = tryOpen().seekToNewSource(targetPos);
250 } catch (NullPointerException e) {
251 res = tryOpen().seekToNewSource(targetPos);
252 } catch (AssertionError e) {
253 res = tryOpen().seekToNewSource(targetPos);
254 }
255 if (res) pos = targetPos;
256 return res;
257 }
258
259 @Override
260 public void close() throws IOException {
261 in.close();
262 }
263
264 @Override
265 public synchronized void mark(int readlimit) {
266 }
267
268 @Override
269 public synchronized void reset() throws IOException {
270 throw new IOException("mark/reset not supported");
271 }
272
273 @Override
274 public boolean markSupported() {
275 return false;
276 }
277
278
279
280
281
282
283
284 private FSDataInputStream tryOpen() throws IOException {
285 for (Path path: fileLink.getLocations()) {
286 if (path.equals(currentPath)) continue;
287 try {
288 in = fs.open(path, bufferSize);
289 in.seek(pos);
290 assert(in.getPos() == pos) : "Link unable to seek to the right position=" + pos;
291 if (LOG.isTraceEnabled()) {
292 if (currentPath != null) {
293 LOG.debug("link open path=" + path);
294 } else {
295 LOG.trace("link switch from path=" + currentPath + " to path=" + path);
296 }
297 }
298 currentPath = path;
299 return(in);
300 } catch (FileNotFoundException e) {
301
302 }
303 }
304 throw new FileNotFoundException("Unable to open link: " + fileLink);
305 }
306 }
307
308 private Path[] locations = null;
309
310 protected FileLink() {
311 this.locations = null;
312 }
313
314
315
316
317
318 public FileLink(Path originPath, Path... alternativePaths) {
319 setLocations(originPath, alternativePaths);
320 }
321
322
323
324
325 public FileLink(final Collection<Path> locations) {
326 this.locations = locations.toArray(new Path[locations.size()]);
327 }
328
329
330
331
332 public Path[] getLocations() {
333 return locations;
334 }
335
336 public String toString() {
337 StringBuilder str = new StringBuilder(getClass().getName());
338 str.append(" locations=[");
339 int i = 0;
340 for (Path location: locations) {
341 if (i++ > 0) str.append(", ");
342 str.append(location.toString());
343 }
344 str.append("]");
345 return str.toString();
346 }
347
348
349
350
351 public Path getAvailablePath(FileSystem fs) throws IOException {
352 for (Path path: locations) {
353 if (fs.exists(path)) {
354 return path;
355 }
356 }
357 throw new FileNotFoundException("Unable to open link: " + this);
358 }
359
360
361
362
363
364
365
366
367 public FileStatus getFileStatus(FileSystem fs) throws IOException {
368 for (Path path: locations) {
369 try {
370 return fs.getFileStatus(path);
371 } catch (FileNotFoundException e) {
372
373 }
374 }
375 throw new FileNotFoundException("Unable to open link: " + this);
376 }
377
378
379
380
381
382
383
384
385
386
387
388 public FSDataInputStream open(final FileSystem fs) throws IOException {
389 return new FSDataInputStream(new FileLinkInputStream(fs, this));
390 }
391
392
393
394
395
396
397
398
399
400
401
402
403 public FSDataInputStream open(final FileSystem fs, int bufferSize) throws IOException {
404 return new FSDataInputStream(new FileLinkInputStream(fs, this, bufferSize));
405 }
406
407
408
409
410
411 protected void setLocations(Path originPath, Path... alternativePaths) {
412 assert this.locations == null : "Link locations already set";
413 this.locations = new Path[1 + alternativePaths.length];
414 this.locations[0] = originPath;
415 for (int i = 0; i < alternativePaths.length; i++) {
416 this.locations[i + 1] = alternativePaths[i];
417 }
418 }
419
420
421
422
423
424
425
426
427
428
429
430 public static Path getBackReferencesDir(final Path storeDir, final String fileName) {
431 return new Path(storeDir, BACK_REFERENCES_DIRECTORY_PREFIX + fileName);
432 }
433
434
435
436
437
438
439
440 public static String getBackReferenceFileName(final Path dirPath) {
441 return dirPath.getName().substring(BACK_REFERENCES_DIRECTORY_PREFIX.length());
442 }
443
444
445
446
447
448
449
450 public static boolean isBackReferencesDir(final Path dirPath) {
451 if (dirPath == null) return false;
452 return dirPath.getName().startsWith(BACK_REFERENCES_DIRECTORY_PREFIX);
453 }
454 }
455