Skip to content
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 @@ -393,6 +393,10 @@ public final class OzoneConfigKeys {
"ozone.s3.token.max.lifetime";
public static final String OZONE_S3_AUTHINFO_MAX_LIFETIME_KEY_DEFAULT = "3m";

public static final String OZONE_FS_ITERATE_BATCH_SIZE =
Copy link
Contributor

Choose a reason for hiding this comment

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

Will this property be used for batch rename too?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes, batch Rename will also use this property.

"ozone.fs.iterate.batch-size";
public static final int OZONE_FS_ITERATE_BATCH_SIZE_DEFAULT = 100;

// Ozone Client Retry and Failover configurations
public static final String OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY =
"ozone.client.failover.max.attempts";
Expand Down
8 changes: 8 additions & 0 deletions hadoop-hdds/common/src/main/resources/ozone-default.xml
Original file line number Diff line number Diff line change
Expand Up @@ -1936,6 +1936,14 @@
</description>
</property>

<property>
<name>ozone.fs.iterate.batch-size</name>
<value>100</value>
<tag>OZONE, OZONEFS</tag>
<description>
Iterate batch size of delete when use BasicOzoneFileSystem.
</description>
</property>
<property>
<name>ozone.manager.db.checkpoint.transfer.bandwidthPerSec</name>
<value>0</value>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -382,6 +382,21 @@ public void deleteKey(String key) throws IOException {
proxy.deleteKey(volumeName, name, key);
}

/**
* Deletes the given list of keys from the bucket.
* @param keyList List of the key name to be deleted.
* @throws IOException
*/
public void deleteKeys(List<String> keyList) 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.

When we delete a list of keys, upon failure in the middle, can we return a list of deleted keys and undeleted keys? This may not be an issue when you delete a single key but when batch deleting, it is hard to recover from the failures without that information.

Copy link
Contributor

Choose a reason for hiding this comment

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

Agree with Xiaoyu, we should return a list of deleted key result on failure.

Copy link
Member Author

Choose a reason for hiding this comment

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

I will add unDeletedKeys and deletedKeys in exception when an exception occurs when delete keys.

proxy.deleteKeys(volumeName, name, keyList);
}

/**
* Rename the keyname from fromKeyName to toKeyName.
* @param fromKeyName The original key name.
* @param toKeyName New key name.
* @throws IOException
*/
public void renameKey(String fromKeyName, String toKeyName)
throws IOException {
proxy.renameKey(volumeName, name, fromKeyName, toKeyName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -294,6 +294,17 @@ OzoneInputStream getKey(String volumeName, String bucketName, String keyName)
void deleteKey(String volumeName, String bucketName, String keyName)
throws IOException;

/**
* Deletes keys through the list.
* @param volumeName Name of the Volume
* @param bucketName Name of the Bucket
* @param keyNameList List of the Key
* @throws IOException
*/
void deleteKeys(String volumeName, String bucketName,
List<String> keyNameList)
throws IOException;

/**
* Renames an existing key within a bucket.
* @param volumeName Name of the Volume
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -713,6 +713,24 @@ public void deleteKey(
ozoneManagerClient.deleteKey(keyArgs);
}

@Override
public void deleteKeys(
String volumeName, String bucketName, List<String> keyNameList)
throws IOException {
HddsClientUtils.verifyResourceName(volumeName, bucketName);
Preconditions.checkNotNull(keyNameList);
List<OmKeyArgs> keyArgsList = new ArrayList<>();
for (String keyName: keyNameList) {
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(volumeName)
.setBucketName(bucketName)
.setKeyName(keyName)
.build();
keyArgsList.add(keyArgs);
}
ozoneManagerClient.deleteKeys(keyArgsList);
}

@Override
public void renameKey(String volumeName, String bucketName,
String fromKeyName, String toKeyName) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,7 @@ public static boolean isReadOnly(
case CreateKey:
case RenameKey:
case DeleteKey:
case DeleteKeys:
case CommitKey:
case AllocateBlock:
case InitiateMultiPartUpload:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -224,6 +224,16 @@ OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID,
*/
void deleteKey(OmKeyArgs args) throws IOException;

/**
* Deletes existing key/keys. This interface supports delete
* multiple keys and a single key. Used by deleting files
* through OzoneFileSystem.
*
* @param args the list args of the key.
* @throws IOException
*/
void deleteKeys(List<OmKeyArgs> args) throws IOException;

/**
* Deletes an existing empty bucket from volume.
* @param volume - Name of the volume.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteBucketRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeysRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeyRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetAclRequest;
Expand Down Expand Up @@ -712,6 +713,34 @@ public void deleteKey(OmKeyArgs args) throws IOException {

}

/**
* Deletes existing key/keys. This interface supports delete
* multiple keys and a single key.
*
* @param args the list args of the key.
* @throws IOException
*/
@Override
public void deleteKeys(List<OmKeyArgs> args) throws IOException {
DeleteKeysRequest.Builder req = DeleteKeysRequest.newBuilder();
List <KeyArgs> keyArgsList = new ArrayList<KeyArgs>();
for (OmKeyArgs omKeyArgs : args) {
KeyArgs keyArgs = KeyArgs.newBuilder()
.setVolumeName(omKeyArgs.getVolumeName())
.setBucketName(omKeyArgs.getBucketName())
.setKeyName(omKeyArgs.getKeyName()).build();
keyArgsList.add(keyArgs);
}
req.addAllKeyArgs(keyArgsList);

OMRequest omRequest = createOMRequest(Type.DeleteKeys)
.setDeleteKeysRequest(req)
.build();

handleError(submitRequest(omRequest));

}

/**
* Deletes an existing empty bucket from volume.
* @param volume - Name of the volume.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@

import org.apache.commons.io.IOUtils;

import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
Expand Down Expand Up @@ -171,6 +172,7 @@ public void testFileSystem() throws Exception {

Copy link
Contributor

Choose a reason for hiding this comment

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

Not related to this class. But you wanna visit TestOzoneManagerHAWithData and see if HA needs a test case for deleting a list of keys.

testCreateDoesNotAddParentDirKeys();
testDeleteCreatesFakeParentDir();
testFileDelete();
testNonExplicitlyCreatedPathExistsAfterItsLeafsWereRemoved();

testRenameDir();
Expand Down Expand Up @@ -204,6 +206,8 @@ private void setupOzoneFileSystem()

// Set the fs.defaultFS and start the filesystem
conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath);
// Set the number of keys to be processed during batch operate.
conf.setInt(OZONE_FS_ITERATE_BATCH_SIZE, 5);
fs = FileSystem.get(conf);
}

Expand Down Expand Up @@ -262,6 +266,44 @@ private void testDeleteCreatesFakeParentDir() throws Exception {
assertEquals(parentKey, parentKeyInfo.getName());
}

Copy link
Contributor

Choose a reason for hiding this comment

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

It would be great if we add a failure case here like there is an unknown key in a list of known keys. We would like to also test exceptions.

private void testFileDelete() throws Exception {
Path grandparent = new Path("/testBatchDelete");
Path parent = new Path(grandparent, "parent");
Path childFolder = new Path(parent, "childFolder");
// BatchSize is 5, so we're going to set a number that's not a
// multiple of 5. In order to test the final number of keys less than
// batchSize can also be deleted.
for (int i = 0; i < 8; i++) {
Path childFile = new Path(parent, "child" + i);
Path childFolderFile = new Path(childFolder, "child" + i);
ContractTestUtils.touch(fs, childFile);
ContractTestUtils.touch(fs, childFolderFile);
}

assertTrue(fs.listStatus(grandparent).length == 1);
assertTrue(fs.listStatus(parent).length == 9);
assertTrue(fs.listStatus(childFolder).length == 8);

Boolean successResult = fs.delete(grandparent, true);
assertTrue(successResult);
assertTrue(!o3fs.exists(grandparent));
for (int i = 0; i < 8; i++) {
Path childFile = new Path(parent, "child" + i);
// Make sure all keys under testBatchDelete/parent should be deleted
assertTrue(!o3fs.exists(childFile));

// Test to recursively delete child folder, make sure all keys under
// testBatchDelete/parent/childFolder should be deleted.
Path childFolderFile = new Path(childFolder, "child" + i);
assertTrue(!o3fs.exists(childFolderFile));
}
// Will get: WARN ozone.BasicOzoneFileSystem delete: Path does not exist.
// This will return false.
Boolean falseResult = fs.delete(parent, true);
assertFalse(falseResult);

}

private void testListStatus() throws Exception {
Path parent = new Path("/testListStatus");
Path file1 = new Path(parent, "key1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.DIRECTORY_NOT_FOUND;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_ALREADY_EXISTS;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_A_FILE;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
import static org.junit.Assert.fail;

/**
Expand Down Expand Up @@ -151,6 +152,46 @@ public void testFileOperationsWithRecursive() throws Exception {

}

@Test
public void testKeysDelete() throws Exception {
OzoneBucket ozoneBucket = setupBucket();
String data = "random data";
String keyName1 = "dir/file1";
String keyName2 = "dir/file2";
String keyName3 = "dir/file3";
String keyName4 = "dir/file4";
List<String> keyList1 = new ArrayList<>();
keyList1.add(keyName2);
keyList1.add(keyName3);

testCreateFile(ozoneBucket, keyName1, data, true, false);
testCreateFile(ozoneBucket, keyName2, data, true, false);
testCreateFile(ozoneBucket, keyName3, data, true, false);
testCreateFile(ozoneBucket, keyName4, data, true, false);
ozoneBucket.getKey("dir/file1").getName();

// Delete keyName1 use deleteKey api.
ozoneBucket.deleteKey(keyName1);

// Delete keyName2 and keyName3 in keyList1 using the deleteKeys api.
ozoneBucket.deleteKeys(keyList1);

// In keyList2 keyName3 was previously deleted and KeyName4 exists .
List<String> keyList2 = new ArrayList<>();
keyList2.add(keyName3);
keyList2.add(keyName4);

// Because keyName3 has been deleted, there should be a KEY_NOT_FOUND
// exception. In this case, we test for deletion failure.
try {
ozoneBucket.deleteKeys(keyList2);
fail("testFilesDelete");
} catch (OMException ex) {
// The expected exception KEY_NOT_FOUND.
Assert.assertEquals(KEY_NOT_FOUND, ex.getResult());
}
}


@Test
public void testFileOperationsWithNonRecursive() throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ enum Type {
ListKeys = 35;
CommitKey = 36;
AllocateBlock = 37;
DeleteKeys = 38;

InitiateMultiPartUpload = 45;
CommitMultiPartUpload = 46;
Expand Down Expand Up @@ -124,6 +125,7 @@ message OMRequest {
optional ListKeysRequest listKeysRequest = 35;
optional CommitKeyRequest commitKeyRequest = 36;
optional AllocateBlockRequest allocateBlockRequest = 37;
optional DeleteKeysRequest deleteKeysRequest = 38;

optional MultipartInfoInitiateRequest initiateMultiPartUploadRequest = 45;
optional MultipartCommitUploadPartRequest commitMultiPartUploadRequest = 46;
Expand Down Expand Up @@ -195,6 +197,7 @@ message OMResponse {
optional ListKeysResponse listKeysResponse = 35;
optional CommitKeyResponse commitKeyResponse = 36;
optional AllocateBlockResponse allocateBlockResponse = 37;
optional DeleteKeysResponse deleteKeysResponse = 38;

optional MultipartInfoInitiateResponse initiateMultiPartUploadResponse = 45;
optional MultipartCommitUploadPartResponse commitMultiPartUploadResponse = 46;
Expand Down Expand Up @@ -841,6 +844,10 @@ message DeleteKeyRequest {
required KeyArgs keyArgs = 1;
}

message DeleteKeysRequest {
repeated KeyArgs keyArgs = 1;
}

message DeleteKeyResponse {

optional KeyInfo keyInfo = 2;
Expand All @@ -856,6 +863,11 @@ message DeletedKeys {
repeated string keys = 3;
}

message DeleteKeysResponse {
repeated KeyInfo deletedKeys = 1;
repeated KeyInfo unDeletedKeys = 2;
}

message PurgeKeysRequest {
repeated DeletedKeys deletedKeys = 1;
}
Expand Down
Loading