|
@@ -27,6 +27,8 @@ import java.util.ArrayList;
|
|
|
import java.util.Date;
|
|
|
import java.util.List;
|
|
|
|
|
|
+import org.apache.hadoop.fs.s3.S3Credentials;
|
|
|
+
|
|
|
import com.amazonaws.AmazonClientException;
|
|
|
import com.amazonaws.AmazonServiceException;
|
|
|
import com.amazonaws.ClientConfiguration;
|
|
@@ -80,6 +82,8 @@ public class S3AFileSystem extends FileSystem {
|
|
|
private CannedAccessControlList cannedACL;
|
|
|
private String serverSideEncryptionAlgorithm;
|
|
|
|
|
|
+ // The maximum number of entries that can be deleted in any call to s3
|
|
|
+ private static final int MAX_ENTRIES_TO_DELETE = 1000;
|
|
|
|
|
|
/** Called after a new FileSystem instance is constructed.
|
|
|
* @param name a uri whose authority section names the host, port, etc.
|
|
@@ -95,22 +99,12 @@ public class S3AFileSystem extends FileSystem {
|
|
|
this.getWorkingDirectory());
|
|
|
|
|
|
// Try to get our credentials or just connect anonymously
|
|
|
- String accessKey = conf.get(ACCESS_KEY, null);
|
|
|
- String secretKey = conf.get(SECRET_KEY, null);
|
|
|
-
|
|
|
- String userInfo = name.getUserInfo();
|
|
|
- if (userInfo != null) {
|
|
|
- int index = userInfo.indexOf(':');
|
|
|
- if (index != -1) {
|
|
|
- accessKey = userInfo.substring(0, index);
|
|
|
- secretKey = userInfo.substring(index + 1);
|
|
|
- } else {
|
|
|
- accessKey = userInfo;
|
|
|
- }
|
|
|
- }
|
|
|
+ S3Credentials s3Credentials = new S3Credentials();
|
|
|
+ s3Credentials.initialize(name, conf);
|
|
|
|
|
|
AWSCredentialsProviderChain credentials = new AWSCredentialsProviderChain(
|
|
|
- new BasicAWSCredentialsProvider(accessKey, secretKey),
|
|
|
+ new BasicAWSCredentialsProvider(s3Credentials.getAccessKey(),
|
|
|
+ s3Credentials.getSecretAccessKey()),
|
|
|
new InstanceProfileCredentialsProvider(),
|
|
|
new AnonymousAWSCredentialsProvider()
|
|
|
);
|
|
@@ -295,15 +289,12 @@ public class S3AFileSystem extends FileSystem {
|
|
|
String dstKey = pathToKey(dst);
|
|
|
|
|
|
if (srcKey.length() == 0 || dstKey.length() == 0) {
|
|
|
- LOG.info("rename: src or dst are empty");
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("rename: src or dst are empty");
|
|
|
+ }
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- if (srcKey.equals(dstKey)) {
|
|
|
- LOG.info("rename: src and dst refer to the same file");
|
|
|
- return true;
|
|
|
- }
|
|
|
-
|
|
|
S3AFileStatus srcStatus;
|
|
|
try {
|
|
|
srcStatus = getFileStatus(src);
|
|
@@ -312,20 +303,27 @@ public class S3AFileSystem extends FileSystem {
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
+ if (srcKey.equals(dstKey)) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("rename: src and dst refer to the same file or directory");
|
|
|
+ }
|
|
|
+ return srcStatus.isFile();
|
|
|
+ }
|
|
|
+
|
|
|
S3AFileStatus dstStatus = null;
|
|
|
try {
|
|
|
dstStatus = getFileStatus(dst);
|
|
|
|
|
|
- if (srcStatus.isFile() && dstStatus.isDirectory()) {
|
|
|
- LOG.info("rename: src is a file and dst is a directory");
|
|
|
+ if (srcStatus.isDirectory() && dstStatus.isFile()) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("rename: src is a directory and dst is a file");
|
|
|
+ }
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- if (srcStatus.isDirectory() && dstStatus.isFile()) {
|
|
|
- LOG.info("rename: src is a directory and dst is a file");
|
|
|
+ if (dstStatus.isDirectory() && !dstStatus.isEmptyDirectory()) {
|
|
|
return false;
|
|
|
}
|
|
|
-
|
|
|
} catch (FileNotFoundException e) {
|
|
|
// Parent must exist
|
|
|
Path parent = dst.getParent();
|
|
@@ -346,7 +344,18 @@ public class S3AFileSystem extends FileSystem {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("rename: renaming file " + src + " to " + dst);
|
|
|
}
|
|
|
- copyFile(srcKey, dstKey);
|
|
|
+ if (dstStatus != null && dstStatus.isDirectory()) {
|
|
|
+ String newDstKey = dstKey;
|
|
|
+ if (!newDstKey.endsWith("/")) {
|
|
|
+ newDstKey = newDstKey + "/";
|
|
|
+ }
|
|
|
+ String filename =
|
|
|
+ srcKey.substring(pathToKey(src.getParent()).length()+1);
|
|
|
+ newDstKey = newDstKey + filename;
|
|
|
+ copyFile(srcKey, newDstKey);
|
|
|
+ } else {
|
|
|
+ copyFile(srcKey, dstKey);
|
|
|
+ }
|
|
|
delete(src, false);
|
|
|
} else {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -362,12 +371,19 @@ public class S3AFileSystem extends FileSystem {
|
|
|
srcKey = srcKey + "/";
|
|
|
}
|
|
|
|
|
|
+ //Verify dest is not a child of the source directory
|
|
|
+ if (dstKey.startsWith(srcKey)) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("cannot rename a directory to a subdirectory of self");
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
List<DeleteObjectsRequest.KeyVersion> keysToDelete =
|
|
|
new ArrayList<DeleteObjectsRequest.KeyVersion>();
|
|
|
if (dstStatus != null && dstStatus.isEmptyDirectory()) {
|
|
|
- copyFile(srcKey, dstKey);
|
|
|
- statistics.incrementWriteOps(1);
|
|
|
- keysToDelete.add(new DeleteObjectsRequest.KeyVersion(srcKey));
|
|
|
+ // delete unnecessary fake directory.
|
|
|
+ keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey));
|
|
|
}
|
|
|
|
|
|
ListObjectsRequest request = new ListObjectsRequest();
|
|
@@ -383,23 +399,29 @@ public class S3AFileSystem extends FileSystem {
|
|
|
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(summary.getKey()));
|
|
|
String newDstKey = dstKey + summary.getKey().substring(srcKey.length());
|
|
|
copyFile(summary.getKey(), newDstKey);
|
|
|
+
|
|
|
+ if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
|
|
|
+ DeleteObjectsRequest deleteRequest =
|
|
|
+ new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
|
|
|
+ s3.deleteObjects(deleteRequest);
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ keysToDelete.clear();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
if (objects.isTruncated()) {
|
|
|
objects = s3.listNextBatchOfObjects(objects);
|
|
|
statistics.incrementReadOps(1);
|
|
|
} else {
|
|
|
+ if (keysToDelete.size() > 0) {
|
|
|
+ DeleteObjectsRequest deleteRequest =
|
|
|
+ new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
|
|
|
+ s3.deleteObjects(deleteRequest);
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ }
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
- if (!keysToDelete.isEmpty()) {
|
|
|
- DeleteObjectsRequest deleteRequest = new DeleteObjectsRequest(bucket);
|
|
|
- deleteRequest.setKeys(keysToDelete);
|
|
|
- s3.deleteObjects(deleteRequest);
|
|
|
- statistics.incrementWriteOps(1);
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
if (src.getParent() != dst.getParent()) {
|
|
@@ -419,7 +441,9 @@ public class S3AFileSystem extends FileSystem {
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
public boolean delete(Path f, boolean recursive) throws IOException {
|
|
|
- LOG.info("Delete path " + f + " - recursive " + recursive);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Delete path " + f + " - recursive " + recursive);
|
|
|
+ }
|
|
|
S3AFileStatus status;
|
|
|
try {
|
|
|
status = getFileStatus(f);
|
|
@@ -479,18 +503,26 @@ public class S3AFileSystem extends FileSystem {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Got object to delete " + summary.getKey());
|
|
|
}
|
|
|
- }
|
|
|
|
|
|
- DeleteObjectsRequest deleteRequest = new DeleteObjectsRequest(bucket);
|
|
|
- deleteRequest.setKeys(keys);
|
|
|
- s3.deleteObjects(deleteRequest);
|
|
|
- statistics.incrementWriteOps(1);
|
|
|
- keys.clear();
|
|
|
+ if (keys.size() == MAX_ENTRIES_TO_DELETE) {
|
|
|
+ DeleteObjectsRequest deleteRequest =
|
|
|
+ new DeleteObjectsRequest(bucket).withKeys(keys);
|
|
|
+ s3.deleteObjects(deleteRequest);
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ keys.clear();
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
if (objects.isTruncated()) {
|
|
|
objects = s3.listNextBatchOfObjects(objects);
|
|
|
statistics.incrementReadOps(1);
|
|
|
} else {
|
|
|
+ if (keys.size() > 0) {
|
|
|
+ DeleteObjectsRequest deleteRequest =
|
|
|
+ new DeleteObjectsRequest(bucket).withKeys(keys);
|
|
|
+ s3.deleteObjects(deleteRequest);
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ }
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
@@ -530,7 +562,9 @@ public class S3AFileSystem extends FileSystem {
|
|
|
public FileStatus[] listStatus(Path f) throws FileNotFoundException,
|
|
|
IOException {
|
|
|
String key = pathToKey(f);
|
|
|
- LOG.info("List status for path: " + f);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("List status for path: " + f);
|
|
|
+ }
|
|
|
|
|
|
final List<FileStatus> result = new ArrayList<FileStatus>();
|
|
|
final FileStatus fileStatus = getFileStatus(f);
|
|
@@ -640,7 +674,10 @@ public class S3AFileSystem extends FileSystem {
|
|
|
// TODO: If we have created an empty file at /foo/bar and we then call
|
|
|
// mkdirs for /foo/bar/baz/roo what happens to the empty file /foo/bar/?
|
|
|
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
|
|
|
- LOG.info("Making directory: " + f);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Making directory: " + f);
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
try {
|
|
|
FileStatus fileStatus = getFileStatus(f);
|
|
@@ -680,8 +717,10 @@ public class S3AFileSystem extends FileSystem {
|
|
|
*/
|
|
|
public S3AFileStatus getFileStatus(Path f) throws IOException {
|
|
|
String key = pathToKey(f);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Getting path status for " + f + " (" + key + ")");
|
|
|
+ }
|
|
|
|
|
|
- LOG.info("Getting path status for " + f + " (" + key + ")");
|
|
|
|
|
|
if (!key.isEmpty()) {
|
|
|
try {
|
|
@@ -723,7 +762,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
return new S3AFileStatus(true, true, f.makeQualified(uri, workingDir));
|
|
|
} else {
|
|
|
- LOG.warn("Found file (with /): real file? should not happen: " + key);
|
|
|
+ LOG.warn("Found file (with /): real file? should not happen: {}", key);
|
|
|
|
|
|
return new S3AFileStatus(meta.getContentLength(), dateToLong(meta.getLastModified()),
|
|
|
f.makeQualified(uri, workingDir));
|
|
@@ -753,7 +792,8 @@ public class S3AFileSystem extends FileSystem {
|
|
|
ObjectListing objects = s3.listObjects(request);
|
|
|
statistics.incrementReadOps(1);
|
|
|
|
|
|
- if (objects.getCommonPrefixes().size() > 0 || objects.getObjectSummaries().size() > 0) {
|
|
|
+ if (!objects.getCommonPrefixes().isEmpty()
|
|
|
+ || objects.getObjectSummaries().size() > 0) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Found path as directory (with /): " +
|
|
|
objects.getCommonPrefixes().size() + "/" +
|
|
@@ -806,8 +846,9 @@ public class S3AFileSystem extends FileSystem {
|
|
|
if (!overwrite && exists(dst)) {
|
|
|
throw new IOException(dst + " already exists");
|
|
|
}
|
|
|
-
|
|
|
- LOG.info("Copying local file from " + src + " to " + dst);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Copying local file from " + src + " to " + dst);
|
|
|
+ }
|
|
|
|
|
|
// Since we have a local file, we don't need to stream into a temporary file
|
|
|
LocalFileSystem local = getLocal(getConf());
|
|
@@ -992,7 +1033,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
@Deprecated
|
|
|
public long getDefaultBlockSize() {
|
|
|
// default to 32MB: large enough to minimize the impact of seeks
|
|
|
- return getConf().getLong("fs.s3a.block.size", 32 * 1024 * 1024);
|
|
|
+ return getConf().getLong(FS_S3A_BLOCK_SIZE, 32 * 1024 * 1024);
|
|
|
}
|
|
|
|
|
|
private void printAmazonServiceException(AmazonServiceException ase) {
|
|
@@ -1010,6 +1051,6 @@ public class S3AFileSystem extends FileSystem {
|
|
|
LOG.info("Caught an AmazonClientException, which means the client encountered " +
|
|
|
"a serious internal problem while trying to communicate with S3, " +
|
|
|
"such as not being able to access the network.");
|
|
|
- LOG.info("Error Message: " + ace.getMessage());
|
|
|
+ LOG.info("Error Message: {}" + ace, ace);
|
|
|
}
|
|
|
}
|