diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneSnapshot.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneSnapshot.java index f1ea1c67ab9..8e041d95ecc 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneSnapshot.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneSnapshot.java @@ -35,6 +35,9 @@ public class OzoneSnapshot { private final String snapshotPath; // snapshot mask private final String checkpointDir; private final long referencedSize; + private final long referencedReplicatedSize; + private final long exclusiveSize; + private final long exclusiveReplicatedSize; /** * Constructs OzoneSnapshot from SnapshotInfo. @@ -48,6 +51,9 @@ public class OzoneSnapshot { * @param snapshotPath Path of the snapshot. * @param checkpointDir Snapshot checkpoint directory. * @param referencedSize Snapshot referenced size. + * @param referencedReplicatedSize Snapshot referenced size after replication. + * @param exclusiveSize Snapshot exclusive size. + * @param exclusiveReplicatedSize Snapshot exclusive size after replication. */ @SuppressWarnings("parameternumber") public OzoneSnapshot(String volumeName, @@ -58,7 +64,10 @@ public OzoneSnapshot(String volumeName, UUID snapshotId, String snapshotPath, String checkpointDir, - long referencedSize) { + long referencedSize, + long referencedReplicatedSize, + long exclusiveSize, + long exclusiveReplicatedSize) { this.volumeName = volumeName; this.bucketName = bucketName; this.name = name; @@ -68,6 +77,9 @@ public OzoneSnapshot(String volumeName, this.snapshotPath = snapshotPath; this.checkpointDir = checkpointDir; this.referencedSize = referencedSize; + this.referencedReplicatedSize = referencedReplicatedSize; + this.exclusiveSize = exclusiveSize; + this.exclusiveReplicatedSize = exclusiveReplicatedSize; } /** @@ -149,6 +161,27 @@ public long getReferencedSize() { return referencedSize; } + /** + * @return Reference size after replication/EC of the snapshot + */ + public long getReferencedReplicatedSize() { + return referencedReplicatedSize; + } + + /** + * @return Exclusive size of the snapshot. + */ + public long getExclusiveSize() { + return exclusiveSize; + } + + /** + * @return Exclusive size after replication/EC of the snapshot. + */ + public long getExclusiveReplicatedSize() { + return exclusiveReplicatedSize; + } + public static OzoneSnapshot fromSnapshotInfo(SnapshotInfo snapshotInfo) { return new OzoneSnapshot( snapshotInfo.getVolumeName(), @@ -159,6 +192,10 @@ public static OzoneSnapshot fromSnapshotInfo(SnapshotInfo snapshotInfo) { snapshotInfo.getSnapshotId(), snapshotInfo.getSnapshotPath(), snapshotInfo.getCheckpointDir(), - snapshotInfo.getReferencedSize()); + snapshotInfo.getReferencedSize(), + snapshotInfo.getReferencedReplicatedSize(), + snapshotInfo.getExclusiveSize(), + snapshotInfo.getExclusiveReplicatedSize() + ); } } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java index 09e04409885..7597c039e0b 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java @@ -20,6 +20,8 @@ import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS; @@ -29,8 +31,11 @@ */ public final class QuotaUtil { + private static final Logger LOG = + LoggerFactory.getLogger(QuotaUtil.class); + private QuotaUtil() { - }; + } /** * From the used space and replicationConfig, calculate the expected @@ -42,10 +47,10 @@ private QuotaUtil() { public static long getReplicatedSize( long dataSize, ReplicationConfig repConfig) { if (repConfig.getReplicationType() == RATIS) { - return dataSize * ((RatisReplicationConfig)repConfig) + return dataSize * ((RatisReplicationConfig) repConfig) .getReplicationFactor().getNumber(); } else if (repConfig.getReplicationType() == EC) { - ECReplicationConfig rc = (ECReplicationConfig)repConfig; + ECReplicationConfig rc = (ECReplicationConfig) repConfig; int dataStripeSize = rc.getData() * rc.getEcChunkSize(); long fullStripes = dataSize / dataStripeSize; long partialFirstChunk = @@ -55,8 +60,36 @@ public static long getReplicatedSize( + partialFirstChunk * rc.getParity(); return dataSize + replicationOverhead; } else { + LOG.warn("Unknown replication type {}. Returning original data size.", + repConfig.getReplicationType()); return dataSize; } } + /** + * Get an estimated data size (before replication) from the replicated size. + * An (inaccurate) reverse of getReplicatedSize(). + * @param replicatedSize size after replication. + * @param repConfig The replicationConfig used to store the data. + * @return Data size before replication. + */ + public static long getDataSize(long replicatedSize, + ReplicationConfig repConfig) { + if (repConfig.getReplicationType() == RATIS) { + final int ratisReplicationFactor = ((RatisReplicationConfig) repConfig) + .getReplicationFactor().getNumber(); + // May not be divisible. But it's fine to ignore remainder in our use case + return replicatedSize / ratisReplicationFactor; + } else if (repConfig.getReplicationType() == EC) { + ECReplicationConfig rc = (ECReplicationConfig) repConfig; + // Because for EC we don't know if keys have partial chunks or not, + // we assume no partial chunks so as to return an estimate. + return replicatedSize * rc.getData() / rc.getRequiredNodes(); + } else { + LOG.warn("Unknown replication type {}. Returning replicatedSize.", + repConfig.getReplicationType()); + return replicatedSize; + } + } + } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/SnapshotInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/SnapshotInfo.java index 2a3fbcff78e..0ef8a2d3e9b 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/SnapshotInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/SnapshotInfo.java @@ -128,6 +128,9 @@ public static SnapshotStatus valueOf(SnapshotStatusProto status) { private boolean deepClean; private boolean sstFiltered; private long referencedSize; + private long referencedReplicatedSize; + private long exclusiveSize; + private long exclusiveReplicatedSize; /** * Private constructor, constructed via builder. @@ -145,6 +148,10 @@ public static SnapshotStatus valueOf(SnapshotStatusProto status) { * @param checkpointDir - Snapshot checkpoint directory. * @param dbTxSequenceNumber - RDB latest transaction sequence number. * @param deepCleaned - To be deep cleaned status for snapshot. + * @param referencedSize - Snapshot referenced size. + * @param referencedReplicatedSize - Snapshot referenced size w/ replication. + * @param exclusiveSize - Snapshot exclusive size. + * @param exclusiveReplicatedSize - Snapshot exclusive size w/ replication. */ @SuppressWarnings("checkstyle:ParameterNumber") private SnapshotInfo(UUID snapshotId, @@ -161,7 +168,10 @@ private SnapshotInfo(UUID snapshotId, long dbTxSequenceNumber, boolean deepCleaned, boolean sstFiltered, - long referencedSize) { + long referencedSize, + long referencedReplicatedSize, + long exclusiveSize, + long exclusiveReplicatedSize) { this.snapshotId = snapshotId; this.name = name; this.volumeName = volumeName; @@ -177,6 +187,9 @@ private SnapshotInfo(UUID snapshotId, this.deepClean = deepCleaned; this.sstFiltered = sstFiltered; this.referencedSize = referencedSize; + this.referencedReplicatedSize = referencedReplicatedSize; + this.exclusiveSize = exclusiveSize; + this.exclusiveReplicatedSize = exclusiveReplicatedSize; } public void setName(String name) { @@ -295,7 +308,10 @@ public SnapshotInfo.Builder toBuilder() { .setCheckpointDir(checkpointDir) .setDeepClean(deepClean) .setSstFiltered(sstFiltered) - .setReferencedSize(referencedSize); + .setReferencedSize(referencedSize) + .setReferencedReplicatedSize(referencedReplicatedSize) + .setExclusiveSize(exclusiveSize) + .setExclusiveReplicatedSize(exclusiveReplicatedSize); } /** @@ -317,6 +333,9 @@ public static class Builder { private boolean deepClean; private boolean sstFiltered; private long referencedSize; + private long referencedReplicatedSize; + private long exclusiveSize; + private long exclusiveReplicatedSize; public Builder() { // default values @@ -398,6 +417,21 @@ public Builder setReferencedSize(long referencedSize) { return this; } + public Builder setReferencedReplicatedSize(long referencedReplicatedSize) { + this.referencedReplicatedSize = referencedReplicatedSize; + return this; + } + + public Builder setExclusiveSize(long exclusiveSize) { + this.exclusiveSize = exclusiveSize; + return this; + } + + public Builder setExclusiveReplicatedSize(long exclusiveReplicatedSize) { + this.exclusiveReplicatedSize = exclusiveReplicatedSize; + return this; + } + public SnapshotInfo build() { Preconditions.checkNotNull(name); return new SnapshotInfo( @@ -415,7 +449,10 @@ public SnapshotInfo build() { dbTxSequenceNumber, deepClean, sstFiltered, - referencedSize + referencedSize, + referencedReplicatedSize, + exclusiveSize, + exclusiveReplicatedSize ); } } @@ -426,15 +463,18 @@ public SnapshotInfo build() { public OzoneManagerProtocolProtos.SnapshotInfo getProtobuf() { OzoneManagerProtocolProtos.SnapshotInfo.Builder sib = OzoneManagerProtocolProtos.SnapshotInfo.newBuilder() - .setSnapshotID(toProtobuf(snapshotId)) - .setName(name) - .setVolumeName(volumeName) - .setBucketName(bucketName) - .setSnapshotStatus(snapshotStatus.toProto()) - .setCreationTime(creationTime) - .setDeletionTime(deletionTime) - .setSstFiltered(sstFiltered) - .setReferencedSize(referencedSize); + .setSnapshotID(toProtobuf(snapshotId)) + .setName(name) + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setSnapshotStatus(snapshotStatus.toProto()) + .setCreationTime(creationTime) + .setDeletionTime(deletionTime) + .setSstFiltered(sstFiltered) + .setReferencedSize(referencedSize) + .setReferencedReplicatedSize(referencedReplicatedSize) + .setExclusiveSize(exclusiveSize) + .setExclusiveReplicatedSize(exclusiveReplicatedSize); if (pathPreviousSnapshotId != null) { sib.setPathPreviousSnapshotID(toProtobuf(pathPreviousSnapshotId)); @@ -488,7 +528,23 @@ public static SnapshotInfo getFromProtobuf( } if (snapshotInfoProto.hasReferencedSize()) { - osib.setReferencedSize(snapshotInfoProto.getReferencedSize()); + osib.setReferencedSize( + snapshotInfoProto.getReferencedSize()); + } + + if (snapshotInfoProto.hasReferencedReplicatedSize()) { + osib.setReferencedReplicatedSize( + snapshotInfoProto.getReferencedReplicatedSize()); + } + + if (snapshotInfoProto.hasExclusiveSize()) { + osib.setExclusiveSize( + snapshotInfoProto.getExclusiveSize()); + } + + if (snapshotInfoProto.hasExclusiveReplicatedSize()) { + osib.setExclusiveReplicatedSize( + snapshotInfoProto.getExclusiveReplicatedSize()); } osib.setSnapshotPath(snapshotInfoProto.getSnapshotPath()) @@ -551,6 +607,30 @@ public long getReferencedSize() { return referencedSize; } + public void setReferencedReplicatedSize(long referencedReplicatedSize) { + this.referencedReplicatedSize = referencedReplicatedSize; + } + + public long getReferencedReplicatedSize() { + return referencedReplicatedSize; + } + + public void setExclusiveSize(long exclusiveSize) { + this.exclusiveSize = exclusiveSize; + } + + public long getExclusiveSize() { + return exclusiveSize; + } + + public void setExclusiveReplicatedSize(long exclusiveReplicatedSize) { + this.exclusiveReplicatedSize = exclusiveReplicatedSize; + } + + public long getExclusiveReplicatedSize() { + return exclusiveReplicatedSize; + } + /** * Generate default name of snapshot, (used if user doesn't provide one). */ @@ -612,9 +692,12 @@ public boolean equals(Object o) { globalPreviousSnapshotId, that.globalPreviousSnapshotId) && snapshotPath.equals(that.snapshotPath) && checkpointDir.equals(that.checkpointDir) && - Objects.equals(deepClean, that.deepClean) && - Objects.equals(sstFiltered, that.sstFiltered) && - Objects.equals(referencedSize, that.referencedSize); + deepClean == that.deepClean && + sstFiltered == that.sstFiltered && + referencedSize == that.referencedSize && + referencedReplicatedSize == that.referencedReplicatedSize && + exclusiveSize == that.exclusiveSize && + exclusiveReplicatedSize == that.exclusiveReplicatedSize; } @Override @@ -623,6 +706,8 @@ public int hashCode() { snapshotStatus, creationTime, deletionTime, pathPreviousSnapshotId, globalPreviousSnapshotId, snapshotPath, checkpointDir, - deepClean, sstFiltered, referencedSize); + deepClean, sstFiltered, + referencedSize, referencedReplicatedSize, + exclusiveSize, exclusiveReplicatedSize); } } diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmSnapshotInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmSnapshotInfo.java index d85fa1b14f6..bbc416150ca 100644 --- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmSnapshotInfo.java +++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmSnapshotInfo.java @@ -68,7 +68,10 @@ private SnapshotInfo createSnapshotInfo() { .setDbTxSequenceNumber(DB_TX_SEQUENCE_NUMBER) .setDeepClean(true) .setSstFiltered(false) - .setReferencedSize(10000L) + .setReferencedSize(2000L) + .setReferencedReplicatedSize(6000L) + .setExclusiveSize(1000L) + .setExclusiveReplicatedSize(3000L) .build(); } @@ -88,7 +91,10 @@ private OzoneManagerProtocolProtos.SnapshotInfo createSnapshotInfoProto() { .setDbTxSequenceNumber(DB_TX_SEQUENCE_NUMBER) .setDeepClean(true) .setSstFiltered(false) - .setReferencedSize(10000L) + .setReferencedSize(2000L) + .setReferencedReplicatedSize(6000L) + .setExclusiveSize(1000L) + .setExclusiveReplicatedSize(3000L) .build(); } @@ -126,6 +132,13 @@ public void testSnapshotInfoToProto() { snapshotInfoEntryActual.getSstFiltered()); Assert.assertEquals(snapshotInfoEntryExpected.getReferencedSize(), snapshotInfoEntryActual.getReferencedSize()); + Assert.assertEquals(snapshotInfoEntryExpected.getReferencedReplicatedSize(), + snapshotInfoEntryActual.getReferencedReplicatedSize()); + Assert.assertEquals(snapshotInfoEntryExpected.getExclusiveSize(), + snapshotInfoEntryActual.getExclusiveSize()); + Assert.assertEquals(snapshotInfoEntryExpected.getExclusiveReplicatedSize(), + snapshotInfoEntryActual.getExclusiveReplicatedSize()); + Assert.assertEquals(snapshotInfoEntryExpected, snapshotInfoEntryActual); } @@ -155,6 +168,13 @@ public void testSnapshotInfoProtoToSnapshotInfo() { snapshotInfoActual.isSstFiltered()); Assert.assertEquals(snapshotInfoExpected.getReferencedSize(), snapshotInfoActual.getReferencedSize()); + Assert.assertEquals(snapshotInfoExpected.getReferencedReplicatedSize(), + snapshotInfoActual.getReferencedReplicatedSize()); + Assert.assertEquals(snapshotInfoExpected.getExclusiveSize(), + snapshotInfoActual.getExclusiveSize()); + Assert.assertEquals(snapshotInfoExpected.getExclusiveReplicatedSize(), + snapshotInfoActual.getExclusiveReplicatedSize()); + Assert.assertEquals(snapshotInfoExpected, snapshotInfoActual); } diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto index 026a6937ece..ace70281284 100644 --- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto +++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto @@ -820,7 +820,15 @@ message SnapshotInfo { optional int64 dbTxSequenceNumber = 12; optional bool deepClean = 13; optional bool sstFiltered = 14; + // snapshot reference size before any key replication or EC optional uint64 referencedSize = 15; + // snapshot reference size after replication + optional uint64 referencedReplicatedSize = 16; + // snapshot exclusive size before any key replication or EC + optional uint64 exclusiveSize = 17; + // snapshot exclusive size after replication + optional uint64 exclusiveReplicatedSize = 18; + // note: shared sizes can be calculated from: referenced - exclusive } message SnapshotDiffJobProto { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index 27303dff77b..5cded9a8f60 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -189,7 +189,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager, * |-------------------------------------------------------------------------| * | Column Family | VALUE | * |-------------------------------------------------------------------------| - * | snapshotInfoTable | /volume/bucket/snapshotName -> SnapshotInfo | + * | snapshotInfoTable | /volume/bucket/snapshotName -> SnapshotInfo | * |-------------------------------------------------------------------------| * | snapshotRenamedTable | /volumeName/bucketName/objectID -> One of: | * | | 1. /volumeId/bucketId/parentId/dirName | @@ -728,6 +728,8 @@ protected void initializeOmTables(boolean addCacheMetrics) String.class, OmDBTenantState.class); checkTableStatus(tenantStateTable, TENANT_STATE_TABLE, addCacheMetrics); + // TODO: Consider FULL_CACHE for snapshotInfoTable since exclusiveSize in + // SnapshotInfo can be frequently updated. // path -> snapshotInfo (snapshot info for snapshot) snapshotInfoTable = this.store.getTable(SNAPSHOT_INFO_TABLE, String.class, SnapshotInfo.class); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java index fc0e80dc6e2..ddc8622a55f 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java @@ -31,6 +31,7 @@ import org.apache.hadoop.ozone.om.SnapshotChainManager; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.QuotaUtil; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; import org.apache.hadoop.ozone.om.request.OMClientRequest; @@ -165,7 +166,18 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, // Snapshot referenced size should be bucket's used bytes OmBucketInfo omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName); - snapshotInfo.setReferencedSize(omBucketInfo.getUsedBytes()); + snapshotInfo.setReferencedReplicatedSize(omBucketInfo.getUsedBytes()); + + // Snapshot referenced size in this case is an *estimate* inferred from + // the bucket default replication policy right now. + // This may well not be the actual sum of all key data sizes in this + // bucket because each key can have its own replication policy, + // depending on the client at the time of writing it. + // And we are NOT doing O(n) walk over the keyTable (fileTable) here + // because it is a design goal of CreateSnapshot to be an O(1) operation. + // TODO: Assign actual data size once we have the pre-replicated key size + // counter in OmBucketInfo. + snapshotInfo.setReferencedSize(estimateBucketDataSize(omBucketInfo)); addSnapshotInfoToSnapshotChainAndCache(omMetadataManager, transactionLogIndex); @@ -303,4 +315,16 @@ protected OmBucketInfo getBucketInfo(OMMetadataManager omMetadataManager, return value != null ? value.getCacheValue() : null; } + + /** + * Estimate the sum data sizes of all keys in the bucket by dividing + * bucket used size (w/ replication) by the replication factor of the bucket. + * @param bucketInfo OmBucketInfo + */ + private long estimateBucketDataSize(OmBucketInfo bucketInfo) { + return QuotaUtil.getDataSize( + bucketInfo.getUsedBytes(), + bucketInfo.getDefaultReplicationConfig().getReplicationConfig()); + } + } diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java index d6ea58a48f7..30ef71865b2 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java @@ -1210,7 +1210,8 @@ private static FileStatusAdapter getFileStatusAdapterForBucketSnapshot( } Path path = new Path(pathStr); return new FileStatusAdapter( - ozoneSnapshot.getReferencedSize(), ozoneSnapshot.getReferencedSize(), + ozoneSnapshot.getReferencedSize(), + ozoneSnapshot.getReferencedReplicatedSize(), path, true, (short) 0, 0L, ozoneSnapshot.getCreationTime(), 0L, FsPermission.getDirDefault().toShort(),