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