|
@@ -84,6 +84,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.Listing.ACCEPT_ALL;
|
|
|
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
|
|
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
|
|
|
|
@@ -115,6 +116,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
private AmazonS3Client s3;
|
|
|
private String bucket;
|
|
|
private int maxKeys;
|
|
|
+ private Listing listing;
|
|
|
private long partSize;
|
|
|
private boolean enableMultiObjectsDelete;
|
|
|
private TransferManager transfers;
|
|
@@ -181,6 +183,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
initAmazonS3Client(conf, credentials, awsConf);
|
|
|
|
|
|
maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
|
|
|
+ listing = new Listing(this);
|
|
|
partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
|
|
|
if (partSize < 5 * 1024 * 1024) {
|
|
|
LOG.error(MULTIPART_SIZE + " must be at least 5 MB");
|
|
@@ -514,7 +517,11 @@ public class S3AFileSystem extends FileSystem {
|
|
|
super();
|
|
|
}
|
|
|
|
|
|
- /* Turns a path (relative or otherwise) into an S3 key
|
|
|
+ /**
|
|
|
+ * Turns a path (relative or otherwise) into an S3 key.
|
|
|
+ *
|
|
|
+ * @param path input path, may be relative to the working dir
|
|
|
+ * @return a key excluding the leading "/", or, if it is the root path, ""
|
|
|
*/
|
|
|
private String pathToKey(Path path) {
|
|
|
if (!path.isAbsolute()) {
|
|
@@ -528,10 +535,49 @@ public class S3AFileSystem extends FileSystem {
|
|
|
return path.toUri().getPath().substring(1);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Turns a path (relative or otherwise) into an S3 key, adding a trailing
|
|
|
+ * "/" if the path is not the root <i>and</i> does not already have a "/"
|
|
|
+ * at the end.
|
|
|
+ *
|
|
|
+ * @param key s3 key or ""
|
|
|
+ * @return the with a trailing "/", or, if it is the root key, "",
|
|
|
+ */
|
|
|
+ private String maybeAddTrailingSlash(String key) {
|
|
|
+ if (!key.isEmpty() && !key.endsWith("/")) {
|
|
|
+ return key + '/';
|
|
|
+ } else {
|
|
|
+ return key;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Convert a path back to a key.
|
|
|
+ * @param key input key
|
|
|
+ * @return the path from this key
|
|
|
+ */
|
|
|
private Path keyToPath(String key) {
|
|
|
return new Path("/" + key);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Convert a key to a fully qualified path.
|
|
|
+ * @param key input key
|
|
|
+ * @return the fully qualified path including URI scheme and bucket name.
|
|
|
+ */
|
|
|
+ Path keyToQualifiedPath(String key) {
|
|
|
+ return qualify(keyToPath(key));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Qualify a path.
|
|
|
+ * @param path path to qualify
|
|
|
+ * @return a qualified path.
|
|
|
+ */
|
|
|
+ Path qualify(Path path) {
|
|
|
+ return path.makeQualified(uri, workingDir);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Check that a Path belongs to this FileSystem.
|
|
|
* Unlike the superclass, this version does not look at authority,
|
|
@@ -550,10 +596,10 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Opens an FSDataInputStream at the indicated Path.
|
|
|
- * @param f the file name to open
|
|
|
- * @param bufferSize the size of the buffer to be used.
|
|
|
- */
|
|
|
+ * Opens an FSDataInputStream at the indicated Path.
|
|
|
+ * @param f the file name to open
|
|
|
+ * @param bufferSize the size of the buffer to be used.
|
|
|
+ */
|
|
|
public FSDataInputStream open(Path f, int bufferSize)
|
|
|
throws IOException {
|
|
|
|
|
@@ -875,7 +921,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
* @return the next result object
|
|
|
*/
|
|
|
protected ObjectListing continueListObjects(ObjectListing objects) {
|
|
|
- incrementStatistic(OBJECT_LIST_REQUESTS);
|
|
|
+ incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS);
|
|
|
incrementReadOperations();
|
|
|
return s3.listNextBatchOfObjects(objects);
|
|
|
}
|
|
@@ -1132,12 +1178,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
} else {
|
|
|
LOG.debug("Getting objects for directory prefix {} to delete", key);
|
|
|
|
|
|
- ListObjectsRequest request = new ListObjectsRequest();
|
|
|
- request.setBucketName(bucket);
|
|
|
- request.setPrefix(key);
|
|
|
- // Hopefully not setting a delimiter will cause this to find everything
|
|
|
- //request.setDelimiter("/");
|
|
|
- request.setMaxKeys(maxKeys);
|
|
|
+ ListObjectsRequest request = createListObjectsRequest(key, null);
|
|
|
|
|
|
ObjectListing objects = listObjects(request);
|
|
|
List<DeleteObjectsRequest.KeyVersion> keys =
|
|
@@ -1211,66 +1252,57 @@ public class S3AFileSystem extends FileSystem {
|
|
|
*/
|
|
|
public FileStatus[] innerListStatus(Path f) throws FileNotFoundException,
|
|
|
IOException, AmazonClientException {
|
|
|
- String key = pathToKey(f);
|
|
|
- LOG.debug("List status for path: {}", f);
|
|
|
+ Path path = qualify(f);
|
|
|
+ String key = pathToKey(path);
|
|
|
+ LOG.debug("List status for path: {}", path);
|
|
|
incrementStatistic(INVOCATION_LIST_STATUS);
|
|
|
|
|
|
- final List<FileStatus> result = new ArrayList<FileStatus>();
|
|
|
- final FileStatus fileStatus = getFileStatus(f);
|
|
|
+ List<FileStatus> result;
|
|
|
+ final FileStatus fileStatus = getFileStatus(path);
|
|
|
|
|
|
if (fileStatus.isDirectory()) {
|
|
|
if (!key.isEmpty()) {
|
|
|
- key = key + "/";
|
|
|
+ key = key + '/';
|
|
|
}
|
|
|
|
|
|
- ListObjectsRequest request = new ListObjectsRequest();
|
|
|
- request.setBucketName(bucket);
|
|
|
- request.setPrefix(key);
|
|
|
- request.setDelimiter("/");
|
|
|
- request.setMaxKeys(maxKeys);
|
|
|
-
|
|
|
+ ListObjectsRequest request = createListObjectsRequest(key, "/");
|
|
|
LOG.debug("listStatus: doing listObjects for directory {}", key);
|
|
|
|
|
|
- ObjectListing objects = listObjects(request);
|
|
|
-
|
|
|
- Path fQualified = f.makeQualified(uri, workingDir);
|
|
|
-
|
|
|
- while (true) {
|
|
|
- for (S3ObjectSummary summary : objects.getObjectSummaries()) {
|
|
|
- Path keyPath = keyToPath(summary.getKey()).makeQualified(uri, workingDir);
|
|
|
- // Skip over keys that are ourselves and old S3N _$folder$ files
|
|
|
- if (keyPath.equals(fQualified) ||
|
|
|
- summary.getKey().endsWith(S3N_FOLDER_SUFFIX)) {
|
|
|
- LOG.debug("Ignoring: {}", keyPath);
|
|
|
- } else {
|
|
|
- S3AFileStatus status = createFileStatus(keyPath, summary,
|
|
|
- getDefaultBlockSize(keyPath));
|
|
|
- result.add(status);
|
|
|
- LOG.debug("Adding: {}", status);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- for (String prefix : objects.getCommonPrefixes()) {
|
|
|
- Path keyPath = keyToPath(prefix).makeQualified(uri, workingDir);
|
|
|
- if (!keyPath.equals(f)) {
|
|
|
- result.add(new S3AFileStatus(true, false, keyPath));
|
|
|
- LOG.debug("Adding: rd: {}", keyPath);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (objects.isTruncated()) {
|
|
|
- LOG.debug("listStatus: list truncated - getting next batch");
|
|
|
- objects = continueListObjects(objects);
|
|
|
- } else {
|
|
|
- break;
|
|
|
- }
|
|
|
+ Listing.FileStatusListingIterator files =
|
|
|
+ listing.createFileStatusListingIterator(path,
|
|
|
+ request,
|
|
|
+ ACCEPT_ALL,
|
|
|
+ new Listing.AcceptAllButSelfAndS3nDirs(path));
|
|
|
+ result = new ArrayList<>(files.getBatchSize());
|
|
|
+ while (files.hasNext()) {
|
|
|
+ result.add(files.next());
|
|
|
}
|
|
|
+ return result.toArray(new FileStatus[result.size()]);
|
|
|
} else {
|
|
|
- LOG.debug("Adding: rd (not a dir): {}", f);
|
|
|
- result.add(fileStatus);
|
|
|
+ LOG.debug("Adding: rd (not a dir): {}", path);
|
|
|
+ FileStatus[] stats = new FileStatus[1];
|
|
|
+ stats[0]= fileStatus;
|
|
|
+ return stats;
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- return result.toArray(new FileStatus[result.size()]);
|
|
|
+ /**
|
|
|
+ * Create a {@code ListObjectsRequest} request against this bucket,
|
|
|
+ * with the maximum keys returned in a query set by {@link #maxKeys}.
|
|
|
+ * @param key key for request
|
|
|
+ * @param delimiter any delimiter
|
|
|
+ * @return the request
|
|
|
+ */
|
|
|
+ private ListObjectsRequest createListObjectsRequest(String key,
|
|
|
+ String delimiter) {
|
|
|
+ ListObjectsRequest request = new ListObjectsRequest();
|
|
|
+ request.setBucketName(bucket);
|
|
|
+ request.setMaxKeys(maxKeys);
|
|
|
+ request.setPrefix(key);
|
|
|
+ if (delimiter != null) {
|
|
|
+ request.setDelimiter(delimiter);
|
|
|
+ }
|
|
|
+ return request;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1371,11 +1403,11 @@ public class S3AFileSystem extends FileSystem {
|
|
|
* @throws java.io.FileNotFoundException when the path does not exist;
|
|
|
* @throws IOException on other problems.
|
|
|
*/
|
|
|
- public S3AFileStatus getFileStatus(Path f) throws IOException {
|
|
|
- String key = pathToKey(f);
|
|
|
+ public S3AFileStatus getFileStatus(final Path f) throws IOException {
|
|
|
incrementStatistic(INVOCATION_GET_FILE_STATUS);
|
|
|
- LOG.debug("Getting path status for {} ({})", f , key);
|
|
|
-
|
|
|
+ final Path path = qualify(f);
|
|
|
+ String key = pathToKey(path);
|
|
|
+ LOG.debug("Getting path status for {} ({})", path , key);
|
|
|
if (!key.isEmpty()) {
|
|
|
try {
|
|
|
ObjectMetadata meta = getObjectMetadata(key);
|
|
@@ -1383,20 +1415,20 @@ public class S3AFileSystem extends FileSystem {
|
|
|
if (objectRepresentsDirectory(key, meta.getContentLength())) {
|
|
|
LOG.debug("Found exact file: fake directory");
|
|
|
return new S3AFileStatus(true, true,
|
|
|
- f.makeQualified(uri, workingDir));
|
|
|
+ path);
|
|
|
} else {
|
|
|
LOG.debug("Found exact file: normal file");
|
|
|
return new S3AFileStatus(meta.getContentLength(),
|
|
|
dateToLong(meta.getLastModified()),
|
|
|
- f.makeQualified(uri, workingDir),
|
|
|
- getDefaultBlockSize(f.makeQualified(uri, workingDir)));
|
|
|
+ path,
|
|
|
+ getDefaultBlockSize(path));
|
|
|
}
|
|
|
} catch (AmazonServiceException e) {
|
|
|
if (e.getStatusCode() != 404) {
|
|
|
- throw translateException("getFileStatus", f, e);
|
|
|
+ throw translateException("getFileStatus", path, e);
|
|
|
}
|
|
|
} catch (AmazonClientException e) {
|
|
|
- throw translateException("getFileStatus", f, e);
|
|
|
+ throw translateException("getFileStatus", path, e);
|
|
|
}
|
|
|
|
|
|
// Necessary?
|
|
@@ -1407,14 +1439,14 @@ public class S3AFileSystem extends FileSystem {
|
|
|
|
|
|
if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
|
|
|
LOG.debug("Found file (with /): fake directory");
|
|
|
- return new S3AFileStatus(true, true, f.makeQualified(uri, workingDir));
|
|
|
+ return new S3AFileStatus(true, true, path);
|
|
|
} else {
|
|
|
LOG.warn("Found file (with /): real file? should not happen: {}", key);
|
|
|
|
|
|
return new S3AFileStatus(meta.getContentLength(),
|
|
|
dateToLong(meta.getLastModified()),
|
|
|
- f.makeQualified(uri, workingDir),
|
|
|
- getDefaultBlockSize(f.makeQualified(uri, workingDir)));
|
|
|
+ path,
|
|
|
+ getDefaultBlockSize(path));
|
|
|
}
|
|
|
} catch (AmazonServiceException e) {
|
|
|
if (e.getStatusCode() != 404) {
|
|
@@ -1427,9 +1459,7 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- if (!key.isEmpty() && !key.endsWith("/")) {
|
|
|
- key = key + "/";
|
|
|
- }
|
|
|
+ key = maybeAddTrailingSlash(key);
|
|
|
ListObjectsRequest request = new ListObjectsRequest();
|
|
|
request.setBucketName(bucket);
|
|
|
request.setPrefix(key);
|
|
@@ -1453,11 +1483,10 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- return new S3AFileStatus(true, false,
|
|
|
- f.makeQualified(uri, workingDir));
|
|
|
+ return new S3AFileStatus(true, false, path);
|
|
|
} else if (key.isEmpty()) {
|
|
|
LOG.debug("Found root directory");
|
|
|
- return new S3AFileStatus(true, true, f.makeQualified(uri, workingDir));
|
|
|
+ return new S3AFileStatus(true, true, path);
|
|
|
}
|
|
|
} catch (AmazonServiceException e) {
|
|
|
if (e.getStatusCode() != 404) {
|
|
@@ -1467,8 +1496,8 @@ public class S3AFileSystem extends FileSystem {
|
|
|
throw translateException("getFileStatus", key, e);
|
|
|
}
|
|
|
|
|
|
- LOG.debug("Not Found: {}", f);
|
|
|
- throw new FileNotFoundException("No such file or directory: " + f);
|
|
|
+ LOG.debug("Not Found: {}", path);
|
|
|
+ throw new FileNotFoundException("No such file or directory: " + path);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1829,7 +1858,15 @@ public class S3AFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Override superclass so as to add statistic collection.
|
|
|
+ * Get the maximum key count.
|
|
|
+ * @return a value, valid after initialization
|
|
|
+ */
|
|
|
+ int getMaxKeys() {
|
|
|
+ return maxKeys;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Increments the statistic {@link Statistic#INVOCATION_GLOB_STATUS}.
|
|
|
* {@inheritDoc}
|
|
|
*/
|
|
|
@Override
|
|
@@ -1849,24 +1886,6 @@ public class S3AFileSystem extends FileSystem {
|
|
|
return super.globStatus(pathPattern, filter);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Override superclass so as to add statistic collection.
|
|
|
- * {@inheritDoc}
|
|
|
- */
|
|
|
- @Override
|
|
|
- public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
|
|
|
- throws FileNotFoundException, IOException {
|
|
|
- incrementStatistic(INVOCATION_LIST_LOCATED_STATUS);
|
|
|
- return super.listLocatedStatus(f);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public RemoteIterator<LocatedFileStatus> listFiles(Path f,
|
|
|
- boolean recursive) throws FileNotFoundException, IOException {
|
|
|
- incrementStatistic(INVOCATION_LIST_FILES);
|
|
|
- return super.listFiles(f, recursive);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Override superclass so as to add statistic collection.
|
|
|
* {@inheritDoc}
|
|
@@ -1897,6 +1916,129 @@ public class S3AFileSystem extends FileSystem {
|
|
|
return super.isFile(f);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * {@inheritDoc}.
|
|
|
+ *
|
|
|
+ * This implementation is optimized for S3, which can do a bulk listing
|
|
|
+ * off all entries under a path in one single operation. Thus there is
|
|
|
+ * no need to recursively walk the directory tree.
|
|
|
+ *
|
|
|
+ * Instead a {@link ListObjectsRequest} is created requesting a (windowed)
|
|
|
+ * listing of all entries under the given path. This is used to construct
|
|
|
+ * an {@code ObjectListingIterator} instance, iteratively returning the
|
|
|
+ * sequence of lists of elements under the path. This is then iterated
|
|
|
+ * over in a {@code FileStatusListingIterator}, which generates
|
|
|
+ * {@link S3AFileStatus} instances, one per listing entry.
|
|
|
+ * These are then translated into {@link LocatedFileStatus} instances.
|
|
|
+ *
|
|
|
+ * This is essentially a nested and wrapped set of iterators, with some
|
|
|
+ * generator classes; an architecture which may become less convoluted
|
|
|
+ * using lambda-expressions.
|
|
|
+ * @param f a path
|
|
|
+ * @param recursive if the subdirectories need to be traversed recursively
|
|
|
+ *
|
|
|
+ * @return an iterator that traverses statuses of the files/directories
|
|
|
+ * in the given path
|
|
|
+ * @throws FileNotFoundException if {@code path} does not exist
|
|
|
+ * @throws IOException if any I/O error occurred
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public RemoteIterator<LocatedFileStatus> listFiles(Path f,
|
|
|
+ boolean recursive) throws FileNotFoundException, IOException {
|
|
|
+ incrementStatistic(INVOCATION_LIST_FILES);
|
|
|
+ Path path = qualify(f);
|
|
|
+ LOG.debug("listFiles({}, {})", path, recursive);
|
|
|
+ try {
|
|
|
+ // lookup dir triggers existence check
|
|
|
+ final FileStatus fileStatus = getFileStatus(path);
|
|
|
+ if (fileStatus.isFile()) {
|
|
|
+ // simple case: File
|
|
|
+ LOG.debug("Path is a file");
|
|
|
+ return new Listing.SingleStatusRemoteIterator(
|
|
|
+ toLocatedFileStatus(fileStatus));
|
|
|
+ } else {
|
|
|
+ // directory: do a bulk operation
|
|
|
+ String key = maybeAddTrailingSlash(pathToKey(path));
|
|
|
+ String delimiter = recursive ? null : "/";
|
|
|
+ LOG.debug("Requesting all entries under {} with delimiter '{}'",
|
|
|
+ key, delimiter);
|
|
|
+ return listing.createLocatedFileStatusIterator(
|
|
|
+ listing.createFileStatusListingIterator(path,
|
|
|
+ createListObjectsRequest(key, delimiter),
|
|
|
+ ACCEPT_ALL,
|
|
|
+ new Listing.AcceptFilesOnly(path)));
|
|
|
+ }
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("listFiles", path, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Override superclass so as to add statistic collection.
|
|
|
+ * {@inheritDoc}
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
|
|
|
+ throws FileNotFoundException, IOException {
|
|
|
+ return listLocatedStatus(f, ACCEPT_ALL);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * {@inheritDoc}.
|
|
|
+ *
|
|
|
+ * S3 Optimized directory listing. The initial operation performs the
|
|
|
+ * first bulk listing; extra listings will take place
|
|
|
+ * when all the current set of results are used up.
|
|
|
+ * @param f a path
|
|
|
+ * @param filter a path filter
|
|
|
+ * @return an iterator that traverses statuses of the files/directories
|
|
|
+ * in the given path
|
|
|
+ * @throws FileNotFoundException if {@code path} does not exist
|
|
|
+ * @throws IOException if any I/O error occurred
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path f,
|
|
|
+ final PathFilter filter)
|
|
|
+ throws FileNotFoundException, IOException {
|
|
|
+ incrementStatistic(INVOCATION_LIST_LOCATED_STATUS);
|
|
|
+ Path path = qualify(f);
|
|
|
+ LOG.debug("listLocatedStatus({}, {}", path, filter);
|
|
|
+ try {
|
|
|
+ // lookup dir triggers existence check
|
|
|
+ final FileStatus fileStatus = getFileStatus(path);
|
|
|
+ if (fileStatus.isFile()) {
|
|
|
+ // simple case: File
|
|
|
+ LOG.debug("Path is a file");
|
|
|
+ return new Listing.SingleStatusRemoteIterator(
|
|
|
+ filter.accept(path) ? toLocatedFileStatus(fileStatus) : null);
|
|
|
+ } else {
|
|
|
+ // directory: trigger a lookup
|
|
|
+ String key = maybeAddTrailingSlash(pathToKey(path));
|
|
|
+ return listing.createLocatedFileStatusIterator(
|
|
|
+ listing.createFileStatusListingIterator(path,
|
|
|
+ createListObjectsRequest(key, "/"),
|
|
|
+ filter,
|
|
|
+ new Listing.AcceptAllButSelfAndS3nDirs(path)));
|
|
|
+ }
|
|
|
+ } catch (AmazonClientException e) {
|
|
|
+ throw translateException("listLocatedStatus", path, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Build a {@link LocatedFileStatus} from a {@link FileStatus} instance.
|
|
|
+ * @param status file status
|
|
|
+ * @return a located status with block locations set up from this FS.
|
|
|
+ * @throws IOException IO Problems.
|
|
|
+ */
|
|
|
+ LocatedFileStatus toLocatedFileStatus(FileStatus status)
|
|
|
+ throws IOException {
|
|
|
+ return new LocatedFileStatus(status,
|
|
|
+ status.isFile() ?
|
|
|
+ getFileBlockLocations(status, 0, status.getLen())
|
|
|
+ : null);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Get a integer option >= the minimum allowed value.
|
|
|
* @param conf configuration
|
|
@@ -1934,38 +2076,4 @@ public class S3AFileSystem extends FileSystem {
|
|
|
return v;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * This is a simple encapsulation of the
|
|
|
- * S3 access key and secret.
|
|
|
- */
|
|
|
- static class AWSAccessKeys {
|
|
|
- private String accessKey = null;
|
|
|
- private String accessSecret = null;
|
|
|
-
|
|
|
- /**
|
|
|
- * Constructor.
|
|
|
- * @param key - AWS access key
|
|
|
- * @param secret - AWS secret key
|
|
|
- */
|
|
|
- public AWSAccessKeys(String key, String secret) {
|
|
|
- accessKey = key;
|
|
|
- accessSecret = secret;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Return the AWS access key.
|
|
|
- * @return key
|
|
|
- */
|
|
|
- public String getAccessKey() {
|
|
|
- return accessKey;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Return the AWS secret key.
|
|
|
- * @return secret
|
|
|
- */
|
|
|
- public String getAccessSecret() {
|
|
|
- return accessSecret;
|
|
|
- }
|
|
|
- }
|
|
|
}
|