Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-11697. Integrate Ozone Filesystem Implementation with Ozone ListStatusLight API #9

Merged
merged 12 commits into from
Nov 18, 2024
Original file line number Diff line number Diff line change
Expand Up @@ -985,8 +985,23 @@ public OzoneDataStreamOutput createStreamFile(String keyName, long size,
*/
public List<OzoneFileStatus> listStatus(String keyName, boolean recursive,
String startKey, long numEntries) throws IOException {
return proxy
.listStatus(volumeName, name, keyName, recursive, startKey, numEntries);
return proxy.listStatus(volumeName, name, keyName, recursive, startKey, numEntries);
}

/**
* List the lightweight status for a file or a directory and its contents.
*
* @param keyName Absolute path of the entry to be listed
* @param recursive For a directory if true all the descendants of a
* particular directory are listed
* @param startKey Key from which listing needs to start. If startKey exists
* its status is included in the final list.
* @param numEntries Number of entries to list from the start key
* @return list of file status
*/
public List<OzoneFileStatusLight> listStatusLight(String keyName, boolean recursive,
String startKey, long numEntries) throws IOException {
return proxy.listStatusLight(volumeName, name, keyName, recursive, startKey, numEntries, false);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,10 @@ public String getOwnerName() {
return ownerName;
}

public long getReplicatedSize() {
return QuotaUtil.getReplicatedSize(getDataSize(), replicationConfig);
}

/**
* Builder of BasicOmKeyInfo.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2112,6 +2112,41 @@ void testListStatus2() throws IOException {
}
}

@Test
public void testOzoneManagerListLocatedStatusAndListStatus() throws IOException {
String data = RandomStringUtils.randomAlphanumeric(20);
String directory = RandomStringUtils.randomAlphanumeric(5);
String filePath = RandomStringUtils.randomAlphanumeric(5);
Path path = createPath("/" + directory + "/" + filePath);
try (FSDataOutputStream stream = fs.create(path)) {
stream.writeBytes(data);
}
RemoteIterator<LocatedFileStatus> listLocatedStatus = fs.listLocatedStatus(path);
int count = 0;
while (listLocatedStatus.hasNext()) {
LocatedFileStatus locatedFileStatus = listLocatedStatus.next();
assertTrue(locatedFileStatus.getBlockLocations().length >= 1);

for (BlockLocation blockLocation : locatedFileStatus.getBlockLocations()) {
assertTrue(blockLocation.getNames().length >= 1);
assertTrue(blockLocation.getHosts().length >= 1);
}
count++;
}
assertEquals(1, count);
count = 0;
RemoteIterator<FileStatus> listStatus = fs.listStatusIterator(path);
while (listStatus.hasNext()) {
FileStatus fileStatus = listStatus.next();
assertFalse(fileStatus instanceof LocatedFileStatus);
count++;
}
assertEquals(1, count);
FileStatus[] fileStatuses = fs.listStatus(path.getParent());
assertEquals(1, fileStatuses.length);
assertFalse(fileStatuses[0] instanceof LocatedFileStatus);
}

@Test
void testOzoneManagerFileSystemInterface() throws IOException {
String dirPath = RandomStringUtils.randomAlphanumeric(5);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1071,7 +1071,7 @@ private void listStatusRecursiveHelper(Path curPath, List<FileStatus> result)
private List<FileStatus> callAdapterListStatus(String pathStr,
boolean recursive, String startPath, long numEntries) throws IOException {
return adapter.listStatus(pathStr, recursive, startPath, numEntries,
ofs.getUri(), ofs.getWorkingDirectory(), ofs.getUsername())
ofs.getUri(), ofs.getWorkingDirectory(), ofs.getUsername(), false)
.stream().map(ofs::convertFileStatus).collect(Collectors.toList());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
import org.apache.hadoop.ozone.om.helpers.OzoneFileStatusLight;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.TestInstance;
Expand Down Expand Up @@ -151,5 +152,11 @@ private void assertListStatus(OzoneBucket bucket, String keyName,

List<?> versions = files.get(0).getKeyInfo().getKeyLocationVersions();
assertEquals(expectedVersionCount, versions.size());

List<OzoneFileStatusLight> lightFiles = bucket.listStatusLight(keyName, false, "", 1);

assertNotNull(lightFiles);
assertEquals(1, lightFiles.size());

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@
import org.apache.hadoop.ozone.client.rpc.RpcClient;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.BasicOmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
Expand All @@ -77,6 +78,7 @@
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
import org.apache.hadoop.ozone.om.helpers.OzoneFileStatusLight;
import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
import org.apache.hadoop.ozone.snapshot.SnapshotDiffReportOzone;
import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
Expand Down Expand Up @@ -436,15 +438,22 @@ public Iterator<BasicKeyInfo> listKeys(String pathKey) throws IOException {
@Override
public List<FileStatusAdapter> listStatus(String keyName, boolean recursive,
String startKey, long numEntries, URI uri,
Path workingDir, String username) throws IOException {
Path workingDir, String username, boolean lite) throws IOException {
try {
incrementCounter(Statistic.OBJECTS_LIST, 1);
List<OzoneFileStatus> statuses = bucket
.listStatus(keyName, recursive, startKey, numEntries);

List<FileStatusAdapter> result = new ArrayList<>();
for (OzoneFileStatus status : statuses) {
result.add(toFileStatusAdapter(status, username, uri, workingDir));
if (lite) {
List<OzoneFileStatusLight> statuses = bucket
.listStatusLight(keyName, recursive, startKey, numEntries);
for (OzoneFileStatusLight status : statuses) {
result.add(toFileStatusAdapter(status, username, uri, workingDir));
}
} else {
List<OzoneFileStatus> statuses = bucket
.listStatus(keyName, recursive, startKey, numEntries);
for (OzoneFileStatus status : statuses) {
result.add(toFileStatusAdapter(status, username, uri, workingDir));
}
}
return result;
} catch (OMException e) {
Expand Down Expand Up @@ -549,6 +558,31 @@ private FileStatusAdapter toFileStatusAdapter(OzoneFileStatus status,
);
}

private FileStatusAdapter toFileStatusAdapter(OzoneFileStatusLight status,
String owner, URI defaultUri, Path workingDir) {
BasicOmKeyInfo keyInfo = status.getKeyInfo();
short replication = (short) keyInfo.getReplicationConfig()
.getRequiredNodes();
return new FileStatusAdapter(
keyInfo.getDataSize(),
keyInfo.getReplicatedSize(),
new Path(OZONE_URI_DELIMITER + keyInfo.getKeyName())
.makeQualified(defaultUri, workingDir),
status.isDirectory(),
replication,
status.getBlockSize(),
keyInfo.getModificationTime(),
keyInfo.getModificationTime(),
status.isDirectory() ? (short) 00777 : (short) 00666,
StringUtils.defaultIfEmpty(keyInfo.getOwnerName(), owner),
owner,
null,
getBlockLocations(null),
false,
OzoneClientUtils.isKeyErasureCode(keyInfo)
);
}

/**
* Helper method to get List of BlockLocation from OM Key info.
* @param fileStatus Ozone key file status.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hadoop.fs.ozone;

import com.google.common.base.Function;
import com.google.common.base.Preconditions;

import org.apache.hadoop.conf.Configuration;
Expand Down Expand Up @@ -688,7 +689,7 @@ public FileStatus[] listStatus(Path f) throws IOException {
do {
tmpStatusList =
adapter.listStatus(pathToKey(f), false, startKey, numEntries, uri,
workingDir, getUsername())
workingDir, getUsername(), true)
.stream()
.map(this::convertFileStatus)
.collect(Collectors.toList());
Expand Down Expand Up @@ -947,13 +948,15 @@ public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive)
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
throws IOException {
incrementCounter(Statistic.INVOCATION_LIST_LOCATED_STATUS);
return super.listLocatedStatus(f);
return new OzoneFileStatusIterator<>(f,
(stat) -> stat instanceof LocatedFileStatus ? (LocatedFileStatus) stat : new LocatedFileStatus(stat, null),
false);
}

@Override
public RemoteIterator<FileStatus> listStatusIterator(Path f)
throws IOException {
return new OzoneFileStatusIterator<>(f);
return new OzoneFileStatusIterator<>(f, stat -> stat, true);
}

@Override
Expand Down Expand Up @@ -986,7 +989,6 @@ public void setTimes(Path f, long mtime, long atime) throws IOException {
String key = pathToKey(qualifiedPath);
adapter.setTimes(key, mtime, atime);
}

/**
* A private class implementation for iterating list of file status.
*
Expand All @@ -999,18 +1001,24 @@ private final class OzoneFileStatusIterator<T extends FileStatus>
private Path p;
private T curStat = null;
private String startPath = "";
private boolean lite;
private Function<FileStatus, T> transformFunc;

/**
* Constructor to initialize OzoneFileStatusIterator.
* Get the first batch of entry for iteration.
*
* @param p path to file/directory.
* @param transformFunc function to convert FileStatus into an expected type.
* @param lite if true it should look into fetching a lightweight keys from server.
* @throws IOException
*/
private OzoneFileStatusIterator(Path p) throws IOException {
private OzoneFileStatusIterator(Path p, Function<FileStatus, T> transformFunc, boolean lite) throws IOException {
this.p = p;
this.lite = lite;
this.transformFunc = transformFunc;
// fetch the first batch of entries in the directory
thisListing = listFileStatus(p, startPath);
thisListing = listFileStatus(p, startPath, lite);
if (thisListing != null && !thisListing.isEmpty()) {
startPath = pathToKey(
thisListing.get(thisListing.size() - 1).getPath());
Expand All @@ -1029,7 +1037,7 @@ public boolean hasNext() throws IOException {
while (curStat == null && hasNextNoFilter()) {
T next;
FileStatus fileStat = thisListing.get(i++);
next = (T) (fileStat);
next = this.transformFunc.apply(fileStat);
curStat = next;
}
return curStat != null;
Expand All @@ -1050,7 +1058,7 @@ private boolean hasNextNoFilter() throws IOException {
if (startPath != null && (thisListing.size() == listingPageSize ||
thisListing.size() == listingPageSize - 1)) {
// current listing is exhausted & fetch a new listing
thisListing = listFileStatus(p, startPath);
thisListing = listFileStatus(p, startPath, lite);
if (thisListing != null && !thisListing.isEmpty()) {
startPath = pathToKey(
thisListing.get(thisListing.size() - 1).getPath());
Expand Down Expand Up @@ -1085,18 +1093,19 @@ public T next() throws IOException {
*
* @param f
* @param startPath
* @param lite if true return lightweight keys
* @return list of file status.
* @throws IOException
*/
private List<FileStatus> listFileStatus(Path f, String startPath)
private List<FileStatus> listFileStatus(Path f, String startPath, boolean lite)
throws IOException {
incrementCounter(Statistic.INVOCATION_LIST_STATUS, 1);
statistics.incrementReadOps(1);
LOG.trace("listFileStatus() path:{}", f);
List<FileStatus> statusList;
statusList =
adapter.listStatus(pathToKey(f), false, startPath,
listingPageSize, uri, workingDir, getUsername())
listingPageSize, uri, workingDir, getUsername(), lite)
.stream()
.map(this::convertFileStatus)
.collect(Collectors.toList());
Expand Down
Loading