Skip to content

HDDS-5450. Avoid refresh pipeline for S3 headObject #2431

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

Merged
merged 5 commits into from
Jul 27, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -529,6 +529,22 @@ public OzoneKeyDetails getKey(String key) throws IOException {
return proxy.getKeyDetails(volumeName, name, key);
}

/**
*
* Returns OzoneKey that contains the application generated/visible
* metadata for an Ozone Object.
*
* If Key exists, return returns OzoneKey.
* If Key does not exist, throws an exception with error code KEY_NOT_FOUND
*
* @param key
* @return OzoneKey which gives basic information about the key.
* @throws IOException
*/
public OzoneKey headObject(String key) throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rename to getOzoneKeyMetadata?

Copy link
Contributor Author

@bharatviswa504 bharatviswa504 Jul 20, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is getKey which does the same. To be clear, we can leave it as headObject. (As main use of this API is for S3 headObject)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

return proxy.headObject(volumeName, name, key);
}

public long getUsedBytes() {
return usedBytes;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,4 +164,8 @@ public int getReplicationFactor() {
return replicationConfig.getRequiredNodes();
}

public ReplicationConfig getReplicationConfig() {
return replicationConfig;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -736,4 +736,20 @@ List<OzoneFileStatus> listStatus(String volumeName, String bucketName,
*/
void setBucketQuota(String volumeName, String bucketName,
long quotaInNamespace, long quotaInBytes) throws IOException;

/**
* Returns OzoneKey that contains the application generated/visible
* metadata for an Ozone Object.
*
* If Key exists, return returns OzoneKey.
* If Key does not exist, throws an exception with error code KEY_NOT_FOUND
*
* @param volumeName
* @param bucketName
* @param keyName
* @return OzoneKey which gives basic information about the key.
* @throws IOException
*/
OzoneKey headObject(String volumeName, String bucketName,
String keyName) throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -1450,4 +1450,25 @@ public OzoneManagerProtocol getOzoneManagerClient() {
public Cache<URI, KeyProvider> getKeyProviderCache() {
return keyProviderCache;
}

@Override
public OzoneKey headObject(String volumeName, String bucketName,
String keyName) throws IOException {
Preconditions.checkNotNull(volumeName);
Preconditions.checkNotNull(bucketName);
Preconditions.checkNotNull(keyName);
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(volumeName)
.setBucketName(bucketName)
.setKeyName(keyName)
.setLatestVersionLocation(true)
.setHeadOp(true)
.build();
OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);

return new OzoneKey(keyInfo.getVolumeName(), keyInfo.getBucketName(),
keyInfo.getKeyName(), keyInfo.getDataSize(), keyInfo.getCreationTime(),
keyInfo.getModificationTime(), keyInfo.getReplicationConfig());

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ public final class OmKeyArgs implements Auditable {
private List<OzoneAcl> acls;
private boolean latestVersionLocation;
private boolean recursive;
private boolean headOp;

@SuppressWarnings("parameternumber")
private OmKeyArgs(String volumeName, String bucketName, String keyName,
Expand All @@ -56,7 +57,7 @@ private OmKeyArgs(String volumeName, String bucketName, String keyName,
String uploadID, int partNumber,
Map<String, String> metadataMap, boolean refreshPipeline,
List<OzoneAcl> acls, boolean sortDatanode,
boolean latestVersionLocation, boolean recursive) {
boolean latestVersionLocation, boolean recursive, boolean headOp) {
this.volumeName = volumeName;
this.bucketName = bucketName;
this.keyName = keyName;
Expand All @@ -72,6 +73,7 @@ private OmKeyArgs(String volumeName, String bucketName, String keyName,
this.sortDatanodesInPipeline = sortDatanode;
this.latestVersionLocation = latestVersionLocation;
this.recursive = recursive;
this.headOp = headOp;
}

public boolean getIsMultipartKey() {
Expand Down Expand Up @@ -146,6 +148,10 @@ public boolean isRecursive() {
return recursive;
}

public boolean isHeadOp() {
return headOp;
}

@Override
public Map<String, String> toAuditMap() {
Map<String, String> auditMap = new LinkedHashMap<>();
Expand Down Expand Up @@ -204,6 +210,7 @@ public static class Builder {
private boolean latestVersionLocation;
private List<OzoneAcl> acls;
private boolean recursive;
private boolean headOp;

public Builder setVolumeName(String volume) {
this.volumeName = volume;
Expand Down Expand Up @@ -285,12 +292,17 @@ public Builder setRecursive(boolean isRecursive) {
return this;
}

public Builder setHeadOp(boolean isHeadOp) {
this.headOp = isHeadOp;
return this;
}

public OmKeyArgs build() {
return new OmKeyArgs(volumeName, bucketName, keyName, dataSize,
replicationConfig, locationInfoList, isMultipartKey,
multipartUploadID,
multipartUploadPartNumber, metadata, refreshPipeline, acls,
sortDatanodesInPipeline, latestVersionLocation, recursive);
sortDatanodesInPipeline, latestVersionLocation, recursive, headOp);
}

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -690,6 +690,7 @@ public OmKeyInfo lookupKey(OmKeyArgs args) throws IOException {
.setDataSize(args.getDataSize())
.setSortDatanodes(args.getSortDatanodes())
.setLatestVersionLocation(args.getLatestVersionLocation())
.setHeadOp(args.isHeadOp())
.build();
req.setKeyArgs(keyArgs);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,9 +113,11 @@
import org.apache.commons.lang3.RandomUtils;
import org.apache.commons.lang3.StringUtils;
import static org.apache.hadoop.hdds.StringUtils.string2Bytes;
import static org.apache.hadoop.hdds.client.ReplicationConfig.fromTypeAndFactor;
import static org.apache.hadoop.hdds.client.ReplicationFactor.ONE;
import static org.apache.hadoop.hdds.client.ReplicationFactor.THREE;
import static org.apache.hadoop.hdds.client.ReplicationType.STAND_ALONE;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
import static org.apache.hadoop.ozone.OmUtils.MAX_TRXN_ID;
import static org.apache.hadoop.ozone.OzoneAcl.AclScope.ACCESS;
import static org.apache.hadoop.ozone.OzoneAcl.AclScope.DEFAULT;
Expand Down Expand Up @@ -3599,4 +3601,43 @@ public void testSetS3VolumeAcl() throws Exception {

Assert.assertTrue(ozoneAclList.contains(ozoneAcl));
}

@Test
public void testHeadObject() throws IOException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
ReplicationConfig replicationConfig = fromTypeAndFactor(RATIS,
HddsProtos.ReplicationFactor.THREE);

String value = "sample value";
store.createVolume(volumeName);
OzoneVolume volume = store.getVolume(volumeName);
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);


String keyName = UUID.randomUUID().toString();

OzoneOutputStream out = bucket.createKey(keyName,
value.getBytes(UTF_8).length, replicationConfig, new HashMap<>());
out.write(value.getBytes(UTF_8));
out.close();

OzoneKey key = bucket.headObject(keyName);
Assert.assertEquals(volumeName, key.getVolumeName());
Assert.assertEquals(bucketName, key.getBucketName());
Assert.assertEquals(keyName, key.getName());
Assert.assertEquals(replicationConfig.getReplicationType(),
key.getReplicationConfig().getReplicationType());
Assert.assertEquals(replicationConfig.getRequiredNodes(),
key.getReplicationConfig().getRequiredNodes());
Assert.assertEquals(value.getBytes(UTF_8).length, key.getDataSize());

try {
bucket.headObject(UUID.randomUUID().toString());
} catch (OMException ex) {
Assert.assertEquals(ResultCodes.KEY_NOT_FOUND, ex.getResult());
}

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -769,6 +769,9 @@ message KeyArgs {

// This will be set when user performs delete directory recursively.
optional bool recursive = 17;

// When it is a head operation which is to check whether key exist
optional bool headOp = 18;
}

message KeyLocation {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -676,21 +676,29 @@ public OmKeyInfo lookupKey(OmKeyArgs args, String clientAddress)
throw new OMException("Key:" + keyName + " not found", KEY_NOT_FOUND);
}


if (args.getLatestVersionLocation()) {
slimLocationVersion(value);
}

// add block token for read.
addBlockToken4Read(value);
// If operation is head, do not perform any additional steps based on flags.
// As head operation does not need any of those details.
if (!args.isHeadOp()) {

// add block token for read.
addBlockToken4Read(value);

// Refresh container pipeline info from SCM
// based on OmKeyArgs.refreshPipeline flag
// value won't be null as the check is done inside try/catch block.
refresh(value);
// Refresh container pipeline info from SCM
// based on OmKeyArgs.refreshPipeline flag
// value won't be null as the check is done inside try/catch block.
refresh(value);

if (args.getSortDatanodes()) {
sortDatanodes(clientAddress, value);
}

if (args.getSortDatanodes()) {
sortDatanodes(clientAddress, value);
}

return value;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -386,12 +386,14 @@ private LookupKeyResponse lookupKey(LookupKeyRequest request,
.setVolumeName(keyArgs.getVolumeName())
.setBucketName(keyArgs.getBucketName())
.setKeyName(keyArgs.getKeyName())
.setRefreshPipeline(true)
.setSortDatanodesInPipeline(keyArgs.getSortDatanodes())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is "sort datanodes" flag no longer set?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It was a mistake, thanks for catching

.setLatestVersionLocation(keyArgs.getLatestVersionLocation())
.setSortDatanodesInPipeline(keyArgs.getSortDatanodes())
.setHeadOp(keyArgs.getHeadOp())
.build();
OmKeyInfo keyInfo = impl.lookupKey(omKeyArgs);
resp.setKeyInfo(keyInfo.getProtobuf(false, clientVersion));

resp.setKeyInfo(keyInfo.getProtobuf(keyArgs.getHeadOp(), clientVersion));


return resp.build();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import org.apache.hadoop.hdds.conf.StorageUnit;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.OzoneKeyDetails;
import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
Expand Down Expand Up @@ -337,7 +338,7 @@ public Response get(
}

private void addLastModifiedDate(
ResponseBuilder responseBuilder, OzoneKeyDetails key) {
ResponseBuilder responseBuilder, OzoneKey key) {

ZonedDateTime lastModificationTime = key.getModificationTime()
.atZone(ZoneId.of(OzoneConsts.OZONE_TIME_ZONE));
Expand All @@ -358,10 +359,10 @@ public Response head(
@PathParam("bucket") String bucketName,
@PathParam("path") String keyPath) throws IOException, OS3Exception {

OzoneKeyDetails key;
OzoneKey key;

try {
key = getBucket(bucketName).getKey(keyPath);
key = getBucket(bucketName).headObject(keyPath);
// TODO: return the specified range bytes of this object.
} catch (OMException ex) {
if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.util.stream.Collectors;

import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
Expand Down Expand Up @@ -131,6 +132,25 @@ public OzoneKeyDetails getKey(String key) throws IOException {
}
}

@Override
public OzoneKey headObject(String key) throws IOException {
if (keyDetails.containsKey(key)) {
OzoneKeyDetails ozoneKeyDetails = keyDetails.get(key);
return new OzoneKey(ozoneKeyDetails.getVolumeName(),
ozoneKeyDetails.getBucketName(),
ozoneKeyDetails.getName(),
ozoneKeyDetails.getDataSize(),
ozoneKeyDetails.getCreationTime().toEpochMilli(),
ozoneKeyDetails.getModificationTime().toEpochMilli(),
ReplicationConfig.fromTypeAndFactor(
ozoneKeyDetails.getReplicationType(),
ReplicationFactor.valueOf(ozoneKeyDetails.getReplicationFactor())
));
} else {
throw new OMException(ResultCodes.KEY_NOT_FOUND);
}
}

@Override
public Iterator<? extends OzoneKey> listKeys(String keyPrefix) {
Map<String, OzoneKey> sortedKey = new TreeMap<String, OzoneKey>(keyDetails);
Expand Down