Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
20 commits
Select commit Hold shift + click to select a range
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 @@ -237,6 +237,8 @@ public enum Units { TB, GB, MB, KB, B }

public static final int INVALID_PORT = -1;

public static final long OBJECT_ID_DEFAULT = 0L;


/**
* Default SCM Datanode ID file name.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public long getUpdateID() {
}

/**
* Set the Object ID. If this value is already set then this function throws.
* Set the Object ID.
* There is a reason why we cannot use the final here. The object
* ({@link OmVolumeArgs}/ {@link OmBucketInfo}/ {@link OmKeyInfo}) is
* deserialized from the protobuf in many places in code. We need to set
Expand All @@ -61,10 +61,6 @@ public long getUpdateID() {
* @param obId - long
*/
public void setObjectID(long obId) {
if (this.objectID != 0) {
throw new UnsupportedOperationException("Attempt to modify object ID " +
"which is not zero. Current Object ID is " + this.objectID);
}
this.objectID = obId;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1706,9 +1706,8 @@ message DeleteSnapshotRequest {

message SnapshotMoveDeletedKeysRequest {
optional SnapshotInfo fromSnapshot = 1;
optional SnapshotInfo nextSnapshot = 2;
repeated SnapshotMoveKeyInfos activeDBKeys = 3;
repeated SnapshotMoveKeyInfos nextDBKeys = 4;
repeated SnapshotMoveKeyInfos nextDBKeys = 2;
repeated SnapshotMoveKeyInfos reclaimKeys = 3;
}

message SnapshotMoveKeyInfos {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@

import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto.READ;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto.WRITE;
import static org.apache.hadoop.ozone.OzoneConsts.OBJECT_ID_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes
.BUCKET_NOT_FOUND;
Expand Down Expand Up @@ -837,6 +838,7 @@ protected OmKeyInfo wrapUncommittedBlocksAsPseudoKey(
LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
uncommitted, omKeyInfo.getKeyName());
OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
pseudoKeyInfo.setObjectID(OBJECT_ID_DEFAULT);
// TODO dataSize of pseudoKey is not real here
List<OmKeyLocationInfoGroup> uncommittedGroups = new ArrayList<>();
// version not matters in the current logic of keyDeletingService,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,15 +72,19 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
OmResponseUtil.getOMResponseBuilder(getOmRequest());
try {
OmSnapshot omFromSnapshot = (OmSnapshot) omSnapshotManager
.checkForSnapshot(fromSnapshot.getVolumeName(),
fromSnapshot.getBucketName(),
getSnapshotPrefix(fromSnapshot.getName()));

nextSnapshot = getNextActiveSnapshot(fromSnapshot,
snapshotChainManager, omSnapshotManager);

// Get next non-deleted snapshot.

List<SnapshotMoveKeyInfos> activeDBKeysList =
moveDeletedKeysRequest.getActiveDBKeysList();
List<SnapshotMoveKeyInfos> nextDBKeysList =
moveDeletedKeysRequest.getNextDBKeysList();
List<SnapshotMoveKeyInfos> reclaimKeysList =
moveDeletedKeysRequest.getReclaimKeysList();

OmSnapshot omNextSnapshot = null;

Expand All @@ -92,7 +96,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
}

omClientResponse = new OMSnapshotMoveDeletedKeysResponse(
omResponse.build(), omNextSnapshot, activeDBKeysList, nextDBKeysList);
omResponse.build(), omFromSnapshot, omNextSnapshot,
nextDBKeysList, reclaimKeysList);

} catch (IOException ex) {
omClientResponse = new OMSnapshotMoveDeletedKeysResponse(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hadoop.ozone.om.response.snapshot;

import org.apache.hadoop.hdds.utils.db.BatchOperation;
import org.apache.hadoop.hdds.utils.db.DBStore;
import org.apache.hadoop.ozone.om.OMMetadataManager;
import org.apache.hadoop.ozone.om.OmSnapshot;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
Expand All @@ -41,17 +42,20 @@
@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {

private OmSnapshot fromSnapshot;
private OmSnapshot nextSnapshot;
private List<SnapshotMoveKeyInfos> activeDBKeysList;
private List<SnapshotMoveKeyInfos> nextDBKeysList;
private List<SnapshotMoveKeyInfos> reclaimKeyList;

public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
List<SnapshotMoveKeyInfos> nextDBKeysList) {
@Nonnull OmSnapshot omFromSnapshot, OmSnapshot omNextSnapshot,
List<SnapshotMoveKeyInfos> nextDBKeysList,
List<SnapshotMoveKeyInfos> reclaimKeysList) {
super(omResponse);
this.fromSnapshot = omFromSnapshot;
this.nextSnapshot = omNextSnapshot;
this.activeDBKeysList = activeDBKeysList;
this.nextDBKeysList = nextDBKeysList;
this.reclaimKeyList = reclaimKeysList;
}

/**
Expand All @@ -67,34 +71,40 @@ public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
protected void addToDBBatch(OMMetadataManager omMetadataManager,
BatchOperation batchOperation) throws IOException {

for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
RepeatedOmKeyInfo activeDBOmKeyInfo =
createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());

if (activeDBOmKeyInfo == null) {
continue;
if (nextSnapshot != null) {
DBStore nextSnapshotStore = nextSnapshot.getMetadataManager().getStore();
// Init Batch Operation for snapshot db.
try (BatchOperation writeBatch = nextSnapshotStore.initBatchOperation()) {
processKeys(writeBatch, nextSnapshot.getMetadataManager(),
nextDBKeysList);
nextSnapshotStore.commitBatchOperation(writeBatch);
}

omMetadataManager.getDeletedTable().putWithBatch(
batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
} else {
// Handle case when there is no next Snapshot
processKeys(batchOperation, omMetadataManager, nextDBKeysList);
}

for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
RepeatedOmKeyInfo nextDBOmKeyInfo =
createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
// Update From Snapshot Deleted Table.
DBStore fromSnapshotStore = fromSnapshot.getMetadataManager().getStore();
try (BatchOperation fromSnapshotBatchOp =
fromSnapshotStore.initBatchOperation()) {
processKeys(fromSnapshotBatchOp, fromSnapshot.getMetadataManager(),
reclaimKeyList);
fromSnapshotStore.commitBatchOperation(fromSnapshotBatchOp);
}
}

if (nextDBOmKeyInfo == null) {
private void processKeys(BatchOperation batchOp,
OMMetadataManager metadataManager,
List<SnapshotMoveKeyInfos> keyList) throws IOException {
for (SnapshotMoveKeyInfos dBKey : keyList) {
RepeatedOmKeyInfo omKeyInfos =
createRepeatedOmKeyInfo(dBKey.getKeyInfosList());
if (omKeyInfos == null) {
continue;
}

if (nextSnapshot != null) {
nextSnapshot.getMetadataManager()
.getDeletedTable().putWithBatch(batchOperation,
nextDBKey.getKey(), nextDBOmKeyInfo);
} else {
omMetadataManager.getDeletedTable().putWithBatch(
batchOperation, nextDBKey.getKey(), nextDBOmKeyInfo);
}
metadataManager.getDeletedTable().putWithBatch(batchOp,
dBKey.getKey(), omKeyInfos);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;

import static org.apache.hadoop.ozone.OzoneConsts.OBJECT_ID_DEFAULT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
Expand Down Expand Up @@ -157,8 +158,8 @@ public BackgroundTaskResult call() throws Exception {
Table<String, OmKeyInfo> previousKeyTable = null;
OmSnapshot omPreviousSnapshot = null;

// Handle case when the deleted snapshot is the first snapshot.
// Move deleted keys to activeDB's deletedKeyTable
// Split RepeatedOmKeyInfo and update current snapshot deletedKeyTable
// and next snapshot deletedKeyTable.
if (previousSnapshot != null) {
omPreviousSnapshot = (OmSnapshot) omSnapshotManager
.checkForSnapshot(previousSnapshot.getVolumeName(),
Expand All @@ -169,10 +170,9 @@ public BackgroundTaskResult call() throws Exception {
.getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
}

// Move key to either next non deleted snapshot's snapshotDeletedTable
// or move to active object store deleted table

List<SnapshotMoveKeyInfos> toActiveDBList = new ArrayList<>();
// Move key to either next non deleted snapshot's deletedTable
// or keep it in current snapshot deleted table.
List<SnapshotMoveKeyInfos> toReclaimList = new ArrayList<>();
List<SnapshotMoveKeyInfos> toNextDBList = new ArrayList<>();

try (TableIterator<String, ? extends Table.KeyValue<String,
Expand All @@ -188,30 +188,35 @@ public BackgroundTaskResult call() throws Exception {
String deletedKey = deletedKeyValue.getKey();

// Exit if it is out of the bucket scope.
if (!deletedKey.contains(snapshotBucketKey)) {
if (!deletedKey.startsWith(snapshotBucketKey)) {
break;
}

RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();

SnapshotMoveKeyInfos.Builder toActiveDb = SnapshotMoveKeyInfos
SnapshotMoveKeyInfos.Builder toReclaim = SnapshotMoveKeyInfos
.newBuilder()
.setKey(deletedKey);
SnapshotMoveKeyInfos.Builder toNextDb = SnapshotMoveKeyInfos
.newBuilder()
.setKey(deletedKey);

for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
splitRepeatedOmKeyInfo(toReclaim, toNextDb,
keyInfo, previousKeyTable);
}

toActiveDBList.add(toActiveDb.build());
// If all the KeyInfos are reclaimable in RepeatedOmKeyInfo
// then no need to update current snapshot deletedKeyTable.
if (!(toReclaim.getKeyInfosCount() ==
repeatedOmKeyInfo.getOmKeyInfoList().size())) {
toReclaimList.add(toReclaim.build());
}
toNextDBList.add(toNextDb.build());

}
// Submit Move request to OM.
submitSnapshotMoveDeletedKeys(snapInfo, toActiveDBList,
submitSnapshotMoveDeletedKeys(snapInfo, toReclaimList,
toNextDBList);
snapshotLimit--;
successRunCount.incrementAndGet();
Expand All @@ -226,30 +231,30 @@ public BackgroundTaskResult call() throws Exception {
return BackgroundTaskResult.EmptyTaskResult.newResult();
}

private void splitRepeatedOmKeyInfo(SnapshotMoveKeyInfos.Builder toActiveDb,
private void splitRepeatedOmKeyInfo(SnapshotMoveKeyInfos.Builder toReclaim,
SnapshotMoveKeyInfos.Builder toNextDb, OmKeyInfo keyInfo,
Table<String, OmKeyInfo> previousKeyTable) throws IOException {
if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
if (checkKeyReclaimable(previousKeyTable, keyInfo)) {
// Move to next non deleted snapshot's deleted table
toNextDb.addKeyInfos(keyInfo.getProtobuf(
ClientVersion.CURRENT_VERSION));
} else {
// Move to active DB Deleted Table.
toActiveDb.addKeyInfos(keyInfo
// Update in current db's deletedKeyTable
toReclaim.addKeyInfos(keyInfo
.getProtobuf(ClientVersion.CURRENT_VERSION));
}
}

private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
List<SnapshotMoveKeyInfos> toActiveDBList,
List<SnapshotMoveKeyInfos> toReclaimList,
List<SnapshotMoveKeyInfos> toNextDBList) {

SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
SnapshotMoveDeletedKeysRequest.newBuilder()
.setFromSnapshot(snapInfo.getProtobuf());

SnapshotMoveDeletedKeysRequest moveDeletedKeys =
moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
moveDeletedKeysBuilder.addAllReclaimKeys(toReclaimList)
.addAllNextDBKeys(toNextDBList).build();


Expand All @@ -262,14 +267,20 @@ private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
submitRequest(omRequest);
}

private boolean checkKeyExistInPreviousTable(
private boolean checkKeyReclaimable(
Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
throws IOException {

// Handle case when the deleted snapshot is the first snapshot.
if (previousKeyTable == null) {
return false;
}

// These are uncommitted blocks wrapped into a pseudo KeyInfo
if (deletedKeyInfo.getObjectID() == OBJECT_ID_DEFAULT) {
return false;
}

//TODO: [SNAPSHOT] Handle Renamed Keys
String dbKey = ozoneManager.getMetadataManager()
.getOzoneKey(deletedKeyInfo.getVolumeName(),
Expand Down Expand Up @@ -366,3 +377,4 @@ public void setSuccessRunCount(long num) {
successRunCount.getAndSet(num);
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -289,8 +289,8 @@ private void createSnapshot(String volName, String bucketName,

GenericTestUtils.waitFor(() -> {
try {
return omMetadataManager.getSnapshotInfoTable()
.getEstimatedKeyCount() >= count;
return omMetadataManager.countRowsInTable(
omMetadataManager.getSnapshotInfoTable()) >= count;
} catch (IOException e) {
e.printStackTrace();
}
Expand Down