|
@@ -25,20 +25,18 @@ import java.net.URISyntaxException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Objects;
|
|
|
-import java.util.Iterator;
|
|
|
import java.util.regex.Matcher;
|
|
|
import java.util.regex.Pattern;
|
|
|
-
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
import java.util.stream.Collectors;
|
|
|
import java.util.stream.Stream;
|
|
|
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
-import org.slf4j.Logger;
|
|
|
-import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
+import org.apache.hadoop.classification.InterfaceStability;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
@@ -47,34 +45,25 @@ import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
|
|
|
-import org.apache.hadoop.ozone.client.ObjectStore;
|
|
|
-import org.apache.hadoop.ozone.client.OzoneBucket;
|
|
|
-import org.apache.hadoop.ozone.client.OzoneClient;
|
|
|
-import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
|
|
-import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
-import org.apache.hadoop.ozone.client.OzoneKey;
|
|
|
-import org.apache.hadoop.ozone.client.OzoneVolume;
|
|
|
-import org.apache.hadoop.hdds.client.ReplicationFactor;
|
|
|
-import org.apache.hadoop.hdds.client.ReplicationType;
|
|
|
-import org.apache.http.client.utils.URIBuilder;
|
|
|
-import org.apache.commons.lang3.StringUtils;
|
|
|
-import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
-import org.apache.hadoop.classification.InterfaceStability;
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.util.Progressable;
|
|
|
-import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import org.apache.commons.lang3.StringUtils;
|
|
|
+import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
|
|
|
import static org.apache.hadoop.fs.ozone.Constants.OZONE_DEFAULT_USER;
|
|
|
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
|
|
|
import static org.apache.hadoop.fs.ozone.Constants.OZONE_USER_DIR;
|
|
|
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
|
|
|
-import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
|
|
|
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
|
|
|
+import org.apache.http.client.utils.URIBuilder;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
/**
|
|
|
* The Ozone Filesystem implementation.
|
|
|
- *
|
|
|
+ * <p>
|
|
|
* 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 {@link OzoneFileSystem}, extra methods and features may be
|
|
@@ -85,16 +74,15 @@ import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
|
|
|
public class OzoneFileSystem extends FileSystem {
|
|
|
static final Logger LOG = LoggerFactory.getLogger(OzoneFileSystem.class);
|
|
|
|
|
|
- /** The Ozone client for connecting to Ozone server. */
|
|
|
- private OzoneClient ozoneClient;
|
|
|
- private ObjectStore objectStore;
|
|
|
- private OzoneVolume volume;
|
|
|
- private OzoneBucket bucket;
|
|
|
+ /**
|
|
|
+ * The Ozone client for connecting to Ozone server.
|
|
|
+ */
|
|
|
+
|
|
|
private URI uri;
|
|
|
private String userName;
|
|
|
private Path workingDir;
|
|
|
- private ReplicationType replicationType;
|
|
|
- private ReplicationFactor replicationFactor;
|
|
|
+
|
|
|
+ private OzoneClientAdapter adapter;
|
|
|
|
|
|
private static final Pattern URL_SCHEMA_PATTERN =
|
|
|
Pattern.compile("(.+)\\.([^\\.]+)");
|
|
@@ -102,11 +90,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void initialize(URI name, Configuration conf) throws IOException {
|
|
|
super.initialize(name, conf);
|
|
|
- if(!(conf instanceof OzoneConfiguration)) {
|
|
|
- setConf(new OzoneConfiguration(conf));
|
|
|
- } else {
|
|
|
- setConf(conf);
|
|
|
- }
|
|
|
+ setConf(conf);
|
|
|
Objects.requireNonNull(name.getScheme(), "No scheme provided in " + name);
|
|
|
assert getScheme().equals(name.getScheme());
|
|
|
|
|
@@ -125,16 +109,32 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
uri = new URIBuilder().setScheme(OZONE_URI_SCHEME)
|
|
|
.setHost(authority).build();
|
|
|
LOG.trace("Ozone URI for ozfs initialization is " + uri);
|
|
|
- this.ozoneClient = OzoneClientFactory.getRpcClient(getConf());
|
|
|
- objectStore = ozoneClient.getObjectStore();
|
|
|
- this.volume = objectStore.getVolume(volumeStr);
|
|
|
- this.bucket = volume.getBucket(bucketStr);
|
|
|
- this.replicationType = ReplicationType.valueOf(
|
|
|
- getConf().get(OzoneConfigKeys.OZONE_REPLICATION_TYPE,
|
|
|
- OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT));
|
|
|
- this.replicationFactor = ReplicationFactor.valueOf(
|
|
|
- getConf().getInt(OzoneConfigKeys.OZONE_REPLICATION,
|
|
|
- OzoneConfigKeys.OZONE_REPLICATION_DEFAULT));
|
|
|
+
|
|
|
+ //isolated is the default for ozonefs-lib-legacy which includes the
|
|
|
+ // /ozonefs.txt, otherwise the default is false. It could be overridden.
|
|
|
+ boolean defaultValue =
|
|
|
+ OzoneFileSystem.class.getClassLoader().getResource("ozonefs.txt")
|
|
|
+ != null;
|
|
|
+
|
|
|
+ //Use string here instead of the constant as constant may not be available
|
|
|
+ //on the classpath of a hadoop 2.7
|
|
|
+ boolean isolatedClassloader =
|
|
|
+ conf.getBoolean("ozone.fs.isolated-classloader", defaultValue);
|
|
|
+
|
|
|
+ if (isolatedClassloader) {
|
|
|
+ this.adapter =
|
|
|
+ OzoneClientAdapterFactory.createAdapter(volumeStr, bucketStr);
|
|
|
+ } else {
|
|
|
+ OzoneConfiguration ozoneConfiguration;
|
|
|
+ if (conf instanceof OzoneConfiguration) {
|
|
|
+ ozoneConfiguration = (OzoneConfiguration) conf;
|
|
|
+ } else {
|
|
|
+ ozoneConfiguration = new OzoneConfiguration(conf);
|
|
|
+ }
|
|
|
+ this.adapter = new OzoneClientAdapterImpl(ozoneConfiguration,
|
|
|
+ volumeStr, bucketStr);
|
|
|
+ }
|
|
|
+
|
|
|
try {
|
|
|
this.userName =
|
|
|
UserGroupInformation.getCurrentUser().getShortUserName();
|
|
@@ -142,7 +142,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
this.userName = OZONE_DEFAULT_USER;
|
|
|
}
|
|
|
this.workingDir = new Path(OZONE_USER_DIR, this.userName)
|
|
|
- .makeQualified(this.uri, this.workingDir);
|
|
|
+ .makeQualified(this.uri, this.workingDir);
|
|
|
} catch (URISyntaxException ue) {
|
|
|
final String msg = "Invalid Ozone endpoint " + name;
|
|
|
LOG.error(msg, ue);
|
|
@@ -153,7 +153,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void close() throws IOException {
|
|
|
try {
|
|
|
- ozoneClient.close();
|
|
|
+ adapter.close();
|
|
|
} finally {
|
|
|
super.close();
|
|
|
}
|
|
@@ -179,7 +179,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
return new FSDataInputStream(
|
|
|
- new OzoneFSInputStream(bucket.readKey(key).getInputStream()));
|
|
|
+ new OzoneFSInputStream(adapter.createInputStream(key)));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -200,19 +200,16 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
throw new FileAlreadyExistsException(f + " already exists");
|
|
|
}
|
|
|
LOG.trace("Overwriting file {}", f);
|
|
|
- deleteObject(key);
|
|
|
+ adapter.deleteObject(key);
|
|
|
}
|
|
|
} catch (FileNotFoundException ignored) {
|
|
|
// this means the file is not found
|
|
|
}
|
|
|
|
|
|
- OzoneOutputStream ozoneOutputStream =
|
|
|
- bucket.createKey(key, 0, replicationType, replicationFactor,
|
|
|
- new HashMap<>());
|
|
|
// We pass null to FSDataOutputStream so it won't count writes that
|
|
|
// are being buffered to a file
|
|
|
return new FSDataOutputStream(
|
|
|
- new OzoneFSOutputStream(ozoneOutputStream.getOutputStream()), null);
|
|
|
+ adapter.createKey(key), null);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -236,7 +233,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
|
|
|
@Override
|
|
|
public FSDataOutputStream append(Path f, int bufferSize,
|
|
|
- Progressable progress) throws IOException {
|
|
|
+ Progressable progress) throws IOException {
|
|
|
throw new UnsupportedOperationException("append() Not implemented by the "
|
|
|
+ getClass().getSimpleName() + " FileSystem implementation");
|
|
|
}
|
|
@@ -256,7 +253,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
boolean processKey(String key) throws IOException {
|
|
|
String newKeyName = dstKey.concat(key.substring(srcKey.length()));
|
|
|
- bucket.renameKey(key, newKeyName);
|
|
|
+ adapter.renameKey(key, newKeyName);
|
|
|
return true;
|
|
|
}
|
|
|
}
|
|
@@ -264,7 +261,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
/**
|
|
|
* Check whether the source and destination path are valid and then perform
|
|
|
* rename from source path to destination path.
|
|
|
- *
|
|
|
+ * <p>
|
|
|
* The rename operation is performed by renaming the keys with src as prefix.
|
|
|
* For such keys the prefix is changed from src to dst.
|
|
|
*
|
|
@@ -361,6 +358,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
|
|
|
private class DeleteIterator extends OzoneListingIterator {
|
|
|
private boolean recursive;
|
|
|
+
|
|
|
DeleteIterator(Path f, boolean recursive)
|
|
|
throws IOException {
|
|
|
super(f);
|
|
@@ -379,7 +377,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
return true;
|
|
|
} else {
|
|
|
LOG.trace("deleting key:" + key);
|
|
|
- boolean succeed = deleteObject(key);
|
|
|
+ boolean succeed = adapter.deleteObject(key);
|
|
|
// if recursive delete is requested ignore the return value of
|
|
|
// deleteObject and issue deletes for other keys.
|
|
|
return recursive || succeed;
|
|
@@ -390,6 +388,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
/**
|
|
|
* Deletes the children of the input dir path by iterating though the
|
|
|
* DeleteIterator.
|
|
|
+ *
|
|
|
* @param f directory path to be deleted
|
|
|
* @return true if successfully deletes all required keys, false otherwise
|
|
|
* @throws IOException
|
|
@@ -431,7 +430,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
result = innerDelete(f, recursive);
|
|
|
} else {
|
|
|
LOG.debug("delete: Path is a file: {}", f);
|
|
|
- result = deleteObject(key);
|
|
|
+ result = adapter.deleteObject(key);
|
|
|
}
|
|
|
|
|
|
if (result) {
|
|
@@ -449,6 +448,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
/**
|
|
|
* Create a fake parent directory key if it does not already exist and no
|
|
|
* other child of this parent directory exists.
|
|
|
+ *
|
|
|
* @param f path to the fake parent directory
|
|
|
* @throws IOException
|
|
|
*/
|
|
@@ -457,12 +457,13 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
if (!key.isEmpty() && !o3Exists(f)) {
|
|
|
LOG.debug("Creating new fake directory at {}", f);
|
|
|
String dirKey = addTrailingSlashIfNeeded(key);
|
|
|
- createDirectory(dirKey);
|
|
|
+ adapter.createDirectory(dirKey);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Check if a file or directory exists corresponding to given path.
|
|
|
+ *
|
|
|
* @param f path to file/directory.
|
|
|
* @return true if it exists, false otherwise.
|
|
|
* @throws IOException
|
|
@@ -487,7 +488,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
new HashMap<>(LISTING_PAGE_SIZE);
|
|
|
private Path f; // the input path
|
|
|
|
|
|
- ListStatusIterator(Path f) throws IOException {
|
|
|
+ ListStatusIterator(Path f) throws IOException {
|
|
|
super(f);
|
|
|
this.f = f;
|
|
|
}
|
|
@@ -495,6 +496,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
/**
|
|
|
* Add the key to the listStatus result if the key corresponds to the
|
|
|
* input path or is an immediate child of the input path.
|
|
|
+ *
|
|
|
* @param key key to be processed
|
|
|
* @return always returns true
|
|
|
* @throws IOException
|
|
@@ -518,7 +520,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
if (pathToKey(keyPath.getParent()).equals(pathToKey(f))) {
|
|
|
// This key is an immediate child. Can be file or directory
|
|
|
if (key.endsWith(OZONE_URI_DELIMITER)) {
|
|
|
- // Key is a directory
|
|
|
+ // Key is a directory
|
|
|
addSubDirStatus(keyPath);
|
|
|
} else {
|
|
|
addFileStatus(keyPath);
|
|
@@ -537,6 +539,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
|
|
|
/**
|
|
|
* Adds the FileStatus of keyPath to final result of listStatus.
|
|
|
+ *
|
|
|
* @param filePath path to the file
|
|
|
* @throws FileNotFoundException
|
|
|
*/
|
|
@@ -547,6 +550,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
/**
|
|
|
* Adds the FileStatus of the subdir to final result of listStatus, if not
|
|
|
* already included.
|
|
|
+ *
|
|
|
* @param dirPath path to the dir
|
|
|
* @throws FileNotFoundException
|
|
|
*/
|
|
@@ -560,9 +564,9 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
/**
|
|
|
* Traverse the parent directory structure of keyPath to determine the
|
|
|
* which parent/ grand-parent/.. is the immediate child of the input path f.
|
|
|
+ *
|
|
|
* @param keyPath path whose parent directory structure should be traversed.
|
|
|
* @return immediate child path of the input path f.
|
|
|
- * @return immediate child path of the input path f.
|
|
|
*/
|
|
|
Path getImmediateChildPath(Path keyPath) {
|
|
|
Path path = keyPath;
|
|
@@ -610,6 +614,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
|
|
|
/**
|
|
|
* Get the username of the FS.
|
|
|
+ *
|
|
|
* @return the short name of the user who instantiated the FS
|
|
|
*/
|
|
|
public String getUsername() {
|
|
@@ -648,7 +653,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
LOG.trace("creating directory for fpart:{}", fPart);
|
|
|
String key = pathToKey(fPart);
|
|
|
String dirKey = addTrailingSlashIfNeeded(key);
|
|
|
- if (!createDirectory(dirKey)) {
|
|
|
+ if (!adapter.createDirectory(dirKey)) {
|
|
|
// Directory creation failed here,
|
|
|
// rollback and delete newly created directories
|
|
|
LOG.trace("Directory creation failed, path:{}", fPart);
|
|
@@ -682,11 +687,11 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
|
|
|
if (key.length() == 0) {
|
|
|
return new FileStatus(0, true, 1, 0,
|
|
|
- bucket.getCreationTime(), qualifiedPath);
|
|
|
+ adapter.getCreationTime(), qualifiedPath);
|
|
|
}
|
|
|
|
|
|
// Check if the key exists
|
|
|
- OzoneKey ozoneKey = getKeyInfo(key);
|
|
|
+ BasicKeyInfo ozoneKey = adapter.getKeyInfo(key);
|
|
|
if (ozoneKey != null) {
|
|
|
LOG.debug("Found exact file for path {}: normal file", f);
|
|
|
return new FileStatus(ozoneKey.getDataSize(), false, 1,
|
|
@@ -702,6 +707,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
* Get the FileStatus for input directory path.
|
|
|
* They key corresponding to input path is appended with a trailing slash
|
|
|
* to return only the corresponding directory key in the bucket.
|
|
|
+ *
|
|
|
* @param f directory path
|
|
|
* @return FileStatus for the input directory path
|
|
|
* @throws FileNotFoundException
|
|
@@ -712,9 +718,9 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
String key = pathToKey(qualifiedPath);
|
|
|
key = addTrailingSlashIfNeeded(key);
|
|
|
|
|
|
- OzoneKey ozoneKey = getKeyInfo(key);
|
|
|
- if(ozoneKey != null) {
|
|
|
- if (isDirectory(ozoneKey)) {
|
|
|
+ BasicKeyInfo ozoneKey = adapter.getKeyInfo(key);
|
|
|
+ if (ozoneKey != null) {
|
|
|
+ if (adapter.isDirectory(ozoneKey)) {
|
|
|
// Key is a directory
|
|
|
LOG.debug("Found file (with /) for path {}: fake directory", f);
|
|
|
} else {
|
|
@@ -730,7 +736,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
|
|
|
// File or directory corresponding to input path does not exist.
|
|
|
// Check if there exists a key prefixed with this key.
|
|
|
- boolean hasChildren = bucket.listKeys(key).hasNext();
|
|
|
+ boolean hasChildren = adapter.hasNextKey(key);
|
|
|
if (hasChildren) {
|
|
|
return new FileStatus(0, true, 1, 0, 0, 0, FsPermission.getDirDefault(),
|
|
|
getUsername(), getUsername(), qualifiedPath);
|
|
@@ -739,65 +745,6 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
throw new FileNotFoundException(f + ": No such file or directory!");
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Helper method to fetch the key metadata info.
|
|
|
- * @param key key whose metadata information needs to be fetched
|
|
|
- * @return metadata info of the key
|
|
|
- */
|
|
|
- private OzoneKey getKeyInfo(String key) {
|
|
|
- try {
|
|
|
- return bucket.getKey(key);
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.trace("Key:{} does not exist", key);
|
|
|
- return null;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Helper method to check if an Ozone key is representing a directory.
|
|
|
- * @param key key to be checked as a directory
|
|
|
- * @return true if key is a directory, false otherwise
|
|
|
- */
|
|
|
- private boolean isDirectory(OzoneKey key) {
|
|
|
- LOG.trace("key name:{} size:{}", key.getName(),
|
|
|
- key.getDataSize());
|
|
|
- return key.getName().endsWith(OZONE_URI_DELIMITER)
|
|
|
- && (key.getDataSize() == 0);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Helper method to create an directory specified by key name in bucket.
|
|
|
- * @param keyName key name to be created as directory
|
|
|
- * @return true if the key is created, false otherwise
|
|
|
- */
|
|
|
- private boolean createDirectory(String keyName) {
|
|
|
- try {
|
|
|
- LOG.trace("creating dir for key:{}", keyName);
|
|
|
- bucket.createKey(keyName, 0, replicationType, replicationFactor,
|
|
|
- new HashMap<>()).close();
|
|
|
- return true;
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.error("create key failed for key:{}", keyName, ioe);
|
|
|
- return false;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Helper method to delete an object specified by key name in bucket.
|
|
|
- * @param keyName key name to be deleted
|
|
|
- * @return true if the key is deleted, false otherwise
|
|
|
- */
|
|
|
- private boolean deleteObject(String keyName) {
|
|
|
- LOG.trace("issuing delete for key" + keyName);
|
|
|
- try {
|
|
|
- bucket.deleteKey(keyName);
|
|
|
- return true;
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.error("delete key failed " + ioe.getMessage());
|
|
|
- return false;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Turn a path (relative or otherwise) into an Ozone key.
|
|
|
*
|
|
@@ -805,7 +752,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
* @return the key of the object that represents the file.
|
|
|
*/
|
|
|
public String pathToKey(Path path) {
|
|
|
- Objects.requireNonNull(path, "Path can not be null!");
|
|
|
+ Objects.requireNonNull(path, "Path canf not be null!");
|
|
|
if (!path.isAbsolute()) {
|
|
|
path = new Path(workingDir, path);
|
|
|
}
|
|
@@ -839,17 +786,17 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * This class provides an interface to iterate through all the keys in the
|
|
|
- * bucket prefixed with the input path key and process them.
|
|
|
- *
|
|
|
- * Each implementing class should define how the keys should be processed
|
|
|
- * through the processKey() function.
|
|
|
+ * This class provides an interface to iterate through all the keys in the
|
|
|
+ * bucket prefixed with the input path key and process them.
|
|
|
+ * <p>
|
|
|
+ * Each implementing class should define how the keys should be processed
|
|
|
+ * through the processKey() function.
|
|
|
*/
|
|
|
private abstract class OzoneListingIterator {
|
|
|
private final Path path;
|
|
|
private final FileStatus status;
|
|
|
private String pathKey;
|
|
|
- private Iterator<? extends OzoneKey> keyIterator;
|
|
|
+ private Iterator<BasicKeyInfo> keyIterator;
|
|
|
|
|
|
OzoneListingIterator(Path path)
|
|
|
throws IOException {
|
|
@@ -859,12 +806,13 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
if (status.isDirectory()) {
|
|
|
this.pathKey = addTrailingSlashIfNeeded(pathKey);
|
|
|
}
|
|
|
- keyIterator = bucket.listKeys(pathKey);
|
|
|
+ keyIterator = adapter.listKeys(pathKey);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* The output of processKey determines if further iteration through the
|
|
|
* keys should be done or not.
|
|
|
+ *
|
|
|
* @return true if we should continue iteration of keys, false otherwise.
|
|
|
* @throws IOException
|
|
|
*/
|
|
@@ -876,6 +824,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
* If for any key, the processKey() returns false, then the iteration is
|
|
|
* stopped and returned with false indicating that all the keys could not
|
|
|
* be processed successfully.
|
|
|
+ *
|
|
|
* @return true if all keys are processed successfully, false otherwise.
|
|
|
* @throws IOException
|
|
|
*/
|
|
@@ -884,7 +833,7 @@ public class OzoneFileSystem extends FileSystem {
|
|
|
if (status.isDirectory()) {
|
|
|
LOG.trace("Iterating directory:{}", pathKey);
|
|
|
while (keyIterator.hasNext()) {
|
|
|
- OzoneKey key = keyIterator.next();
|
|
|
+ BasicKeyInfo key = keyIterator.next();
|
|
|
LOG.trace("iterating key:{}", key.getName());
|
|
|
if (!processKey(key.getName())) {
|
|
|
return false;
|