|
@@ -43,6 +43,7 @@ import com.amazonaws.auth.AWSCredentialsProviderChain;
|
|
|
import com.amazonaws.auth.InstanceProfileCredentialsProvider;
|
|
|
import com.amazonaws.services.s3.AmazonS3Client;
|
|
|
import com.amazonaws.services.s3.S3ClientOptions;
|
|
|
+import com.amazonaws.services.s3.model.AmazonS3Exception;
|
|
|
import com.amazonaws.services.s3.model.CannedAccessControlList;
|
|
|
import com.amazonaws.services.s3.model.DeleteObjectRequest;
|
|
|
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
|
@@ -78,6 +79,7 @@ import org.apache.hadoop.util.Progressable;
|
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
|
|
|
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
|
|
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
|
|
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -181,93 +183,122 @@ public class S3AFileSystem extends FileSystem {
|
|
|
public void initialize(URI name, Configuration conf) throws IOException {
|
|
|
super.initialize(name, conf);
|
|
|
setConf(conf);
|
|
|
- instrumentation = new S3AInstrumentation(name);
|
|
|
-
|
|
|
- uri = URI.create(name.getScheme() + "://" + name.getAuthority());
|
|
|
- workingDir = new Path("/user", System.getProperty("user.name"))
|
|
|
- .makeQualified(this.uri, this.getWorkingDirectory());
|
|
|
-
|
|
|
- bucket = name.getHost();
|
|
|
-
|
|
|
- AWSCredentialsProvider credentials = getAWSCredentialsProvider(name, conf);
|
|
|
-
|
|
|
- ClientConfiguration awsConf = new ClientConfiguration();
|
|
|
- awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
|
|
|
- DEFAULT_MAXIMUM_CONNECTIONS, 1));
|
|
|
- boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
|
|
|
- DEFAULT_SECURE_CONNECTIONS);
|
|
|
- awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
|
|
|
- awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
|
|
|
- DEFAULT_MAX_ERROR_RETRIES, 0));
|
|
|
- awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
|
|
|
- DEFAULT_ESTABLISH_TIMEOUT, 0));
|
|
|
- awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
|
|
|
- DEFAULT_SOCKET_TIMEOUT, 0));
|
|
|
- String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
|
|
|
- if (!signerOverride.isEmpty()) {
|
|
|
- LOG.debug("Signer override = {}", signerOverride);
|
|
|
- awsConf.setSignerOverride(signerOverride);
|
|
|
- }
|
|
|
+ try {
|
|
|
+ instrumentation = new S3AInstrumentation(name);
|
|
|
+
|
|
|
+ uri = URI.create(name.getScheme() + "://" + name.getAuthority());
|
|
|
+ workingDir = new Path("/user", System.getProperty("user.name"))
|
|
|
+ .makeQualified(this.uri, this.getWorkingDirectory());
|
|
|
+
|
|
|
+ bucket = name.getHost();
|
|
|
+
|
|
|
+ AWSCredentialsProvider credentials =
|
|
|
+ getAWSCredentialsProvider(name, conf);
|
|
|
+
|
|
|
+ ClientConfiguration awsConf = new ClientConfiguration();
|
|
|
+ awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
|
|
|
+ DEFAULT_MAXIMUM_CONNECTIONS, 1));
|
|
|
+ boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
|
|
|
+ DEFAULT_SECURE_CONNECTIONS);
|
|
|
+ awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
|
|
|
+ awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
|
|
|
+ DEFAULT_MAX_ERROR_RETRIES, 0));
|
|
|
+ awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
|
|
|
+ DEFAULT_ESTABLISH_TIMEOUT, 0));
|
|
|
+ awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
|
|
|
+ DEFAULT_SOCKET_TIMEOUT, 0));
|
|
|
+ String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
|
|
|
+ if (!signerOverride.isEmpty()) {
|
|
|
+ LOG.debug("Signer override = {}", signerOverride);
|
|
|
+ awsConf.setSignerOverride(signerOverride);
|
|
|
+ }
|
|
|
|
|
|
- initProxySupport(conf, awsConf, secureConnections);
|
|
|
+ initProxySupport(conf, awsConf, secureConnections);
|
|
|
|
|
|
- initUserAgent(conf, awsConf);
|
|
|
+ initUserAgent(conf, awsConf);
|
|
|
|
|
|
- initAmazonS3Client(conf, credentials, awsConf);
|
|
|
+ initAmazonS3Client(conf, credentials, awsConf);
|
|
|
|
|
|
- maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
|
|
|
- partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
|
|
|
- if (partSize < 5 * 1024 * 1024) {
|
|
|
- LOG.error(MULTIPART_SIZE + " must be at least 5 MB");
|
|
|
- partSize = 5 * 1024 * 1024;
|
|
|
- }
|
|
|
+ maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
|
|
|
+ partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
|
|
|
+ if (partSize < 5 * 1024 * 1024) {
|
|
|
+ LOG.error(MULTIPART_SIZE + " must be at least 5 MB");
|
|
|
+ partSize = 5 * 1024 * 1024;
|
|
|
+ }
|
|
|
|
|
|
- multiPartThreshold = conf.getLong(MIN_MULTIPART_THRESHOLD,
|
|
|
- DEFAULT_MIN_MULTIPART_THRESHOLD);
|
|
|
- if (multiPartThreshold < 5 * 1024 * 1024) {
|
|
|
- LOG.error(MIN_MULTIPART_THRESHOLD + " must be at least 5 MB");
|
|
|
- multiPartThreshold = 5 * 1024 * 1024;
|
|
|
- }
|
|
|
- //check but do not store the block size
|
|
|
- longOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
|
|
|
- enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
|
|
|
+ multiPartThreshold = conf.getLong(MIN_MULTIPART_THRESHOLD,
|
|
|
+ DEFAULT_MIN_MULTIPART_THRESHOLD);
|
|
|
+ if (multiPartThreshold < 5 * 1024 * 1024) {
|
|
|
+ LOG.error(MIN_MULTIPART_THRESHOLD + " must be at least 5 MB");
|
|
|
+ multiPartThreshold = 5 * 1024 * 1024;
|
|
|
+ }
|
|
|
+ //check but do not store the block size
|
|
|
+ longOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
|
|
|
+ enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
|
|
|
|
|
|
- readAhead = longOption(conf, READAHEAD_RANGE, DEFAULT_READAHEAD_RANGE, 0);
|
|
|
+ readAhead = longOption(conf, READAHEAD_RANGE, DEFAULT_READAHEAD_RANGE, 0);
|
|
|
|
|
|
- int maxThreads = intOption(conf, MAX_THREADS, DEFAULT_MAX_THREADS, 0);
|
|
|
- int coreThreads = intOption(conf, CORE_THREADS, DEFAULT_CORE_THREADS, 0);
|
|
|
- if (maxThreads == 0) {
|
|
|
- maxThreads = Runtime.getRuntime().availableProcessors() * 8;
|
|
|
- }
|
|
|
- if (coreThreads == 0) {
|
|
|
- coreThreads = Runtime.getRuntime().availableProcessors() * 8;
|
|
|
+ int maxThreads = intOption(conf, MAX_THREADS, DEFAULT_MAX_THREADS, 0);
|
|
|
+ int coreThreads = intOption(conf, CORE_THREADS, DEFAULT_CORE_THREADS, 0);
|
|
|
+ if (maxThreads == 0) {
|
|
|
+ maxThreads = Runtime.getRuntime().availableProcessors() * 8;
|
|
|
+ }
|
|
|
+ if (coreThreads == 0) {
|
|
|
+ coreThreads = Runtime.getRuntime().availableProcessors() * 8;
|
|
|
+ }
|
|
|
+ long keepAliveTime = longOption(conf, KEEPALIVE_TIME,
|
|
|
+ DEFAULT_KEEPALIVE_TIME, 0);
|
|
|
+ LinkedBlockingQueue<Runnable> workQueue =
|
|
|
+ new LinkedBlockingQueue<>(maxThreads *
|
|
|
+ intOption(conf, MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1));
|
|
|
+ threadPoolExecutor = new ThreadPoolExecutor(
|
|
|
+ coreThreads,
|
|
|
+ maxThreads,
|
|
|
+ keepAliveTime,
|
|
|
+ TimeUnit.SECONDS,
|
|
|
+ workQueue,
|
|
|
+ newDaemonThreadFactory("s3a-transfer-shared-"));
|
|
|
+ threadPoolExecutor.allowCoreThreadTimeOut(true);
|
|
|
+
|
|
|
+ initTransferManager();
|
|
|
+
|
|
|
+ initCannedAcls(conf);
|
|
|
+
|
|
|
+ verifyBucketExists();
|
|
|
+
|
|
|
+ initMultipartUploads(conf);
|
|
|
+
|
|
|
+ serverSideEncryptionAlgorithm =
|
|
|
+ conf.getTrimmed(SERVER_SIDE_ENCRYPTION_ALGORITHM);
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("initializing ", new Path(name), e);
|
|
|
}
|
|
|
- long keepAliveTime = longOption(conf, KEEPALIVE_TIME,
|
|
|
- DEFAULT_KEEPALIVE_TIME, 0);
|
|
|
- LinkedBlockingQueue<Runnable> workQueue =
|
|
|
- new LinkedBlockingQueue<>(maxThreads *
|
|
|
- intOption(conf, MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1));
|
|
|
- threadPoolExecutor = new ThreadPoolExecutor(
|
|
|
- coreThreads,
|
|
|
- maxThreads,
|
|
|
- keepAliveTime,
|
|
|
- TimeUnit.SECONDS,
|
|
|
- workQueue,
|
|
|
- newDaemonThreadFactory("s3a-transfer-shared-"));
|
|
|
- threadPoolExecutor.allowCoreThreadTimeOut(true);
|
|
|
|
|
|
- initTransferManager();
|
|
|
-
|
|
|
- initCannedAcls(conf);
|
|
|
+ }
|
|
|
|
|
|
- if (!s3.doesBucketExist(bucket)) {
|
|
|
- throw new FileNotFoundException("Bucket " + bucket + " does not exist");
|
|
|
+ /**
|
|
|
+ * Verify that the bucket exists. This does not check permissions,
|
|
|
+ * not even read access.
|
|
|
+ * @throws FileNotFoundException the bucket is absent
|
|
|
+ * @throws IOException any other problem talking to S3
|
|
|
+ */
|
|
|
+ protected void verifyBucketExists()
|
|
|
+ throws FileNotFoundException, IOException {
|
|
|
+ try {
|
|
|
+ if (!s3.doesBucketExist(bucket)) {
|
|
|
+ throw new FileNotFoundException("Bucket " + bucket + " does not exist");
|
|
|
+ }
|
|
|
+ } catch (AmazonS3Exception e) {
|
|
|
+ // this is a sign of a serious startup problem so do dump everything
|
|
|
+ LOG.warn(stringify(e), e);
|
|
|
+ throw translateException("doesBucketExist", bucket, e);
|
|
|
+ } catch (AmazonServiceException e) {
|
|
|
+ // this is a sign of a serious startup problem so do dump everything
|
|
|
+ LOG.warn(stringify(e), e);
|
|
|
+ throw translateException("doesBucketExist", bucket, e);
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("doesBucketExist", bucket, e);
|
|
|
}
|
|
|
-
|
|
|
- initMultipartUploads(conf);
|
|
|
-
|
|
|
- serverSideEncryptionAlgorithm = conf.get(SERVER_SIDE_ENCRYPTION_ALGORITHM);
|
|
|
-
|
|
|
}
|
|
|
|
|
|
void initProxySupport(Configuration conf, ClientConfiguration awsConf,
|
|
@@ -379,7 +410,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void initMultipartUploads(Configuration conf) {
|
|
|
+ private void initMultipartUploads(Configuration conf) throws IOException {
|
|
|
boolean purgeExistingMultipart = conf.getBoolean(PURGE_EXISTING_MULTIPART,
|
|
|
DEFAULT_PURGE_EXISTING_MULTIPART);
|
|
|
long purgeExistingMultipartAge = longOption(conf,
|
|
@@ -394,10 +425,10 @@ public class S3AFileSystem extends FileSystem {
|
|
|
} catch (AmazonServiceException e) {
|
|
|
if (e.getStatusCode() == 403) {
|
|
|
instrumentation.errorIgnored();
|
|
|
- LOG.debug("Failed to abort multipart uploads against {}," +
|
|
|
+ LOG.debug("Failed to purging multipart uploads against {}," +
|
|
|
" FS may be read only", bucket, e);
|
|
|
} else {
|
|
|
- throw e;
|
|
|
+ throw translateException("purging multipart uploads", bucket, e);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -639,10 +670,28 @@ public class S3AFileSystem extends FileSystem {
|
|
|
*
|
|
|
* @param src path to be renamed
|
|
|
* @param dst new path after rename
|
|
|
- * @throws IOException on failure
|
|
|
+ * @throws IOException on IO failure
|
|
|
* @return true if rename is successful
|
|
|
*/
|
|
|
public boolean rename(Path src, Path dst) throws IOException {
|
|
|
+ try {
|
|
|
+ return innerRename(src, dst);
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("rename(" + src +", " + dst + ")", src, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The inner rename operation. See {@link #rename(Path, Path)} for
|
|
|
+ * the description of the operation.
|
|
|
+ * @param src path to be renamed
|
|
|
+ * @param dst new path after rename
|
|
|
+ * @return true if rename is successful
|
|
|
+ * @throws IOException on IO failure.
|
|
|
+ * @throws AmazonClientException on failures inside the AWS SDK
|
|
|
+ */
|
|
|
+ private boolean innerRename(Path src, Path dst) throws IOException,
|
|
|
+ AmazonClientException {
|
|
|
LOG.debug("Rename path {} to {}", src, dst);
|
|
|
|
|
|
String srcKey = pathToKey(src);
|
|
@@ -785,7 +834,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
* when set to true
|
|
|
*/
|
|
|
private void removeKeys(List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
- boolean clearKeys) {
|
|
|
+ boolean clearKeys) throws AmazonClientException {
|
|
|
if (enableMultiObjectsDelete) {
|
|
|
DeleteObjectsRequest deleteRequest
|
|
|
= new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
|
|
@@ -808,7 +857,9 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** Delete a file.
|
|
|
+ /**
|
|
|
+ * Delete a Path. This operation is at least {@code O(files)}, with
|
|
|
+ * added overheads to enumerate the path. It is also not atomic.
|
|
|
*
|
|
|
* @param f the path to delete.
|
|
|
* @param recursive if path is a directory and set to
|
|
@@ -818,6 +869,26 @@ public class S3AFileSystem extends FileSystem {
|
|
|
* @throws IOException due to inability to delete a directory or file.
|
|
|
*/
|
|
|
public boolean delete(Path f, boolean recursive) throws IOException {
|
|
|
+ try {
|
|
|
+ return innerDelete(f, recursive);
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("delete", f, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Delete a path. See {@link #delete(Path, boolean)}.
|
|
|
+ *
|
|
|
+ * @param f the path to delete.
|
|
|
+ * @param recursive if path is a directory and set to
|
|
|
+ * true, the directory is deleted else throws an exception. In
|
|
|
+ * case of a file the recursive can be set to either true or false.
|
|
|
+ * @return true if delete is successful else false.
|
|
|
+ * @throws IOException due to inability to delete a directory or file.
|
|
|
+ * @throws AmazonClientException on failures inside the AWS SDK
|
|
|
+ */
|
|
|
+ private boolean innerDelete(Path f, boolean recursive) throws IOException,
|
|
|
+ AmazonClientException {
|
|
|
LOG.debug("Delete path {} - recursive {}", f , recursive);
|
|
|
S3AFileStatus status;
|
|
|
try {
|
|
@@ -898,7 +969,8 @@ public class S3AFileSystem extends FileSystem {
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- private void createFakeDirectoryIfNecessary(Path f) throws IOException {
|
|
|
+ private void createFakeDirectoryIfNecessary(Path f)
|
|
|
+ throws IOException, AmazonClientException {
|
|
|
String key = pathToKey(f);
|
|
|
if (!key.isEmpty() && !exists(f)) {
|
|
|
LOG.debug("Creating new fake directory at {}", f);
|
|
@@ -917,6 +989,25 @@ public class S3AFileSystem extends FileSystem {
|
|
|
*/
|
|
|
public FileStatus[] listStatus(Path f) throws FileNotFoundException,
|
|
|
IOException {
|
|
|
+ try {
|
|
|
+ return innerListStatus(f);
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("listStatus", f, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * List the statuses of the files/directories in the given path if the path is
|
|
|
+ * a directory.
|
|
|
+ *
|
|
|
+ * @param f given path
|
|
|
+ * @return the statuses of the files/directories in the given patch
|
|
|
+ * @throws FileNotFoundException when the path does not exist;
|
|
|
+ * @throws IOException due to an IO problem.
|
|
|
+ * @throws AmazonClientException on failures inside the AWS SDK
|
|
|
+ */
|
|
|
+ public FileStatus[] innerListStatus(Path f) throws FileNotFoundException,
|
|
|
+ IOException, AmazonClientException {
|
|
|
String key = pathToKey(f);
|
|
|
LOG.debug("List status for path: {}", f);
|
|
|
|
|
@@ -1008,15 +1099,42 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Make the given file and all non-existent parents into
|
|
|
- * directories. Has the semantics of Unix 'mkdir -p'.
|
|
|
+ *
|
|
|
+ * Make the given path and all non-existent parents into
|
|
|
+ * directories. Has the semantics of Unix @{code 'mkdir -p'}.
|
|
|
* Existence of the directory hierarchy is not an error.
|
|
|
+ * @param path path to create
|
|
|
+ * @param permission to apply to f
|
|
|
+ * @return true if a directory was created
|
|
|
+ * @throws FileAlreadyExistsException there is a file at the path specified
|
|
|
+ * @throws IOException other IO problems
|
|
|
+ */
|
|
|
+ // 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 path, FsPermission permission) throws IOException,
|
|
|
+ FileAlreadyExistsException {
|
|
|
+ try {
|
|
|
+ return innerMkdirs(path, permission);
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("innerMkdirs", path, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ /**
|
|
|
+ *
|
|
|
+ * Make the given path and all non-existent parents into
|
|
|
+ * directories.
|
|
|
+ * See {@link #mkdirs(Path, FsPermission)}
|
|
|
* @param f path to create
|
|
|
* @param permission to apply to f
|
|
|
+ * @return true if a directory was created
|
|
|
+ * @throws FileAlreadyExistsException there is a file at the path specified
|
|
|
+ * @throws IOException other IO problems
|
|
|
+ * @throws AmazonClientException on failures inside the AWS SDK
|
|
|
*/
|
|
|
// 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 {
|
|
|
+ private boolean innerMkdirs(Path f, FsPermission permission)
|
|
|
+ throws IOException, FileAlreadyExistsException, AmazonClientException {
|
|
|
LOG.debug("Making directory: {}", f);
|
|
|
|
|
|
try {
|
|
@@ -1054,7 +1172,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
* @param f The path we want information from
|
|
|
* @return a FileStatus object
|
|
|
* @throws java.io.FileNotFoundException when the path does not exist;
|
|
|
- * IOException see specific implementation
|
|
|
+ * @throws IOException on other problems.
|
|
|
*/
|
|
|
public S3AFileStatus getFileStatus(Path f) throws IOException {
|
|
|
String key = pathToKey(f);
|
|
@@ -1078,12 +1196,10 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
} catch (AmazonServiceException e) {
|
|
|
if (e.getStatusCode() != 404) {
|
|
|
- printAmazonServiceException(f.toString(), e);
|
|
|
- throw e;
|
|
|
+ throw translateException("getFileStatus", f, e);
|
|
|
}
|
|
|
} catch (AmazonClientException e) {
|
|
|
- printAmazonClientException(f.toString(), e);
|
|
|
- throw e;
|
|
|
+ throw translateException("getFileStatus", f, e);
|
|
|
}
|
|
|
|
|
|
// Necessary?
|
|
@@ -1106,12 +1222,10 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
} catch (AmazonServiceException e) {
|
|
|
if (e.getStatusCode() != 404) {
|
|
|
- printAmazonServiceException(newKey, e);
|
|
|
- throw e;
|
|
|
+ throw translateException("getFileStatus", newKey, e);
|
|
|
}
|
|
|
} catch (AmazonClientException e) {
|
|
|
- printAmazonClientException(newKey, e);
|
|
|
- throw e;
|
|
|
+ throw translateException("getFileStatus", newKey, e);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1152,12 +1266,10 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
} catch (AmazonServiceException e) {
|
|
|
if (e.getStatusCode() != 404) {
|
|
|
- printAmazonServiceException(key, e);
|
|
|
- throw e;
|
|
|
+ throw translateException("getFileStatus", key, e);
|
|
|
}
|
|
|
} catch (AmazonClientException e) {
|
|
|
- printAmazonClientException(key, e);
|
|
|
- throw e;
|
|
|
+ throw translateException("getFileStatus", key, e);
|
|
|
}
|
|
|
|
|
|
LOG.debug("Not Found: {}", f);
|
|
@@ -1176,10 +1288,42 @@ public class S3AFileSystem extends FileSystem {
|
|
|
* @param overwrite whether to overwrite an existing file
|
|
|
* @param src path
|
|
|
* @param dst path
|
|
|
+ * @throws IOException IO problem
|
|
|
+ * @throws FileAlreadyExistsException the destination file exists and
|
|
|
+ * overwrite==false
|
|
|
+ * @throws AmazonClientException failure in the AWS SDK
|
|
|
*/
|
|
|
@Override
|
|
|
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src,
|
|
|
Path dst) throws IOException {
|
|
|
+ try {
|
|
|
+ innerCopyFromLocalFile(delSrc, overwrite, src, dst);
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("copyFromLocalFile(" + src + ", " + dst + ")",
|
|
|
+ src, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The src file is on the local disk. Add it to FS at
|
|
|
+ * the given dst name.
|
|
|
+ *
|
|
|
+ * This version doesn't need to create a temporary file to calculate the md5.
|
|
|
+ * Sadly this doesn't seem to be used by the shell cp :(
|
|
|
+ *
|
|
|
+ * delSrc indicates if the source should be removed
|
|
|
+ * @param delSrc whether to delete the src
|
|
|
+ * @param overwrite whether to overwrite an existing file
|
|
|
+ * @param src path
|
|
|
+ * @param dst path
|
|
|
+ * @throws IOException IO problem
|
|
|
+ * @throws FileAlreadyExistsException the destination file exists and
|
|
|
+ * overwrite==false
|
|
|
+ * @throws AmazonClientException failure in the AWS SDK
|
|
|
+ */
|
|
|
+ private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite,
|
|
|
+ Path src, Path dst)
|
|
|
+ throws IOException, FileAlreadyExistsException, AmazonClientException {
|
|
|
String key = pathToKey(dst);
|
|
|
|
|
|
if (!overwrite && exists(dst)) {
|
|
@@ -1229,8 +1373,12 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Close the filesystem. This shuts down all transfers.
|
|
|
+ * @throws IOException IO problem
|
|
|
+ */
|
|
|
@Override
|
|
|
- public void close() throws IOException {
|
|
|
+ public synchronized void close() throws IOException {
|
|
|
try {
|
|
|
super.close();
|
|
|
} finally {
|
|
@@ -1242,49 +1390,63 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Override getCanonicalServiceName because we don't support token in S3A.
|
|
|
- */
|
|
|
+ * Override getCanonicalServiceName because we don't support token in S3A.
|
|
|
+ */
|
|
|
@Override
|
|
|
public String getCanonicalServiceName() {
|
|
|
// Does not support Token
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Copy a single object in the bucket via a COPY operation.
|
|
|
+ * @param srcKey source object path
|
|
|
+ * @param dstKey destination object path
|
|
|
+ * @param size object size
|
|
|
+ * @throws AmazonClientException on failures inside the AWS SDK
|
|
|
+ * @throws InterruptedIOException the operation was interrupted
|
|
|
+ * @throws IOException Other IO problems
|
|
|
+ */
|
|
|
private void copyFile(String srcKey, String dstKey, long size)
|
|
|
- throws IOException {
|
|
|
+ throws IOException, InterruptedIOException, AmazonClientException {
|
|
|
LOG.debug("copyFile {} -> {} ", srcKey, dstKey);
|
|
|
|
|
|
- ObjectMetadata srcom = s3.getObjectMetadata(bucket, srcKey);
|
|
|
- ObjectMetadata dstom = cloneObjectMetadata(srcom);
|
|
|
- if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
|
|
|
- dstom.setSSEAlgorithm(serverSideEncryptionAlgorithm);
|
|
|
- }
|
|
|
- CopyObjectRequest copyObjectRequest =
|
|
|
- new CopyObjectRequest(bucket, srcKey, bucket, dstKey);
|
|
|
- copyObjectRequest.setCannedAccessControlList(cannedACL);
|
|
|
- copyObjectRequest.setNewObjectMetadata(dstom);
|
|
|
-
|
|
|
- ProgressListener progressListener = new ProgressListener() {
|
|
|
- public void progressChanged(ProgressEvent progressEvent) {
|
|
|
- switch (progressEvent.getEventType()) {
|
|
|
- case TRANSFER_PART_COMPLETED_EVENT:
|
|
|
- statistics.incrementWriteOps(1);
|
|
|
- break;
|
|
|
- default:
|
|
|
- break;
|
|
|
- }
|
|
|
+ try {
|
|
|
+ ObjectMetadata srcom = s3.getObjectMetadata(bucket, srcKey);
|
|
|
+ ObjectMetadata dstom = cloneObjectMetadata(srcom);
|
|
|
+ if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
|
|
|
+ dstom.setSSEAlgorithm(serverSideEncryptionAlgorithm);
|
|
|
}
|
|
|
- };
|
|
|
+ CopyObjectRequest copyObjectRequest =
|
|
|
+ new CopyObjectRequest(bucket, srcKey, bucket, dstKey);
|
|
|
+ copyObjectRequest.setCannedAccessControlList(cannedACL);
|
|
|
+ copyObjectRequest.setNewObjectMetadata(dstom);
|
|
|
+
|
|
|
+ ProgressListener progressListener = new ProgressListener() {
|
|
|
+ public void progressChanged(ProgressEvent progressEvent) {
|
|
|
+ switch (progressEvent.getEventType()) {
|
|
|
+ case TRANSFER_PART_COMPLETED_EVENT:
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
|
|
|
- Copy copy = transfers.copy(copyObjectRequest);
|
|
|
- copy.addProgressListener(progressListener);
|
|
|
- try {
|
|
|
- copy.waitForCopyResult();
|
|
|
- statistics.incrementWriteOps(1);
|
|
|
- instrumentation.filesCopied(1, size);
|
|
|
- } catch (InterruptedException e) {
|
|
|
- throw new InterruptedIOException("Interrupted copying " + srcKey
|
|
|
- + " to " + dstKey + ", cancelling");
|
|
|
+ Copy copy = transfers.copy(copyObjectRequest);
|
|
|
+ copy.addProgressListener(progressListener);
|
|
|
+ try {
|
|
|
+ copy.waitForCopyResult();
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ instrumentation.filesCopied(1, size);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw new InterruptedIOException("Interrupted copying " + srcKey
|
|
|
+ + " to " + dstKey + ", cancelling");
|
|
|
+ }
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("copyFile("+ srcKey+ ", " + dstKey + ")",
|
|
|
+ srcKey, e);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1303,11 +1465,20 @@ public class S3AFileSystem extends FileSystem {
|
|
|
return date.getTime();
|
|
|
}
|
|
|
|
|
|
- public void finishedWrite(String key) throws IOException {
|
|
|
+ /**
|
|
|
+ * Perform post-write actions.
|
|
|
+ * @param key key written to
|
|
|
+ */
|
|
|
+ public void finishedWrite(String key) {
|
|
|
deleteUnnecessaryFakeDirectories(keyToPath(key).getParent());
|
|
|
}
|
|
|
|
|
|
- private void deleteUnnecessaryFakeDirectories(Path f) throws IOException {
|
|
|
+ /**
|
|
|
+ * Delete mock parent directories which are no longer needed.
|
|
|
+ * This code swallows IO exceptions encountered
|
|
|
+ * @param f path
|
|
|
+ */
|
|
|
+ private void deleteUnnecessaryFakeDirectories(Path f) {
|
|
|
while (true) {
|
|
|
String key = "";
|
|
|
try {
|
|
@@ -1323,7 +1494,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
s3.deleteObject(bucket, key + "/");
|
|
|
statistics.incrementWriteOps(1);
|
|
|
}
|
|
|
- } catch (FileNotFoundException | AmazonServiceException e) {
|
|
|
+ } catch (IOException | AmazonClientException e) {
|
|
|
LOG.debug("While deleting key {} ", key, e);
|
|
|
instrumentation.errorIgnored();
|
|
|
}
|
|
@@ -1446,28 +1617,6 @@ public class S3AFileSystem extends FileSystem {
|
|
|
return getConf().getLong(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE);
|
|
|
}
|
|
|
|
|
|
- private void printAmazonServiceException(String target,
|
|
|
- AmazonServiceException ase) {
|
|
|
- LOG.info("{}: caught an AmazonServiceException {}", target, ase);
|
|
|
- LOG.info("This means your request made it to Amazon S3," +
|
|
|
- " but was rejected with an error response for some reason.");
|
|
|
- LOG.info("Error Message: {}", ase.getMessage());
|
|
|
- LOG.info("HTTP Status Code: {}", ase.getStatusCode());
|
|
|
- LOG.info("AWS Error Code: {}", ase.getErrorCode());
|
|
|
- LOG.info("Error Type: {}", ase.getErrorType());
|
|
|
- LOG.info("Request ID: {}", ase.getRequestId());
|
|
|
- LOG.info("Class Name: {}", ase.getClass().getName());
|
|
|
- LOG.info("Exception", ase);
|
|
|
- }
|
|
|
-
|
|
|
- private void printAmazonClientException(String target,
|
|
|
- AmazonClientException ace) {
|
|
|
- LOG.info("{}: caught an AmazonClientException {}", target, ace);
|
|
|
- LOG.info("This means the client encountered " +
|
|
|
- "a problem while trying to communicate with S3, " +
|
|
|
- "such as not being able to access the network.", ace);
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
final StringBuilder sb = new StringBuilder(
|