/
NativeS3FileSystem.java
804 lines (722 loc) · 25.7 KB
/
NativeS3FileSystem.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
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
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3native;
import java.io.BufferedOutputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.security.DigestOutputStream;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BufferedFSInputStream;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3.S3Exception;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.util.Progressable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A {@link FileSystem} for reading and writing files stored on
* <a href="http://aws.amazon.com/s3">Amazon S3</a>.
* Unlike {@link org.apache.hadoop.fs.s3.S3FileSystem} this implementation
* stores files on S3 in their
* native form so they can be read by other S3 tools.
* <p>
* A note about directories. S3 of course has no "native" support for them.
* The idiom we choose then is: for any directory created by this class,
* we use an empty object "#{dirpath}_$folder$" as a marker.
* Further, to interoperate with other S3 tools, we also accept the following:
* <ul>
* <li>an object "#{dirpath}/' denoting a directory marker</li>
* <li>
* if there exists any objects with the prefix "#{dirpath}/", then the
* directory is said to exist
* </li>
* <li>
* if both a file with the name of a directory and a marker for that
* directory exists, then the *file masks the directory*, and the directory
* is never returned.
* </li>
* </ul>
*
* @see org.apache.hadoop.fs.s3.S3FileSystem
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class NativeS3FileSystem extends FileSystem {
public static final Logger LOG =
LoggerFactory.getLogger(NativeS3FileSystem.class);
private static final String FOLDER_SUFFIX = "_$folder$";
static final String PATH_DELIMITER = Path.SEPARATOR;
private static final int S3_MAX_LISTING_LENGTH = 1000;
static class NativeS3FsInputStream extends FSInputStream {
private NativeFileSystemStore store;
private Statistics statistics;
private InputStream in;
private final String key;
private long pos = 0;
public NativeS3FsInputStream(NativeFileSystemStore store, Statistics statistics, InputStream in, String key) {
Preconditions.checkNotNull(in, "Null input stream");
this.store = store;
this.statistics = statistics;
this.in = in;
this.key = key;
}
@Override
public synchronized int read() throws IOException {
int result;
try {
result = in.read();
} catch (IOException e) {
LOG.info("Received IOException while reading '{}', attempting to reopen",
key);
LOG.debug("{}", e, e);
try {
reopen(pos);
result = in.read();
} catch (EOFException eof) {
LOG.debug("EOF on input stream read: {}", eof, eof);
result = -1;
}
}
if (result != -1) {
pos++;
}
if (statistics != null && result != -1) {
statistics.incrementBytesRead(1);
}
return result;
}
@Override
public synchronized int read(byte[] b, int off, int len)
throws IOException {
if (in == null) {
throw new EOFException("Cannot read closed stream");
}
int result = -1;
try {
result = in.read(b, off, len);
} catch (EOFException eof) {
throw eof;
} catch (IOException e) {
LOG.info( "Received IOException while reading '{}'," +
" attempting to reopen.", key);
reopen(pos);
result = in.read(b, off, len);
}
if (result > 0) {
pos += result;
}
if (statistics != null && result > 0) {
statistics.incrementBytesRead(result);
}
return result;
}
@Override
public synchronized void close() throws IOException {
closeInnerStream();
}
/**
* Close the inner stream if not null. Even if an exception
* is raised during the close, the field is set to null
*/
private void closeInnerStream() {
IOUtils.closeStream(in);
in = null;
}
/**
* Reopen a new input stream with the specified position
* @param pos the position to reopen a new stream
* @throws IOException
*/
private synchronized void reopen(long pos) throws IOException {
LOG.debug("Reopening key '{}' for reading at position '{}", key, pos);
InputStream newStream = store.retrieve(key, pos);
updateInnerStream(newStream, pos);
}
/**
* Update inner stream with a new stream and position
* @param newStream new stream -must not be null
* @param newpos new position
* @throws IOException IO exception on a failure to close the existing
* stream.
*/
private synchronized void updateInnerStream(InputStream newStream, long newpos) throws IOException {
Preconditions.checkNotNull(newStream, "Null newstream argument");
closeInnerStream();
in = newStream;
this.pos = newpos;
}
@Override
public synchronized void seek(long newpos) throws IOException {
if (newpos < 0) {
throw new EOFException(
FSExceptionMessages.NEGATIVE_SEEK);
}
if (pos != newpos) {
// the seek is attempting to move the current position
reopen(newpos);
}
}
@Override
public synchronized long getPos() throws IOException {
return pos;
}
@Override
public boolean seekToNewSource(long targetPos) throws IOException {
return false;
}
}
private class NativeS3FsOutputStream extends OutputStream {
private Configuration conf;
private String key;
private File backupFile;
private OutputStream backupStream;
private MessageDigest digest;
private boolean closed;
private LocalDirAllocator lDirAlloc;
public NativeS3FsOutputStream(Configuration conf,
NativeFileSystemStore store, String key, Progressable progress,
int bufferSize) throws IOException {
this.conf = conf;
this.key = key;
this.backupFile = newBackupFile();
LOG.info("OutputStream for key '" + key + "' writing to tempfile '" + this.backupFile + "'");
try {
this.digest = MessageDigest.getInstance("MD5");
this.backupStream = new BufferedOutputStream(new DigestOutputStream(
new FileOutputStream(backupFile), this.digest));
} catch (NoSuchAlgorithmException e) {
LOG.warn("Cannot load MD5 digest algorithm," +
"skipping message integrity check.", e);
this.backupStream = new BufferedOutputStream(
new FileOutputStream(backupFile));
}
}
private File newBackupFile() throws IOException {
if (lDirAlloc == null) {
lDirAlloc = new LocalDirAllocator("fs.s3.buffer.dir");
}
File result = lDirAlloc.createTmpFileForWrite("output-", LocalDirAllocator.SIZE_UNKNOWN, conf);
result.deleteOnExit();
return result;
}
@Override
public void flush() throws IOException {
backupStream.flush();
}
@Override
public synchronized void close() throws IOException {
if (closed) {
return;
}
backupStream.close();
LOG.info("OutputStream for key '{}' closed. Now beginning upload", key);
try {
byte[] md5Hash = digest == null ? null : digest.digest();
store.storeFile(key, backupFile, md5Hash);
} finally {
if (!backupFile.delete()) {
LOG.warn("Could not delete temporary s3n file: " + backupFile);
}
super.close();
closed = true;
}
LOG.info("OutputStream for key '{}' upload complete", key);
}
@Override
public void write(int b) throws IOException {
backupStream.write(b);
}
@Override
public void write(byte[] b, int off, int len) throws IOException {
backupStream.write(b, off, len);
}
}
private URI uri;
private NativeFileSystemStore store;
private Path workingDir;
public NativeS3FileSystem() {
// set store in initialize()
}
public NativeS3FileSystem(NativeFileSystemStore store) {
this.store = store;
}
/**
* Return the protocol scheme for the FileSystem.
*
* @return <code>s3n</code>
*/
@Override
public String getScheme() {
return "s3n";
}
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
super.initialize(uri, conf);
if (store == null) {
store = createDefaultStore(conf);
}
store.initialize(uri, conf);
setConf(conf);
this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
this.workingDir =
new Path("/user", System.getProperty("user.name")).makeQualified(this.uri, this.getWorkingDirectory());
}
private static NativeFileSystemStore createDefaultStore(Configuration conf) {
NativeFileSystemStore store = new Jets3tNativeFileSystemStore();
RetryPolicy basePolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
conf.getInt("fs.s3.maxRetries", 4),
conf.getLong("fs.s3.sleepTimeSeconds", 10), TimeUnit.SECONDS);
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();
exceptionToPolicyMap.put(IOException.class, basePolicy);
exceptionToPolicyMap.put(S3Exception.class, basePolicy);
RetryPolicy methodPolicy = RetryPolicies.retryByException(
RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
Map<String, RetryPolicy> methodNameToPolicyMap =
new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("storeFile", methodPolicy);
methodNameToPolicyMap.put("rename", methodPolicy);
return (NativeFileSystemStore)
RetryProxy.create(NativeFileSystemStore.class, store,
methodNameToPolicyMap);
}
private static String pathToKey(Path path) {
if (path.toUri().getScheme() != null && path.toUri().getPath().isEmpty()) {
// allow uris without trailing slash after bucket to refer to root,
// like s3n://mybucket
return "";
}
if (!path.isAbsolute()) {
throw new IllegalArgumentException("Path must be absolute: " + path);
}
String ret = path.toUri().getPath().substring(1); // remove initial slash
if (ret.endsWith("/") && (ret.indexOf("/") != ret.length() - 1)) {
ret = ret.substring(0, ret.length() -1);
}
return ret;
}
private static Path keyToPath(String key) {
return new Path("/" + key);
}
private Path makeAbsolute(Path path) {
if (path.isAbsolute()) {
return path;
}
return new Path(workingDir, path);
}
/** This optional operation is not yet supported. */
@Override
public FSDataOutputStream append(Path f, int bufferSize,
Progressable progress) throws IOException {
throw new IOException("Not supported");
}
@Override
public FSDataOutputStream create(Path f, FsPermission permission,
boolean overwrite, int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException {
if (exists(f) && !overwrite) {
throw new FileAlreadyExistsException("File already exists: " + f);
}
if(LOG.isDebugEnabled()) {
LOG.debug("Creating new file '" + f + "' in S3");
}
Path absolutePath = makeAbsolute(f);
String key = pathToKey(absolutePath);
return new FSDataOutputStream(new NativeS3FsOutputStream(getConf(), store,
key, progress, bufferSize), statistics);
}
@Override
public boolean delete(Path f, boolean recurse) throws IOException {
FileStatus status;
try {
status = getFileStatus(f);
} catch (FileNotFoundException e) {
if(LOG.isDebugEnabled()) {
LOG.debug("Delete called for '" + f +
"' but file does not exist, so returning false");
}
return false;
}
Path absolutePath = makeAbsolute(f);
String key = pathToKey(absolutePath);
if (status.isDirectory()) {
if (!recurse && listStatus(f).length > 0) {
throw new IOException("Can not delete " + f + " as is a not empty directory and recurse option is false");
}
createParent(f);
if(LOG.isDebugEnabled()) {
LOG.debug("Deleting directory '" + f + "'");
}
String priorLastKey = null;
do {
PartialListing listing = store.list(key, S3_MAX_LISTING_LENGTH, priorLastKey, true);
for (FileMetadata file : listing.getFiles()) {
store.delete(file.getKey());
}
priorLastKey = listing.getPriorLastKey();
} while (priorLastKey != null);
try {
store.delete(key + FOLDER_SUFFIX);
} catch (FileNotFoundException e) {
//this is fine, we don't require a marker
}
} else {
if(LOG.isDebugEnabled()) {
LOG.debug("Deleting file '" + f + "'");
}
createParent(f);
store.delete(key);
}
return true;
}
@Override
public FileStatus getFileStatus(Path f) throws IOException {
Path absolutePath = makeAbsolute(f);
String key = pathToKey(absolutePath);
if (key.length() == 0) { // root always exists
return newDirectory(absolutePath);
}
if(LOG.isDebugEnabled()) {
LOG.debug("getFileStatus retrieving metadata for key '" + key + "'");
}
FileMetadata meta = store.retrieveMetadata(key);
if (meta != null) {
if(LOG.isDebugEnabled()) {
LOG.debug("getFileStatus returning 'file' for key '" + key + "'");
}
return newFile(meta, absolutePath);
}
if (store.retrieveMetadata(key + FOLDER_SUFFIX) != null) {
if(LOG.isDebugEnabled()) {
LOG.debug("getFileStatus returning 'directory' for key '" + key +
"' as '" + key + FOLDER_SUFFIX + "' exists");
}
return newDirectory(absolutePath);
}
if(LOG.isDebugEnabled()) {
LOG.debug("getFileStatus listing key '" + key + "'");
}
PartialListing listing = store.list(key, 1);
if (listing.getFiles().length > 0 ||
listing.getCommonPrefixes().length > 0) {
if(LOG.isDebugEnabled()) {
LOG.debug("getFileStatus returning 'directory' for key '" + key +
"' as it has contents");
}
return newDirectory(absolutePath);
}
if(LOG.isDebugEnabled()) {
LOG.debug("getFileStatus could not find key '" + key + "'");
}
throw new FileNotFoundException("No such file or directory '" + absolutePath + "'");
}
@Override
public URI getUri() {
return uri;
}
/**
* <p>
* If <code>f</code> is a file, this method will make a single call to S3.
* If <code>f</code> is a directory, this method will make a maximum of
* (<i>n</i> / 1000) + 2 calls to S3, where <i>n</i> is the total number of
* files and directories contained directly in <code>f</code>.
* </p>
*/
@Override
public FileStatus[] listStatus(Path f) throws IOException {
Path absolutePath = makeAbsolute(f);
String key = pathToKey(absolutePath);
if (key.length() > 0) {
FileMetadata meta = store.retrieveMetadata(key);
if (meta != null) {
return new FileStatus[] { newFile(meta, absolutePath) };
}
}
URI pathUri = absolutePath.toUri();
Set<FileStatus> status = new TreeSet<FileStatus>();
String priorLastKey = null;
do {
PartialListing listing = store.list(key, S3_MAX_LISTING_LENGTH, priorLastKey, false);
for (FileMetadata fileMetadata : listing.getFiles()) {
Path subpath = keyToPath(fileMetadata.getKey());
String relativePath = pathUri.relativize(subpath.toUri()).getPath();
if (fileMetadata.getKey().equals(key + "/")) {
// this is just the directory we have been asked to list
}
else if (relativePath.endsWith(FOLDER_SUFFIX)) {
status.add(newDirectory(new Path(
absolutePath,
relativePath.substring(0, relativePath.indexOf(FOLDER_SUFFIX)))));
}
else {
status.add(newFile(fileMetadata, subpath));
}
}
for (String commonPrefix : listing.getCommonPrefixes()) {
Path subpath = keyToPath(commonPrefix);
String relativePath = pathUri.relativize(subpath.toUri()).getPath();
status.add(newDirectory(new Path(absolutePath, relativePath)));
}
priorLastKey = listing.getPriorLastKey();
} while (priorLastKey != null);
if (status.isEmpty() &&
key.length() > 0 &&
store.retrieveMetadata(key + FOLDER_SUFFIX) == null) {
throw new FileNotFoundException("File " + f + " does not exist.");
}
return status.toArray(new FileStatus[status.size()]);
}
private FileStatus newFile(FileMetadata meta, Path path) {
return new FileStatus(meta.getLength(), false, 1, getDefaultBlockSize(),
meta.getLastModified(), path.makeQualified(this.getUri(), this.getWorkingDirectory()));
}
private FileStatus newDirectory(Path path) {
return new FileStatus(0, true, 1, 0, 0, path.makeQualified(this.getUri(), this.getWorkingDirectory()));
}
@Override
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
Path absolutePath = makeAbsolute(f);
List<Path> paths = new ArrayList<Path>();
do {
paths.add(0, absolutePath);
absolutePath = absolutePath.getParent();
} while (absolutePath != null);
boolean result = true;
for (Path path : paths) {
result &= mkdir(path);
}
return result;
}
private boolean mkdir(Path f) throws IOException {
try {
FileStatus fileStatus = getFileStatus(f);
if (fileStatus.isFile()) {
throw new FileAlreadyExistsException(String.format(
"Can't make directory for path '%s' since it is a file.", f));
}
} catch (FileNotFoundException e) {
if(LOG.isDebugEnabled()) {
LOG.debug("Making dir '" + f + "' in S3");
}
String key = pathToKey(f) + FOLDER_SUFFIX;
store.storeEmptyFile(key);
}
return true;
}
@Override
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
FileStatus fs = getFileStatus(f); // will throw if the file doesn't exist
if (fs.isDirectory()) {
throw new FileNotFoundException("'" + f + "' is a directory");
}
LOG.info("Opening '" + f + "' for reading");
Path absolutePath = makeAbsolute(f);
String key = pathToKey(absolutePath);
return new FSDataInputStream(new BufferedFSInputStream(
new NativeS3FsInputStream(store, statistics, store.retrieve(key), key), bufferSize));
}
// rename() and delete() use this method to ensure that the parent directory
// of the source does not vanish.
private void createParent(Path path) throws IOException {
Path parent = path.getParent();
if (parent != null) {
String key = pathToKey(makeAbsolute(parent));
if (key.length() > 0) {
store.storeEmptyFile(key + FOLDER_SUFFIX);
}
}
}
@Override
public boolean rename(Path src, Path dst) throws IOException {
String srcKey = pathToKey(makeAbsolute(src));
final String debugPreamble = "Renaming '" + src + "' to '" + dst + "' - ";
if (srcKey.length() == 0) {
// Cannot rename root of file system
if (LOG.isDebugEnabled()) {
LOG.debug(debugPreamble +
"returning false as cannot rename the root of a filesystem");
}
return false;
}
//get status of source
boolean srcIsFile;
try {
srcIsFile = getFileStatus(src).isFile();
} catch (FileNotFoundException e) {
//bail out fast if the source does not exist
if (LOG.isDebugEnabled()) {
LOG.debug(debugPreamble + "returning false as src does not exist");
}
return false;
}
// Figure out the final destination
String dstKey = pathToKey(makeAbsolute(dst));
try {
boolean dstIsFile = getFileStatus(dst).isFile();
if (dstIsFile) {
//destination is a file.
//you can't copy a file or a directory onto an existing file
//except for the special case of dest==src, which is a no-op
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble +
"returning without rename as dst is an already existing file");
}
//exit, returning true iff the rename is onto self
return srcKey.equals(dstKey);
} else {
//destination exists and is a directory
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble + "using dst as output directory");
}
//destination goes under the dst path, with the name of the
//source entry
dstKey = pathToKey(makeAbsolute(new Path(dst, src.getName())));
}
} catch (FileNotFoundException e) {
//destination does not exist => the source file or directory
//is copied over with the name of the destination
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble + "using dst as output destination");
}
try {
if (getFileStatus(dst.getParent()).isFile()) {
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble +
"returning false as dst parent exists and is a file");
}
return false;
}
} catch (FileNotFoundException ex) {
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble +
"returning false as dst parent does not exist");
}
return false;
}
}
//rename to self behavior follows Posix rules and is different
//for directories and files -the return code is driven by src type
if (srcKey.equals(dstKey)) {
//fully resolved destination key matches source: fail
if (LOG.isDebugEnabled()) {
LOG.debug(debugPreamble + "renamingToSelf; returning true");
}
return true;
}
if (srcIsFile) {
//source is a file; COPY then DELETE
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble +
"src is file, so doing copy then delete in S3");
}
store.copy(srcKey, dstKey);
store.delete(srcKey);
} else {
//src is a directory
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble + "src is directory, so copying contents");
}
//Verify dest is not a child of the parent
if (dstKey.startsWith(srcKey + "/")) {
if (LOG.isDebugEnabled()) {
LOG.debug(
debugPreamble + "cannot rename a directory to a subdirectory of self");
}
return false;
}
//create the subdir under the destination
store.storeEmptyFile(dstKey + FOLDER_SUFFIX);
List<String> keysToDelete = new ArrayList<String>();
String priorLastKey = null;
do {
PartialListing listing = store.list(srcKey, S3_MAX_LISTING_LENGTH, priorLastKey, true);
for (FileMetadata file : listing.getFiles()) {
keysToDelete.add(file.getKey());
store.copy(file.getKey(), dstKey + file.getKey().substring(srcKey.length()));
}
priorLastKey = listing.getPriorLastKey();
} while (priorLastKey != null);
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble +
"all files in src copied, now removing src files");
}
for (String key: keysToDelete) {
store.delete(key);
}
try {
store.delete(srcKey + FOLDER_SUFFIX);
} catch (FileNotFoundException e) {
//this is fine, we don't require a marker
}
if(LOG.isDebugEnabled()) {
LOG.debug(debugPreamble + "done");
}
}
return true;
}
@Override
public long getDefaultBlockSize() {
return getConf().getLong("fs.s3n.block.size", 64 * 1024 * 1024);
}
/**
* Set the working directory to the given directory.
*/
@Override
public void setWorkingDirectory(Path newDir) {
workingDir = newDir;
}
@Override
public Path getWorkingDirectory() {
return workingDir;
}
@Override
public String getCanonicalServiceName() {
// Does not support Token
return null;
}
}