diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 4d7c7118977a..6e7892a7e466 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -1807,7 +1807,7 @@ private OmKeyInfo getS3PartKeyInfo( .setSortDatanodesInPipeline(topologyAwareReadEnabled) .setLatestVersionLocation(getLatestVersionLocation) .setForceUpdateContainerCacheFromSCM(false) - .setPartNumber(partNumber) + .setMultipartUploadPartNumber(partNumber) .build(); KeyInfoWithVolumeContext keyInfoWithS3Context = ozoneManagerClient.getKeyInfo(keyArgs, true); diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java index b935190a5b27..106ef6a06ab8 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java @@ -54,7 +54,6 @@ public final class OmKeyArgs implements Auditable { private final boolean headOp; private final boolean forceUpdateContainerCacheFromSCM; private final Map tags; - private final Integer partNumber; // expectedDataGeneration, when used in key creation indicates that a // key with the same keyName should exist with the given generation. // For a key commit to succeed, the original key should still be present with the @@ -83,7 +82,6 @@ private OmKeyArgs(Builder b) { this.ownerName = b.ownerName; this.tags = b.tags; this.expectedDataGeneration = b.expectedDataGeneration; - this.partNumber = b.partNumber; } public boolean getIsMultipartKey() { @@ -170,10 +168,6 @@ public Long getExpectedDataGeneration() { return expectedDataGeneration; } - public Integer getPartNumber() { - return partNumber; - } - @Override public Map toAuditMap() { Map auditMap = new LinkedHashMap<>(); @@ -220,10 +214,6 @@ public OmKeyArgs.Builder toBuilder() { builder.setExpectedDataGeneration(expectedDataGeneration); } - if (partNumber != null) { - builder.setPartNumber(partNumber); - } - return builder; } @@ -240,8 +230,8 @@ public KeyArgs toProtobuf() { .setForceUpdateContainerCacheFromSCM( isForceUpdateContainerCacheFromSCM() ); - if (partNumber != null) { - builder.setPartNumber(partNumber); + if (multipartUploadPartNumber != 0) { + builder.setMultipartNumber(multipartUploadPartNumber); } if (expectedDataGeneration != null) { builder.setExpectedDataGeneration(expectedDataGeneration); @@ -272,7 +262,6 @@ public static class Builder { private boolean forceUpdateContainerCacheFromSCM; private final Map tags = new HashMap<>(); private Long expectedDataGeneration = null; - private Integer partNumber; public Builder setVolumeName(String volume) { this.volumeName = volume; @@ -387,11 +376,6 @@ public Builder setExpectedDataGeneration(long generation) { return this; } - public Builder setPartNumber(int partNumber) { - this.partNumber = partNumber; - return this; - } - public OmKeyArgs build() { return new OmKeyArgs(this); } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java index cc7d9a42ca54..5c480860d2b2 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java @@ -17,6 +17,14 @@ */ package org.apache.hadoop.ozone.om.helpers; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CopyOnWriteArrayList; + import com.google.common.collect.ImmutableList; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; @@ -25,7 +33,6 @@ import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; -import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.utils.db.Codec; import org.apache.hadoop.hdds.utils.db.DelegatedCodec; import org.apache.hadoop.hdds.utils.db.CopyObject; @@ -38,18 +45,10 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocationList; import org.apache.hadoop.ozone.protocolPB.OMPBHelper; import org.apache.hadoop.util.Time; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.stream.Collectors; - /** * Args for key block. The block instance for the key requested in putKey. * This is returned from OM to client, and client use class to talk to @@ -81,8 +80,6 @@ public static Codec getCodec(boolean ignorePipeline) { private String keyName; private long dataSize; private List keyLocationVersions; - private Map> currentlocationsPartsMap; - private Map currentDataSizePartsMap; private final long creationTime; private long modificationTime; private ReplicationConfig replicationConfig; @@ -191,17 +188,6 @@ public String getOwnerName() { return ownerName; } - public Map> getCurrentlocationsPartsMap() { - if (currentDataSizePartsMap == null) { - refreshCurrentLocationPartsMap(); - } - return currentlocationsPartsMap; - } - - public Map getCurrentDataSizePartsMap() { - return currentDataSizePartsMap; - } - /** * Returns the generation of the object. Note this is currently the same as updateID for a key. * @return long @@ -224,25 +210,6 @@ public void setKeyLocationVersions( this.keyLocationVersions = keyLocationVersions; } - private void refreshCurrentLocationPartsMap() { - if (this.keyLocationVersions.size() > 0) { - this.currentlocationsPartsMap = this.keyLocationVersions.get(keyLocationVersions.size() - 1) - .getLocationList() - .stream() - .filter(it -> it.getPartNumber() != 0) - .collect(Collectors.groupingBy(OmKeyLocationInfo::getPartNumber)); - this.currentDataSizePartsMap = this.currentlocationsPartsMap.entrySet().stream() - .map(it -> Pair.of( - it.getKey(), - it.getValue().stream(). - mapToLong(BlockLocationInfo::getLength) - .sum() - ) - ) - .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - } - } - public void setFile(boolean file) { isFile = file; } @@ -384,7 +351,6 @@ public synchronized void appendNewBlocks( if (updateTime) { setModificationTime(Time.now()); } - refreshCurrentLocationPartsMap(); } /** diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto index 0b2d74d6b68a..92c2b6b4cc5a 100644 --- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto +++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto @@ -1079,8 +1079,6 @@ message KeyArgs { // This allows a key to be created an committed atomically if the original has not // been modified. optional uint64 expectedDataGeneration = 23; - - optional int32 partNumber = 24; } message KeyLocation { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java index bac6851c171e..c1b98e81f720 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java @@ -509,25 +509,31 @@ private OmKeyInfo readKeyInfo(OmKeyArgs args, BucketLayout bucketLayout) if (args.getLatestVersionLocation()) { slimLocationVersion(value); } - Integer partNumberParam = args.getPartNumber(); - if (partNumberParam != null && partNumberParam > 0) { + int partNumberParam = args.getMultipartUploadPartNumber(); + if (partNumberParam > 0) { OmKeyLocationInfoGroup latestLocationVersion = value.getLatestVersionLocations(); if (latestLocationVersion != null && latestLocationVersion.isMultipartKey()) { + List currentLocations = + value.getKeyLocationVersions().get(value.getKeyLocationVersions().size() - 1) + .getLocationList() + .stream() + .filter(it -> it.getPartNumber() == partNumberParam) + .collect(Collectors.toList()); value.setKeyLocationVersions( Collections.singletonList( new OmKeyLocationInfoGroup( latestLocationVersion.getVersion(), - value.getCurrentlocationsPartsMap() - .getOrDefault(partNumberParam, Collections.emptyList()), + currentLocations, true ) ) ); - value.setDataSize( - value.getCurrentDataSizePartsMap() - .getOrDefault(partNumberParam, 0L) - ); + + long dataLength = currentLocations.stream() + .mapToLong(BlockLocationInfo::getLength) + .sum(); + value.setDataSize(dataLength); } } return value; @@ -822,7 +828,7 @@ public boolean isSstFilteringSvcEnabled() { TimeUnit.MILLISECONDS); return serviceInterval != DISABLE_VALUE; } - + @Override public OmMultipartUploadList listMultipartUploads(String volumeName, String bucketName, String prefix) throws OMException { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java index 87de168b30ef..3c99e26eac25 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java @@ -659,7 +659,7 @@ private GetKeyInfoResponse getKeyInfo(GetKeyInfoRequest request, .setHeadOp(keyArgs.getHeadOp()) .setForceUpdateContainerCacheFromSCM( keyArgs.getForceUpdateContainerCacheFromSCM()) - .setPartNumber(keyArgs.getPartNumber()) + .setMultipartUploadPartNumber(keyArgs.getMultipartNumber()) .build(); KeyInfoWithVolumeContext keyInfo = impl.getKeyInfo(omKeyArgs, request.getAssumeS3Context());