|
@@ -22,6 +22,7 @@ import java.io.File;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
|
|
+import java.io.InterruptedIOException;
|
|
import java.net.URI;
|
|
import java.net.URI;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Date;
|
|
import java.util.Date;
|
|
@@ -56,8 +57,11 @@ import com.amazonaws.event.ProgressListener;
|
|
import com.amazonaws.event.ProgressEvent;
|
|
import com.amazonaws.event.ProgressEvent;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.commons.lang.StringUtils;
|
|
|
|
|
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
|
+import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
@@ -76,9 +80,24 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
+/**
|
|
|
|
+ * The core S3A Filesystem implementation.
|
|
|
|
+ *
|
|
|
|
+ * This subclass is marked as private as code should not be creating it
|
|
|
|
+ * directly; use {@link FileSystem#get(Configuration)} and variants to
|
|
|
|
+ * create one.
|
|
|
|
+ *
|
|
|
|
+ * If cast to {@code S3AFileSystem}, extra methods and features may be accessed.
|
|
|
|
+ * Consider those private and unstable.
|
|
|
|
+ *
|
|
|
|
+ * Because it prints some of the state of the instrumentation,
|
|
|
|
+ * the output of {@link #toString()} must also be considered unstable.
|
|
|
|
+ */
|
|
|
|
+@InterfaceAudience.Private
|
|
|
|
+@InterfaceStability.Evolving
|
|
public class S3AFileSystem extends FileSystem {
|
|
public class S3AFileSystem extends FileSystem {
|
|
/**
|
|
/**
|
|
- * Default blocksize as used in blocksize and FS status queries
|
|
|
|
|
|
+ * Default blocksize as used in blocksize and FS status queries.
|
|
*/
|
|
*/
|
|
public static final int DEFAULT_BLOCKSIZE = 32 * 1024 * 1024;
|
|
public static final int DEFAULT_BLOCKSIZE = 32 * 1024 * 1024;
|
|
private URI uri;
|
|
private URI uri;
|
|
@@ -94,6 +113,8 @@ public class S3AFileSystem extends FileSystem {
|
|
public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
|
|
public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
|
|
private CannedAccessControlList cannedACL;
|
|
private CannedAccessControlList cannedACL;
|
|
private String serverSideEncryptionAlgorithm;
|
|
private String serverSideEncryptionAlgorithm;
|
|
|
|
+ private S3AInstrumentation instrumentation;
|
|
|
|
+ private long readAhead;
|
|
|
|
|
|
// The maximum number of entries that can be deleted in any call to s3
|
|
// The maximum number of entries that can be deleted in any call to s3
|
|
private static final int MAX_ENTRIES_TO_DELETE = 1000;
|
|
private static final int MAX_ENTRIES_TO_DELETE = 1000;
|
|
@@ -105,10 +126,12 @@ public class S3AFileSystem extends FileSystem {
|
|
*/
|
|
*/
|
|
public void initialize(URI name, Configuration conf) throws IOException {
|
|
public void initialize(URI name, Configuration conf) throws IOException {
|
|
super.initialize(name, conf);
|
|
super.initialize(name, conf);
|
|
|
|
+ setConf(conf);
|
|
|
|
+ instrumentation = new S3AInstrumentation(name);
|
|
|
|
|
|
uri = URI.create(name.getScheme() + "://" + name.getAuthority());
|
|
uri = URI.create(name.getScheme() + "://" + name.getAuthority());
|
|
- workingDir = new Path("/user", System.getProperty("user.name")).makeQualified(this.uri,
|
|
|
|
- this.getWorkingDirectory());
|
|
|
|
|
|
+ workingDir = new Path("/user", System.getProperty("user.name"))
|
|
|
|
+ .makeQualified(this.uri, this.getWorkingDirectory());
|
|
|
|
|
|
AWSAccessKeys creds = getAWSAccessKeys(name, conf);
|
|
AWSAccessKeys creds = getAWSAccessKeys(name, conf);
|
|
|
|
|
|
@@ -122,19 +145,20 @@ public class S3AFileSystem extends FileSystem {
|
|
bucket = name.getHost();
|
|
bucket = name.getHost();
|
|
|
|
|
|
ClientConfiguration awsConf = new ClientConfiguration();
|
|
ClientConfiguration awsConf = new ClientConfiguration();
|
|
- awsConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS,
|
|
|
|
- DEFAULT_MAXIMUM_CONNECTIONS));
|
|
|
|
|
|
+ awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
|
|
|
|
+ DEFAULT_MAXIMUM_CONNECTIONS, 1));
|
|
boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
|
|
boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
|
|
DEFAULT_SECURE_CONNECTIONS);
|
|
DEFAULT_SECURE_CONNECTIONS);
|
|
awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
|
|
awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
|
|
- awsConf.setMaxErrorRetry(conf.getInt(MAX_ERROR_RETRIES,
|
|
|
|
- DEFAULT_MAX_ERROR_RETRIES));
|
|
|
|
- awsConf.setConnectionTimeout(conf.getInt(ESTABLISH_TIMEOUT,
|
|
|
|
- DEFAULT_ESTABLISH_TIMEOUT));
|
|
|
|
- awsConf.setSocketTimeout(conf.getInt(SOCKET_TIMEOUT,
|
|
|
|
- DEFAULT_SOCKET_TIMEOUT));
|
|
|
|
|
|
+ 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, "");
|
|
String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
|
|
- if(!signerOverride.isEmpty()) {
|
|
|
|
|
|
+ if (!signerOverride.isEmpty()) {
|
|
|
|
+ LOG.debug("Signer override = {}", signerOverride);
|
|
awsConf.setSignerOverride(signerOverride);
|
|
awsConf.setSignerOverride(signerOverride);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -144,21 +168,23 @@ public class S3AFileSystem extends FileSystem {
|
|
|
|
|
|
initAmazonS3Client(conf, credentials, awsConf);
|
|
initAmazonS3Client(conf, credentials, awsConf);
|
|
|
|
|
|
- maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS);
|
|
|
|
|
|
+ maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
|
|
partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
|
|
partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
|
|
- multiPartThreshold = conf.getLong(MIN_MULTIPART_THRESHOLD,
|
|
|
|
- DEFAULT_MIN_MULTIPART_THRESHOLD);
|
|
|
|
- enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
|
|
|
|
-
|
|
|
|
if (partSize < 5 * 1024 * 1024) {
|
|
if (partSize < 5 * 1024 * 1024) {
|
|
LOG.error(MULTIPART_SIZE + " must be at least 5 MB");
|
|
LOG.error(MULTIPART_SIZE + " must be at least 5 MB");
|
|
partSize = 5 * 1024 * 1024;
|
|
partSize = 5 * 1024 * 1024;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ multiPartThreshold = conf.getLong(MIN_MULTIPART_THRESHOLD,
|
|
|
|
+ DEFAULT_MIN_MULTIPART_THRESHOLD);
|
|
if (multiPartThreshold < 5 * 1024 * 1024) {
|
|
if (multiPartThreshold < 5 * 1024 * 1024) {
|
|
LOG.error(MIN_MULTIPART_THRESHOLD + " must be at least 5 MB");
|
|
LOG.error(MIN_MULTIPART_THRESHOLD + " must be at least 5 MB");
|
|
multiPartThreshold = 5 * 1024 * 1024;
|
|
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);
|
|
|
|
|
|
int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS);
|
|
int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS);
|
|
if (maxThreads < 2) {
|
|
if (maxThreads < 2) {
|
|
@@ -180,19 +206,17 @@ public class S3AFileSystem extends FileSystem {
|
|
initCannedAcls(conf);
|
|
initCannedAcls(conf);
|
|
|
|
|
|
if (!s3.doesBucketExist(bucket)) {
|
|
if (!s3.doesBucketExist(bucket)) {
|
|
- throw new IOException("Bucket " + bucket + " does not exist");
|
|
|
|
|
|
+ throw new FileNotFoundException("Bucket " + bucket + " does not exist");
|
|
}
|
|
}
|
|
|
|
|
|
initMultipartUploads(conf);
|
|
initMultipartUploads(conf);
|
|
|
|
|
|
serverSideEncryptionAlgorithm = conf.get(SERVER_SIDE_ENCRYPTION_ALGORITHM);
|
|
serverSideEncryptionAlgorithm = conf.get(SERVER_SIDE_ENCRYPTION_ALGORITHM);
|
|
|
|
|
|
- setConf(conf);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
void initProxySupport(Configuration conf, ClientConfiguration awsConf,
|
|
void initProxySupport(Configuration conf, ClientConfiguration awsConf,
|
|
- boolean secureConnections) throws IllegalArgumentException,
|
|
|
|
- IllegalArgumentException {
|
|
|
|
|
|
+ boolean secureConnections) throws IllegalArgumentException {
|
|
String proxyHost = conf.getTrimmed(PROXY_HOST, "");
|
|
String proxyHost = conf.getTrimmed(PROXY_HOST, "");
|
|
int proxyPort = conf.getInt(PROXY_PORT, -1);
|
|
int proxyPort = conf.getInt(PROXY_PORT, -1);
|
|
if (!proxyHost.isEmpty()) {
|
|
if (!proxyHost.isEmpty()) {
|
|
@@ -223,7 +247,8 @@ public class S3AFileSystem extends FileSystem {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
|
|
LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
|
|
"domain {} as workstation {}", awsConf.getProxyHost(),
|
|
"domain {} as workstation {}", awsConf.getProxyHost(),
|
|
- awsConf.getProxyPort(), String.valueOf(awsConf.getProxyUsername()),
|
|
|
|
|
|
+ awsConf.getProxyPort(),
|
|
|
|
+ String.valueOf(awsConf.getProxyUsername()),
|
|
awsConf.getProxyPassword(), awsConf.getProxyDomain(),
|
|
awsConf.getProxyPassword(), awsConf.getProxyDomain(),
|
|
awsConf.getProxyWorkstation());
|
|
awsConf.getProxyWorkstation());
|
|
}
|
|
}
|
|
@@ -258,7 +283,7 @@ public class S3AFileSystem extends FileSystem {
|
|
AWSCredentialsProviderChain credentials, ClientConfiguration awsConf)
|
|
AWSCredentialsProviderChain credentials, ClientConfiguration awsConf)
|
|
throws IllegalArgumentException {
|
|
throws IllegalArgumentException {
|
|
s3 = new AmazonS3Client(credentials, awsConf);
|
|
s3 = new AmazonS3Client(credentials, awsConf);
|
|
- String endPoint = conf.getTrimmed(ENDPOINT,"");
|
|
|
|
|
|
+ String endPoint = conf.getTrimmed(ENDPOINT, "");
|
|
if (!endPoint.isEmpty()) {
|
|
if (!endPoint.isEmpty()) {
|
|
try {
|
|
try {
|
|
s3.setEndpoint(endPoint);
|
|
s3.setEndpoint(endPoint);
|
|
@@ -301,14 +326,25 @@ public class S3AFileSystem extends FileSystem {
|
|
|
|
|
|
private void initMultipartUploads(Configuration conf) {
|
|
private void initMultipartUploads(Configuration conf) {
|
|
boolean purgeExistingMultipart = conf.getBoolean(PURGE_EXISTING_MULTIPART,
|
|
boolean purgeExistingMultipart = conf.getBoolean(PURGE_EXISTING_MULTIPART,
|
|
- DEFAULT_PURGE_EXISTING_MULTIPART);
|
|
|
|
- long purgeExistingMultipartAge = conf.getLong(PURGE_EXISTING_MULTIPART_AGE,
|
|
|
|
- DEFAULT_PURGE_EXISTING_MULTIPART_AGE);
|
|
|
|
|
|
+ DEFAULT_PURGE_EXISTING_MULTIPART);
|
|
|
|
+ long purgeExistingMultipartAge = longOption(conf,
|
|
|
|
+ PURGE_EXISTING_MULTIPART_AGE, DEFAULT_PURGE_EXISTING_MULTIPART_AGE, 0);
|
|
|
|
|
|
if (purgeExistingMultipart) {
|
|
if (purgeExistingMultipart) {
|
|
- Date purgeBefore = new Date(new Date().getTime() - purgeExistingMultipartAge*1000);
|
|
|
|
|
|
+ Date purgeBefore =
|
|
|
|
+ new Date(new Date().getTime() - purgeExistingMultipartAge * 1000);
|
|
|
|
|
|
- transfers.abortMultipartUploads(bucket, purgeBefore);
|
|
|
|
|
|
+ try {
|
|
|
|
+ transfers.abortMultipartUploads(bucket, purgeBefore);
|
|
|
|
+ } catch (AmazonServiceException e) {
|
|
|
|
+ if (e.getStatusCode() == 403) {
|
|
|
|
+ instrumentation.errorIgnored();
|
|
|
|
+ LOG.debug("Failed to abort multipart uploads against {}," +
|
|
|
|
+ " FS may be read only", bucket, e);
|
|
|
|
+ } else {
|
|
|
|
+ throw e;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -421,16 +457,15 @@ public class S3AFileSystem extends FileSystem {
|
|
public FSDataInputStream open(Path f, int bufferSize)
|
|
public FSDataInputStream open(Path f, int bufferSize)
|
|
throws IOException {
|
|
throws IOException {
|
|
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Opening '{}' for reading.", f);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Opening '{}' for reading.", f);
|
|
final FileStatus fileStatus = getFileStatus(f);
|
|
final FileStatus fileStatus = getFileStatus(f);
|
|
if (fileStatus.isDirectory()) {
|
|
if (fileStatus.isDirectory()) {
|
|
- throw new FileNotFoundException("Can't open " + f + " because it is a directory");
|
|
|
|
|
|
+ throw new FileNotFoundException("Can't open " + f
|
|
|
|
+ + " because it is a directory");
|
|
}
|
|
}
|
|
|
|
|
|
return new FSDataInputStream(new S3AInputStream(bucket, pathToKey(f),
|
|
return new FSDataInputStream(new S3AInputStream(bucket, pathToKey(f),
|
|
- fileStatus.getLen(), s3, statistics));
|
|
|
|
|
|
+ fileStatus.getLen(), s3, statistics, instrumentation, readAhead));
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -456,16 +491,26 @@ public class S3AFileSystem extends FileSystem {
|
|
if (!overwrite && exists(f)) {
|
|
if (!overwrite && exists(f)) {
|
|
throw new FileAlreadyExistsException(f + " already exists");
|
|
throw new FileAlreadyExistsException(f + " already exists");
|
|
}
|
|
}
|
|
|
|
+ instrumentation.fileCreated();
|
|
if (getConf().getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD)) {
|
|
if (getConf().getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD)) {
|
|
return new FSDataOutputStream(new S3AFastOutputStream(s3, this, bucket,
|
|
return new FSDataOutputStream(new S3AFastOutputStream(s3, this, bucket,
|
|
key, progress, statistics, cannedACL,
|
|
key, progress, statistics, cannedACL,
|
|
serverSideEncryptionAlgorithm, partSize, multiPartThreshold,
|
|
serverSideEncryptionAlgorithm, partSize, multiPartThreshold,
|
|
threadPoolExecutor), statistics);
|
|
threadPoolExecutor), statistics);
|
|
}
|
|
}
|
|
- // We pass null to FSDataOutputStream so it won't count writes that are being buffered to a file
|
|
|
|
- return new FSDataOutputStream(new S3AOutputStream(getConf(), transfers, this,
|
|
|
|
- bucket, key, progress, cannedACL, statistics,
|
|
|
|
- serverSideEncryptionAlgorithm), null);
|
|
|
|
|
|
+ // We pass null to FSDataOutputStream so it won't count writes that
|
|
|
|
+ // are being buffered to a file
|
|
|
|
+ return new FSDataOutputStream(
|
|
|
|
+ new S3AOutputStream(getConf(),
|
|
|
|
+ transfers,
|
|
|
|
+ this,
|
|
|
|
+ bucket,
|
|
|
|
+ key,
|
|
|
|
+ progress,
|
|
|
|
+ cannedACL,
|
|
|
|
+ statistics,
|
|
|
|
+ serverSideEncryptionAlgorithm),
|
|
|
|
+ null);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -476,7 +521,7 @@ public class S3AFileSystem extends FileSystem {
|
|
* @throws IOException indicating that append is not supported.
|
|
* @throws IOException indicating that append is not supported.
|
|
*/
|
|
*/
|
|
public FSDataOutputStream append(Path f, int bufferSize,
|
|
public FSDataOutputStream append(Path f, int bufferSize,
|
|
- Progressable progress) throws IOException {
|
|
|
|
|
|
+ Progressable progress) throws IOException {
|
|
throw new IOException("Not supported");
|
|
throw new IOException("Not supported");
|
|
}
|
|
}
|
|
|
|
|
|
@@ -501,17 +546,13 @@ public class S3AFileSystem extends FileSystem {
|
|
* @return true if rename is successful
|
|
* @return true if rename is successful
|
|
*/
|
|
*/
|
|
public boolean rename(Path src, Path dst) throws IOException {
|
|
public boolean rename(Path src, Path dst) throws IOException {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Rename path {} to {}", src, dst);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Rename path {} to {}", src, dst);
|
|
|
|
|
|
String srcKey = pathToKey(src);
|
|
String srcKey = pathToKey(src);
|
|
String dstKey = pathToKey(dst);
|
|
String dstKey = pathToKey(dst);
|
|
|
|
|
|
if (srcKey.isEmpty() || dstKey.isEmpty()) {
|
|
if (srcKey.isEmpty() || dstKey.isEmpty()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("rename: src or dst are empty");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("rename: source {} or dest {}, is empty", srcKey, dstKey);
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -524,9 +565,8 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
|
|
|
|
if (srcKey.equals(dstKey)) {
|
|
if (srcKey.equals(dstKey)) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("rename: src and dst refer to the same file or directory");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("rename: src and dst refer to the same file or directory: {}",
|
|
|
|
+ dst);
|
|
return srcStatus.isFile();
|
|
return srcStatus.isFile();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -535,9 +575,8 @@ public class S3AFileSystem extends FileSystem {
|
|
dstStatus = getFileStatus(dst);
|
|
dstStatus = getFileStatus(dst);
|
|
|
|
|
|
if (srcStatus.isDirectory() && dstStatus.isFile()) {
|
|
if (srcStatus.isDirectory() && dstStatus.isFile()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("rename: src is a directory and dst is a file");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("rename: src {} is a directory and dst {} is a file",
|
|
|
|
+ src, dst);
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -545,6 +584,7 @@ public class S3AFileSystem extends FileSystem {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
} catch (FileNotFoundException e) {
|
|
} catch (FileNotFoundException e) {
|
|
|
|
+ LOG.debug("rename: destination path {} not found", dst);
|
|
// Parent must exist
|
|
// Parent must exist
|
|
Path parent = dst.getParent();
|
|
Path parent = dst.getParent();
|
|
if (!pathToKey(parent).isEmpty()) {
|
|
if (!pathToKey(parent).isEmpty()) {
|
|
@@ -554,6 +594,8 @@ public class S3AFileSystem extends FileSystem {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
} catch (FileNotFoundException e2) {
|
|
} catch (FileNotFoundException e2) {
|
|
|
|
+ LOG.debug("rename: destination path {} has no parent {}",
|
|
|
|
+ dst, parent);
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -561,9 +603,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
|
|
|
// Ok! Time to start
|
|
// Ok! Time to start
|
|
if (srcStatus.isFile()) {
|
|
if (srcStatus.isFile()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("rename: renaming file " + src + " to " + dst);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("rename: renaming file {} to {}", src, dst);
|
|
if (dstStatus != null && dstStatus.isDirectory()) {
|
|
if (dstStatus != null && dstStatus.isDirectory()) {
|
|
String newDstKey = dstKey;
|
|
String newDstKey = dstKey;
|
|
if (!newDstKey.endsWith("/")) {
|
|
if (!newDstKey.endsWith("/")) {
|
|
@@ -572,15 +612,13 @@ public class S3AFileSystem extends FileSystem {
|
|
String filename =
|
|
String filename =
|
|
srcKey.substring(pathToKey(src.getParent()).length()+1);
|
|
srcKey.substring(pathToKey(src.getParent()).length()+1);
|
|
newDstKey = newDstKey + filename;
|
|
newDstKey = newDstKey + filename;
|
|
- copyFile(srcKey, newDstKey);
|
|
|
|
|
|
+ copyFile(srcKey, newDstKey, srcStatus.getLen());
|
|
} else {
|
|
} else {
|
|
- copyFile(srcKey, dstKey);
|
|
|
|
|
|
+ copyFile(srcKey, dstKey, srcStatus.getLen());
|
|
}
|
|
}
|
|
delete(src, false);
|
|
delete(src, false);
|
|
} else {
|
|
} else {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("rename: renaming directory " + src + " to " + dst);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("rename: renaming directory {} to {}", src, dst);
|
|
|
|
|
|
// This is a directory to directory copy
|
|
// This is a directory to directory copy
|
|
if (!dstKey.endsWith("/")) {
|
|
if (!dstKey.endsWith("/")) {
|
|
@@ -593,14 +631,12 @@ public class S3AFileSystem extends FileSystem {
|
|
|
|
|
|
//Verify dest is not a child of the source directory
|
|
//Verify dest is not a child of the source directory
|
|
if (dstKey.startsWith(srcKey)) {
|
|
if (dstKey.startsWith(srcKey)) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("cannot rename a directory to a subdirectory of self");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("cannot rename a directory {}" +
|
|
|
|
+ " to a subdirectory of self: {}", srcKey, dstKey);
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
- List<DeleteObjectsRequest.KeyVersion> keysToDelete =
|
|
|
|
- new ArrayList<>();
|
|
|
|
|
|
+ List<DeleteObjectsRequest.KeyVersion> keysToDelete = new ArrayList<>();
|
|
if (dstStatus != null && dstStatus.isEmptyDirectory()) {
|
|
if (dstStatus != null && dstStatus.isEmptyDirectory()) {
|
|
// delete unnecessary fake directory.
|
|
// delete unnecessary fake directory.
|
|
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey));
|
|
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey));
|
|
@@ -618,7 +654,7 @@ public class S3AFileSystem extends FileSystem {
|
|
for (S3ObjectSummary summary : objects.getObjectSummaries()) {
|
|
for (S3ObjectSummary summary : objects.getObjectSummaries()) {
|
|
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(summary.getKey()));
|
|
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(summary.getKey()));
|
|
String newDstKey = dstKey + summary.getKey().substring(srcKey.length());
|
|
String newDstKey = dstKey + summary.getKey().substring(srcKey.length());
|
|
- copyFile(summary.getKey(), newDstKey);
|
|
|
|
|
|
+ copyFile(summary.getKey(), newDstKey, summary.getSize());
|
|
|
|
|
|
if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
|
|
if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
|
|
removeKeys(keysToDelete, true);
|
|
removeKeys(keysToDelete, true);
|
|
@@ -657,6 +693,7 @@ public class S3AFileSystem extends FileSystem {
|
|
DeleteObjectsRequest deleteRequest
|
|
DeleteObjectsRequest deleteRequest
|
|
= new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
|
|
= new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
|
|
s3.deleteObjects(deleteRequest);
|
|
s3.deleteObjects(deleteRequest);
|
|
|
|
+ instrumentation.fileDeleted(keysToDelete.size());
|
|
statistics.incrementWriteOps(1);
|
|
statistics.incrementWriteOps(1);
|
|
} else {
|
|
} else {
|
|
int writeops = 0;
|
|
int writeops = 0;
|
|
@@ -666,7 +703,7 @@ public class S3AFileSystem extends FileSystem {
|
|
new DeleteObjectRequest(bucket, keyVersion.getKey()));
|
|
new DeleteObjectRequest(bucket, keyVersion.getKey()));
|
|
writeops++;
|
|
writeops++;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+ instrumentation.fileDeleted(keysToDelete.size());
|
|
statistics.incrementWriteOps(writeops);
|
|
statistics.incrementWriteOps(writeops);
|
|
}
|
|
}
|
|
if (clearKeys) {
|
|
if (clearKeys) {
|
|
@@ -684,25 +721,20 @@ public class S3AFileSystem extends FileSystem {
|
|
* @throws IOException due to inability to delete a directory or file.
|
|
* @throws IOException due to inability to delete a directory or file.
|
|
*/
|
|
*/
|
|
public boolean delete(Path f, boolean recursive) throws IOException {
|
|
public boolean delete(Path f, boolean recursive) throws IOException {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Delete path " + f + " - recursive " + recursive);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Delete path {} - recursive {}", f , recursive);
|
|
S3AFileStatus status;
|
|
S3AFileStatus status;
|
|
try {
|
|
try {
|
|
status = getFileStatus(f);
|
|
status = getFileStatus(f);
|
|
} catch (FileNotFoundException e) {
|
|
} catch (FileNotFoundException e) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Couldn't delete " + f + " - does not exist");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Couldn't delete {} - does not exist", f);
|
|
|
|
+ instrumentation.errorIgnored();
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
String key = pathToKey(f);
|
|
String key = pathToKey(f);
|
|
|
|
|
|
if (status.isDirectory()) {
|
|
if (status.isDirectory()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("delete: Path is a directory");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("delete: Path is a directory: {}", f);
|
|
|
|
|
|
if (!recursive && !status.isEmptyDirectory()) {
|
|
if (!recursive && !status.isEmptyDirectory()) {
|
|
throw new IOException("Path is a folder: " + f +
|
|
throw new IOException("Path is a folder: " + f +
|
|
@@ -719,15 +751,12 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
|
|
|
|
if (status.isEmptyDirectory()) {
|
|
if (status.isEmptyDirectory()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Deleting fake empty directory");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Deleting fake empty directory {}", key);
|
|
s3.deleteObject(bucket, key);
|
|
s3.deleteObject(bucket, key);
|
|
|
|
+ instrumentation.directoryDeleted();
|
|
statistics.incrementWriteOps(1);
|
|
statistics.incrementWriteOps(1);
|
|
} else {
|
|
} else {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Getting objects for directory prefix " + key + " to delete");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Getting objects for directory prefix {} to delete", key);
|
|
|
|
|
|
ListObjectsRequest request = new ListObjectsRequest();
|
|
ListObjectsRequest request = new ListObjectsRequest();
|
|
request.setBucketName(bucket);
|
|
request.setBucketName(bucket);
|
|
@@ -736,16 +765,13 @@ public class S3AFileSystem extends FileSystem {
|
|
//request.setDelimiter("/");
|
|
//request.setDelimiter("/");
|
|
request.setMaxKeys(maxKeys);
|
|
request.setMaxKeys(maxKeys);
|
|
|
|
|
|
- List<DeleteObjectsRequest.KeyVersion> keys =
|
|
|
|
- new ArrayList<>();
|
|
|
|
|
|
+ List<DeleteObjectsRequest.KeyVersion> keys = new ArrayList<>();
|
|
ObjectListing objects = s3.listObjects(request);
|
|
ObjectListing objects = s3.listObjects(request);
|
|
statistics.incrementReadOps(1);
|
|
statistics.incrementReadOps(1);
|
|
while (true) {
|
|
while (true) {
|
|
for (S3ObjectSummary summary : objects.getObjectSummaries()) {
|
|
for (S3ObjectSummary summary : objects.getObjectSummaries()) {
|
|
keys.add(new DeleteObjectsRequest.KeyVersion(summary.getKey()));
|
|
keys.add(new DeleteObjectsRequest.KeyVersion(summary.getKey()));
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Got object to delete " + summary.getKey());
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Got object to delete {}", summary.getKey());
|
|
|
|
|
|
if (keys.size() == MAX_ENTRIES_TO_DELETE) {
|
|
if (keys.size() == MAX_ENTRIES_TO_DELETE) {
|
|
removeKeys(keys, true);
|
|
removeKeys(keys, true);
|
|
@@ -764,10 +790,9 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("delete: Path is a file");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("delete: Path is a file");
|
|
s3.deleteObject(bucket, key);
|
|
s3.deleteObject(bucket, key);
|
|
|
|
+ instrumentation.fileDeleted(1);
|
|
statistics.incrementWriteOps(1);
|
|
statistics.incrementWriteOps(1);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -779,9 +804,7 @@ public class S3AFileSystem extends FileSystem {
|
|
private void createFakeDirectoryIfNecessary(Path f) throws IOException {
|
|
private void createFakeDirectoryIfNecessary(Path f) throws IOException {
|
|
String key = pathToKey(f);
|
|
String key = pathToKey(f);
|
|
if (!key.isEmpty() && !exists(f)) {
|
|
if (!key.isEmpty() && !exists(f)) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Creating new fake directory at " + f);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Creating new fake directory at {}", f);
|
|
createFakeDirectory(bucket, key);
|
|
createFakeDirectory(bucket, key);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -798,9 +821,7 @@ public class S3AFileSystem extends FileSystem {
|
|
public FileStatus[] listStatus(Path f) throws FileNotFoundException,
|
|
public FileStatus[] listStatus(Path f) throws FileNotFoundException,
|
|
IOException {
|
|
IOException {
|
|
String key = pathToKey(f);
|
|
String key = pathToKey(f);
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("List status for path: " + f);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("List status for path: {}", f);
|
|
|
|
|
|
final List<FileStatus> result = new ArrayList<FileStatus>();
|
|
final List<FileStatus> result = new ArrayList<FileStatus>();
|
|
final FileStatus fileStatus = getFileStatus(f);
|
|
final FileStatus fileStatus = getFileStatus(f);
|
|
@@ -816,9 +837,7 @@ public class S3AFileSystem extends FileSystem {
|
|
request.setDelimiter("/");
|
|
request.setDelimiter("/");
|
|
request.setMaxKeys(maxKeys);
|
|
request.setMaxKeys(maxKeys);
|
|
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("listStatus: doing listObjects for directory " + key);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("listStatus: doing listObjects for directory {}", key);
|
|
|
|
|
|
ObjectListing objects = s3.listObjects(request);
|
|
ObjectListing objects = s3.listObjects(request);
|
|
statistics.incrementReadOps(1);
|
|
statistics.incrementReadOps(1);
|
|
@@ -831,24 +850,18 @@ public class S3AFileSystem extends FileSystem {
|
|
// Skip over keys that are ourselves and old S3N _$folder$ files
|
|
// Skip over keys that are ourselves and old S3N _$folder$ files
|
|
if (keyPath.equals(fQualified) ||
|
|
if (keyPath.equals(fQualified) ||
|
|
summary.getKey().endsWith(S3N_FOLDER_SUFFIX)) {
|
|
summary.getKey().endsWith(S3N_FOLDER_SUFFIX)) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Ignoring: " + keyPath);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Ignoring: {}", keyPath);
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
|
|
|
|
if (objectRepresentsDirectory(summary.getKey(), summary.getSize())) {
|
|
if (objectRepresentsDirectory(summary.getKey(), summary.getSize())) {
|
|
result.add(new S3AFileStatus(true, true, keyPath));
|
|
result.add(new S3AFileStatus(true, true, keyPath));
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Adding: fd: " + keyPath);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Adding: fd: {}", keyPath);
|
|
} else {
|
|
} else {
|
|
result.add(new S3AFileStatus(summary.getSize(),
|
|
result.add(new S3AFileStatus(summary.getSize(),
|
|
dateToLong(summary.getLastModified()), keyPath,
|
|
dateToLong(summary.getLastModified()), keyPath,
|
|
getDefaultBlockSize(fQualified)));
|
|
getDefaultBlockSize(fQualified)));
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Adding: fi: " + keyPath);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Adding: fi: {}", keyPath);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -858,16 +871,11 @@ public class S3AFileSystem extends FileSystem {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
result.add(new S3AFileStatus(true, false, keyPath));
|
|
result.add(new S3AFileStatus(true, false, keyPath));
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Adding: rd: " + keyPath);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Adding: rd: {}", keyPath);
|
|
}
|
|
}
|
|
|
|
|
|
if (objects.isTruncated()) {
|
|
if (objects.isTruncated()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("listStatus: list truncated - getting next batch");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ LOG.debug("listStatus: list truncated - getting next batch");
|
|
objects = s3.listNextBatchOfObjects(objects);
|
|
objects = s3.listNextBatchOfObjects(objects);
|
|
statistics.incrementReadOps(1);
|
|
statistics.incrementReadOps(1);
|
|
} else {
|
|
} else {
|
|
@@ -875,9 +883,7 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Adding: rd (not a dir): " + f);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Adding: rd (not a dir): {}", f);
|
|
result.add(fileStatus);
|
|
result.add(fileStatus);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -890,14 +896,14 @@ public class S3AFileSystem extends FileSystem {
|
|
* Set the current working directory for the given file system. All relative
|
|
* Set the current working directory for the given file system. All relative
|
|
* paths will be resolved relative to it.
|
|
* paths will be resolved relative to it.
|
|
*
|
|
*
|
|
- * @param new_dir the current working directory.
|
|
|
|
|
|
+ * @param newDir the current working directory.
|
|
*/
|
|
*/
|
|
- public void setWorkingDirectory(Path new_dir) {
|
|
|
|
- workingDir = new_dir;
|
|
|
|
|
|
+ public void setWorkingDirectory(Path newDir) {
|
|
|
|
+ workingDir = newDir;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Get the current working directory for the given file system
|
|
|
|
|
|
+ * Get the current working directory for the given file system.
|
|
* @return the directory pathname
|
|
* @return the directory pathname
|
|
*/
|
|
*/
|
|
public Path getWorkingDirectory() {
|
|
public Path getWorkingDirectory() {
|
|
@@ -914,10 +920,7 @@ public class S3AFileSystem extends FileSystem {
|
|
// TODO: If we have created an empty file at /foo/bar and we then call
|
|
// 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/?
|
|
// mkdirs for /foo/bar/baz/roo what happens to the empty file /foo/bar/?
|
|
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
|
|
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Making directory: " + f);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ LOG.debug("Making directory: {}", f);
|
|
|
|
|
|
try {
|
|
try {
|
|
FileStatus fileStatus = getFileStatus(f);
|
|
FileStatus fileStatus = getFileStatus(f);
|
|
@@ -938,6 +941,7 @@ public class S3AFileSystem extends FileSystem {
|
|
fPart));
|
|
fPart));
|
|
}
|
|
}
|
|
} catch (FileNotFoundException fnfe) {
|
|
} catch (FileNotFoundException fnfe) {
|
|
|
|
+ instrumentation.errorIgnored();
|
|
}
|
|
}
|
|
fPart = fPart.getParent();
|
|
fPart = fPart.getParent();
|
|
} while (fPart != null);
|
|
} while (fPart != null);
|
|
@@ -957,10 +961,7 @@ public class S3AFileSystem extends FileSystem {
|
|
*/
|
|
*/
|
|
public S3AFileStatus getFileStatus(Path f) throws IOException {
|
|
public S3AFileStatus getFileStatus(Path f) throws IOException {
|
|
String key = pathToKey(f);
|
|
String key = pathToKey(f);
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Getting path status for " + f + " (" + key + ")");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ LOG.debug("Getting path status for {} ({})", f , key);
|
|
|
|
|
|
if (!key.isEmpty()) {
|
|
if (!key.isEmpty()) {
|
|
try {
|
|
try {
|
|
@@ -968,15 +969,11 @@ public class S3AFileSystem extends FileSystem {
|
|
statistics.incrementReadOps(1);
|
|
statistics.incrementReadOps(1);
|
|
|
|
|
|
if (objectRepresentsDirectory(key, meta.getContentLength())) {
|
|
if (objectRepresentsDirectory(key, meta.getContentLength())) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Found exact file: fake directory");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Found exact file: fake directory");
|
|
return new S3AFileStatus(true, true,
|
|
return new S3AFileStatus(true, true,
|
|
f.makeQualified(uri, workingDir));
|
|
f.makeQualified(uri, workingDir));
|
|
} else {
|
|
} else {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Found exact file: normal file");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Found exact file: normal file");
|
|
return new S3AFileStatus(meta.getContentLength(),
|
|
return new S3AFileStatus(meta.getContentLength(),
|
|
dateToLong(meta.getLastModified()),
|
|
dateToLong(meta.getLastModified()),
|
|
f.makeQualified(uri, workingDir),
|
|
f.makeQualified(uri, workingDir),
|
|
@@ -984,25 +981,23 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
} catch (AmazonServiceException e) {
|
|
} catch (AmazonServiceException e) {
|
|
if (e.getStatusCode() != 404) {
|
|
if (e.getStatusCode() != 404) {
|
|
- printAmazonServiceException(e);
|
|
|
|
|
|
+ printAmazonServiceException(f.toString(), e);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
} catch (AmazonClientException e) {
|
|
} catch (AmazonClientException e) {
|
|
- printAmazonClientException(e);
|
|
|
|
|
|
+ printAmazonClientException(f.toString(), e);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
|
|
// Necessary?
|
|
// Necessary?
|
|
if (!key.endsWith("/")) {
|
|
if (!key.endsWith("/")) {
|
|
|
|
+ String newKey = key + "/";
|
|
try {
|
|
try {
|
|
- String newKey = key + "/";
|
|
|
|
ObjectMetadata meta = s3.getObjectMetadata(bucket, newKey);
|
|
ObjectMetadata meta = s3.getObjectMetadata(bucket, newKey);
|
|
statistics.incrementReadOps(1);
|
|
statistics.incrementReadOps(1);
|
|
|
|
|
|
if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
|
|
if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Found file (with /): fake directory");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Found file (with /): fake directory");
|
|
return new S3AFileStatus(true, true, f.makeQualified(uri, workingDir));
|
|
return new S3AFileStatus(true, true, f.makeQualified(uri, workingDir));
|
|
} else {
|
|
} else {
|
|
LOG.warn("Found file (with /): real file? should not happen: {}", key);
|
|
LOG.warn("Found file (with /): real file? should not happen: {}", key);
|
|
@@ -1014,11 +1009,11 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
} catch (AmazonServiceException e) {
|
|
} catch (AmazonServiceException e) {
|
|
if (e.getStatusCode() != 404) {
|
|
if (e.getStatusCode() != 404) {
|
|
- printAmazonServiceException(e);
|
|
|
|
|
|
+ printAmazonServiceException(newKey, e);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
} catch (AmazonClientException e) {
|
|
} catch (AmazonClientException e) {
|
|
- printAmazonClientException(e);
|
|
|
|
|
|
+ printAmazonClientException(newKey, e);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1038,17 +1033,17 @@ public class S3AFileSystem extends FileSystem {
|
|
statistics.incrementReadOps(1);
|
|
statistics.incrementReadOps(1);
|
|
|
|
|
|
if (!objects.getCommonPrefixes().isEmpty()
|
|
if (!objects.getCommonPrefixes().isEmpty()
|
|
- || objects.getObjectSummaries().size() > 0) {
|
|
|
|
|
|
+ || !objects.getObjectSummaries().isEmpty()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
- LOG.debug("Found path as directory (with /): " +
|
|
|
|
- objects.getCommonPrefixes().size() + "/" +
|
|
|
|
|
|
+ LOG.debug("Found path as directory (with /): {}/{}",
|
|
|
|
+ objects.getCommonPrefixes().size() ,
|
|
objects.getObjectSummaries().size());
|
|
objects.getObjectSummaries().size());
|
|
|
|
|
|
for (S3ObjectSummary summary : objects.getObjectSummaries()) {
|
|
for (S3ObjectSummary summary : objects.getObjectSummaries()) {
|
|
- LOG.debug("Summary: " + summary.getKey() + " " + summary.getSize());
|
|
|
|
|
|
+ LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize());
|
|
}
|
|
}
|
|
for (String prefix : objects.getCommonPrefixes()) {
|
|
for (String prefix : objects.getCommonPrefixes()) {
|
|
- LOG.debug("Prefix: " + prefix);
|
|
|
|
|
|
+ LOG.debug("Prefix: {}", prefix);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1060,17 +1055,15 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
} catch (AmazonServiceException e) {
|
|
} catch (AmazonServiceException e) {
|
|
if (e.getStatusCode() != 404) {
|
|
if (e.getStatusCode() != 404) {
|
|
- printAmazonServiceException(e);
|
|
|
|
|
|
+ printAmazonServiceException(key, e);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
} catch (AmazonClientException e) {
|
|
} catch (AmazonClientException e) {
|
|
- printAmazonClientException(e);
|
|
|
|
|
|
+ printAmazonClientException(key, e);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Not Found: " + f);
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Not Found: {}", f);
|
|
throw new FileNotFoundException("No such file or directory: " + f);
|
|
throw new FileNotFoundException("No such file or directory: " + f);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1089,15 +1082,13 @@ public class S3AFileSystem extends FileSystem {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src,
|
|
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src,
|
|
- Path dst) throws IOException {
|
|
|
|
|
|
+ Path dst) throws IOException {
|
|
String key = pathToKey(dst);
|
|
String key = pathToKey(dst);
|
|
|
|
|
|
if (!overwrite && exists(dst)) {
|
|
if (!overwrite && exists(dst)) {
|
|
- throw new IOException(dst + " already exists");
|
|
|
|
- }
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Copying local file from " + src + " to " + dst);
|
|
|
|
|
|
+ throw new FileAlreadyExistsException(dst + " already exists");
|
|
}
|
|
}
|
|
|
|
+ LOG.debug("Copying local file from {} to {}", src, dst);
|
|
|
|
|
|
// Since we have a local file, we don't need to stream into a temporary file
|
|
// Since we have a local file, we don't need to stream into a temporary file
|
|
LocalFileSystem local = getLocal(getConf());
|
|
LocalFileSystem local = getLocal(getConf());
|
|
@@ -1123,13 +1114,14 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
};
|
|
};
|
|
|
|
|
|
|
|
+ statistics.incrementWriteOps(1);
|
|
Upload up = transfers.upload(putObjectRequest);
|
|
Upload up = transfers.upload(putObjectRequest);
|
|
up.addProgressListener(progressListener);
|
|
up.addProgressListener(progressListener);
|
|
try {
|
|
try {
|
|
up.waitForUploadResult();
|
|
up.waitForUploadResult();
|
|
- statistics.incrementWriteOps(1);
|
|
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
- throw new IOException("Got interrupted, cancelling");
|
|
|
|
|
|
+ throw new InterruptedIOException("Interrupted copying " + src
|
|
|
|
+ + " to " + dst + ", cancelling");
|
|
}
|
|
}
|
|
|
|
|
|
// This will delete unnecessary fake parent directories
|
|
// This will delete unnecessary fake parent directories
|
|
@@ -1153,7 +1145,7 @@ public class S3AFileSystem extends FileSystem {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Override getCononicalServiceName because we don't support token in S3A
|
|
|
|
|
|
+ * Override getCanonicalServiceName because we don't support token in S3A.
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public String getCanonicalServiceName() {
|
|
public String getCanonicalServiceName() {
|
|
@@ -1161,17 +1153,17 @@ public class S3AFileSystem extends FileSystem {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- private void copyFile(String srcKey, String dstKey) throws IOException {
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("copyFile " + srcKey + " -> " + dstKey);
|
|
|
|
- }
|
|
|
|
|
|
+ private void copyFile(String srcKey, String dstKey, long size)
|
|
|
|
+ throws IOException {
|
|
|
|
+ LOG.debug("copyFile {} -> {} ", srcKey, dstKey);
|
|
|
|
|
|
ObjectMetadata srcom = s3.getObjectMetadata(bucket, srcKey);
|
|
ObjectMetadata srcom = s3.getObjectMetadata(bucket, srcKey);
|
|
ObjectMetadata dstom = cloneObjectMetadata(srcom);
|
|
ObjectMetadata dstom = cloneObjectMetadata(srcom);
|
|
if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
|
|
if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
|
|
dstom.setSSEAlgorithm(serverSideEncryptionAlgorithm);
|
|
dstom.setSSEAlgorithm(serverSideEncryptionAlgorithm);
|
|
}
|
|
}
|
|
- CopyObjectRequest copyObjectRequest = new CopyObjectRequest(bucket, srcKey, bucket, dstKey);
|
|
|
|
|
|
+ CopyObjectRequest copyObjectRequest =
|
|
|
|
+ new CopyObjectRequest(bucket, srcKey, bucket, dstKey);
|
|
copyObjectRequest.setCannedAccessControlList(cannedACL);
|
|
copyObjectRequest.setCannedAccessControlList(cannedACL);
|
|
copyObjectRequest.setNewObjectMetadata(dstom);
|
|
copyObjectRequest.setNewObjectMetadata(dstom);
|
|
|
|
|
|
@@ -1192,13 +1184,17 @@ public class S3AFileSystem extends FileSystem {
|
|
try {
|
|
try {
|
|
copy.waitForCopyResult();
|
|
copy.waitForCopyResult();
|
|
statistics.incrementWriteOps(1);
|
|
statistics.incrementWriteOps(1);
|
|
|
|
+ instrumentation.filesCopied(1, size);
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
- throw new IOException("Got interrupted, cancelling");
|
|
|
|
|
|
+ throw new InterruptedIOException("Interrupted copying " + srcKey
|
|
|
|
+ + " to " + dstKey + ", cancelling");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
private boolean objectRepresentsDirectory(final String name, final long size) {
|
|
private boolean objectRepresentsDirectory(final String name, final long size) {
|
|
- return !name.isEmpty() && name.charAt(name.length() - 1) == '/' && size == 0L;
|
|
|
|
|
|
+ return !name.isEmpty()
|
|
|
|
+ && name.charAt(name.length() - 1) == '/'
|
|
|
|
+ && size == 0L;
|
|
}
|
|
}
|
|
|
|
|
|
// Handles null Dates that can be returned by AWS
|
|
// Handles null Dates that can be returned by AWS
|
|
@@ -1216,8 +1212,9 @@ public class S3AFileSystem extends FileSystem {
|
|
|
|
|
|
private void deleteUnnecessaryFakeDirectories(Path f) throws IOException {
|
|
private void deleteUnnecessaryFakeDirectories(Path f) throws IOException {
|
|
while (true) {
|
|
while (true) {
|
|
|
|
+ String key = "";
|
|
try {
|
|
try {
|
|
- String key = pathToKey(f);
|
|
|
|
|
|
+ key = pathToKey(f);
|
|
if (key.isEmpty()) {
|
|
if (key.isEmpty()) {
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
@@ -1225,13 +1222,13 @@ public class S3AFileSystem extends FileSystem {
|
|
S3AFileStatus status = getFileStatus(f);
|
|
S3AFileStatus status = getFileStatus(f);
|
|
|
|
|
|
if (status.isDirectory() && status.isEmptyDirectory()) {
|
|
if (status.isDirectory() && status.isEmptyDirectory()) {
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Deleting fake directory " + key + "/");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Deleting fake directory {}/", key);
|
|
s3.deleteObject(bucket, key + "/");
|
|
s3.deleteObject(bucket, key + "/");
|
|
statistics.incrementWriteOps(1);
|
|
statistics.incrementWriteOps(1);
|
|
}
|
|
}
|
|
} catch (FileNotFoundException | AmazonServiceException e) {
|
|
} catch (FileNotFoundException | AmazonServiceException e) {
|
|
|
|
+ LOG.debug("While deleting key {} ", key, e);
|
|
|
|
+ instrumentation.errorIgnored();
|
|
}
|
|
}
|
|
|
|
|
|
if (f.isRoot()) {
|
|
if (f.isRoot()) {
|
|
@@ -1267,10 +1264,12 @@ public class S3AFileSystem extends FileSystem {
|
|
if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
|
|
if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
|
|
om.setSSEAlgorithm(serverSideEncryptionAlgorithm);
|
|
om.setSSEAlgorithm(serverSideEncryptionAlgorithm);
|
|
}
|
|
}
|
|
- PutObjectRequest putObjectRequest = new PutObjectRequest(bucketName, objectName, im, om);
|
|
|
|
|
|
+ PutObjectRequest putObjectRequest =
|
|
|
|
+ new PutObjectRequest(bucketName, objectName, im, om);
|
|
putObjectRequest.setCannedAcl(cannedACL);
|
|
putObjectRequest.setCannedAcl(cannedACL);
|
|
s3.putObject(putObjectRequest);
|
|
s3.putObject(putObjectRequest);
|
|
statistics.incrementWriteOps(1);
|
|
statistics.incrementWriteOps(1);
|
|
|
|
+ instrumentation.directoryCreated();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1342,31 +1341,115 @@ public class S3AFileSystem extends FileSystem {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Return the number of bytes that large input files should be optimally
|
|
* Return the number of bytes that large input files should be optimally
|
|
- * be split into to minimize i/o time.
|
|
|
|
|
|
+ * be split into to minimize I/O time.
|
|
* @deprecated use {@link #getDefaultBlockSize(Path)} instead
|
|
* @deprecated use {@link #getDefaultBlockSize(Path)} instead
|
|
*/
|
|
*/
|
|
@Deprecated
|
|
@Deprecated
|
|
public long getDefaultBlockSize() {
|
|
public long getDefaultBlockSize() {
|
|
- // default to 32MB: large enough to minimize the impact of seeks
|
|
|
|
return getConf().getLong(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE);
|
|
return getConf().getLong(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE);
|
|
}
|
|
}
|
|
|
|
|
|
- private void printAmazonServiceException(AmazonServiceException ase) {
|
|
|
|
- LOG.info("Caught an AmazonServiceException, which 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());
|
|
|
|
|
|
+ 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);
|
|
}
|
|
}
|
|
|
|
|
|
- private void printAmazonClientException(AmazonClientException ace) {
|
|
|
|
- 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, ace);
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public String toString() {
|
|
|
|
+ final StringBuilder sb = new StringBuilder(
|
|
|
|
+ "S3AFileSystem{");
|
|
|
|
+ sb.append("uri=").append(uri);
|
|
|
|
+ sb.append(", workingDir=").append(workingDir);
|
|
|
|
+ sb.append(", partSize=").append(partSize);
|
|
|
|
+ sb.append(", enableMultiObjectsDelete=").append(enableMultiObjectsDelete);
|
|
|
|
+ sb.append(", maxKeys=").append(maxKeys);
|
|
|
|
+ sb.append(", cannedACL=").append(cannedACL.toString());
|
|
|
|
+ sb.append(", readAhead=").append(readAhead);
|
|
|
|
+ sb.append(", blockSize=").append(getDefaultBlockSize());
|
|
|
|
+ sb.append(", multiPartThreshold=").append(multiPartThreshold);
|
|
|
|
+ if (serverSideEncryptionAlgorithm != null) {
|
|
|
|
+ sb.append(", serverSideEncryptionAlgorithm='")
|
|
|
|
+ .append(serverSideEncryptionAlgorithm)
|
|
|
|
+ .append('\'');
|
|
|
|
+ }
|
|
|
|
+ sb.append(", statistics {")
|
|
|
|
+ .append(statistics.toString())
|
|
|
|
+ .append("}");
|
|
|
|
+ sb.append(", metrics {")
|
|
|
|
+ .append(instrumentation.dump("{", "=", "} ", true))
|
|
|
|
+ .append("}");
|
|
|
|
+ sb.append('}');
|
|
|
|
+ return sb.toString();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the partition size for multipart operations.
|
|
|
|
+ * @return the value as set during initialization
|
|
|
|
+ */
|
|
|
|
+ public long getPartitionSize() {
|
|
|
|
+ return partSize;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the threshold for multipart files
|
|
|
|
+ * @return the value as set during initialization
|
|
|
|
+ */
|
|
|
|
+ public long getMultiPartThreshold() {
|
|
|
|
+ return multiPartThreshold;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get a integer option >= the minimum allowed value.
|
|
|
|
+ * @param conf configuration
|
|
|
|
+ * @param key key to look up
|
|
|
|
+ * @param defVal default value
|
|
|
|
+ * @param min minimum value
|
|
|
|
+ * @return the value
|
|
|
|
+ * @throws IllegalArgumentException if the value is below the minimum
|
|
|
|
+ */
|
|
|
|
+ static int intOption(Configuration conf, String key, int defVal, int min) {
|
|
|
|
+ int v = conf.getInt(key, defVal);
|
|
|
|
+ Preconditions.checkArgument(v >= min,
|
|
|
|
+ String.format("Value of %s: %d is below the minimum value %d",
|
|
|
|
+ key, v, min));
|
|
|
|
+ return v;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get a long option >= the minimum allowed value.
|
|
|
|
+ * @param conf configuration
|
|
|
|
+ * @param key key to look up
|
|
|
|
+ * @param defVal default value
|
|
|
|
+ * @param min minimum value
|
|
|
|
+ * @return the value
|
|
|
|
+ * @throws IllegalArgumentException if the value is below the minimum
|
|
|
|
+ */
|
|
|
|
+ static long longOption(Configuration conf,
|
|
|
|
+ String key,
|
|
|
|
+ long defVal,
|
|
|
|
+ long min) {
|
|
|
|
+ long v = conf.getLong(key, defVal);
|
|
|
|
+ Preconditions.checkArgument(v >= min,
|
|
|
|
+ String.format("Value of %s: %d is below the minimum value %d",
|
|
|
|
+ key, v, min));
|
|
|
|
+ return v;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|