Skip to content

Commit

Permalink
HDDS-1731. Implement File CreateFile Request to use Cache and DoubleB…
Browse files Browse the repository at this point in the history
…uffer. (#1044)
  • Loading branch information
bharatviswa504 committed Jul 3, 2019
1 parent 34747c3 commit 8965ddc
Show file tree
Hide file tree
Showing 12 changed files with 1,224 additions and 162 deletions.
3 changes: 3 additions & 0 deletions hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
Expand Up @@ -684,6 +684,9 @@ message CreateFileRequest {
required KeyArgs keyArgs = 1;
required bool isRecursive = 2;
required bool isOverwrite = 3;
// Set in OM HA during preExecute step. This way all OM's use same ID in
// OM HA.
optional uint64 clientID = 4;
}

message CreateFileResponse {
Expand Down
Expand Up @@ -16,6 +16,23 @@
*/
package org.apache.hadoop.ozone.om;

import java.io.IOException;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;

import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.Timeout;
import org.apache.log4j.Logger;

import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
Expand All @@ -30,6 +47,7 @@
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneKeyDetails;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.om.exceptions.OMException;
Expand All @@ -41,22 +59,7 @@
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Logger;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.Timeout;

import java.io.IOException;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;

import static org.apache.hadoop.ozone.MiniOzoneHAClusterImpl
.NODE_FAILURE_TIMEOUT;
Expand All @@ -69,6 +72,9 @@
.OZONE_CLIENT_RETRY_MAX_ATTEMPTS_KEY;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
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.junit.Assert.fail;

/**
* Test Ozone Manager operation in distributed handler scenario.
Expand Down Expand Up @@ -285,6 +291,141 @@ public void testMultipartUpload() throws Exception {

}


@Test
public void testFileOperationsWithRecursive() throws Exception {
OzoneBucket ozoneBucket = setupBucket();

String data = "random data";

// one level key name
String keyName = UUID.randomUUID().toString();
testCreateFile(ozoneBucket, keyName, data, true, false);

// multi level key name
keyName = "dir1/dir2/dir3/file1";
testCreateFile(ozoneBucket, keyName, data, true, false);


data = "random data random data";

// multi level key name with over write set.
testCreateFile(ozoneBucket, keyName, data, true, true);


try {
testCreateFile(ozoneBucket, keyName, data, true, false);
fail("testFileOperationsWithRecursive");
} catch (OMException ex) {
Assert.assertEquals(FILE_ALREADY_EXISTS, ex.getResult());
}

// Try now with a file name which is same as a directory.
try {
keyName = "folder/folder2";
ozoneBucket.createDirectory(keyName);
testCreateFile(ozoneBucket, keyName, data, true, false);
fail("testFileOperationsWithNonRecursive");
} catch (OMException ex) {
Assert.assertEquals(NOT_A_FILE, ex.getResult());
}

}


@Test
public void testFileOperationsWithNonRecursive() throws Exception {
OzoneBucket ozoneBucket = setupBucket();

String data = "random data";

// one level key name
String keyName = UUID.randomUUID().toString();
testCreateFile(ozoneBucket, keyName, data, false, false);

// multi level key name
keyName = "dir1/dir2/dir3/file1";

// Should fail, as this is non-recursive and no parent directories exist
try {
testCreateFile(ozoneBucket, keyName, data, false, false);
} catch (OMException ex) {
Assert.assertEquals(NOT_A_FILE, ex.getResult());
}

// create directory, now this should pass.
ozoneBucket.createDirectory("dir1/dir2/dir3");
testCreateFile(ozoneBucket, keyName, data, false, false);
data = "random data random data";

// multi level key name with over write set.
testCreateFile(ozoneBucket, keyName, data, false, true);

try {
testCreateFile(ozoneBucket, keyName, data, false, false);
fail("testFileOperationsWithRecursive");
} catch (OMException ex) {
Assert.assertEquals(FILE_ALREADY_EXISTS, ex.getResult());
}


// Try now with a file which already exists under the path
ozoneBucket.createDirectory("folder1/folder2/folder3/folder4");

keyName = "folder1/folder2/folder3/folder4/file1";
testCreateFile(ozoneBucket, keyName, data, false, false);

keyName = "folder1/folder2/folder3/file1";
testCreateFile(ozoneBucket, keyName, data, false, false);

// Try now with a file under path already. This should fail.
try {
keyName = "folder/folder2";
ozoneBucket.createDirectory(keyName);
testCreateFile(ozoneBucket, keyName, data, false, false);
fail("testFileOperationsWithNonRecursive");
} catch (OMException ex) {
Assert.assertEquals(NOT_A_FILE, ex.getResult());
}

}

/**
* This method createFile and verifies the file is successfully created or
* not.
* @param ozoneBucket
* @param keyName
* @param data
* @param recursive
* @param overwrite
* @throws Exception
*/
public void testCreateFile(OzoneBucket ozoneBucket, String keyName,
String data, boolean recursive, boolean overwrite)
throws Exception {

OzoneOutputStream ozoneOutputStream = ozoneBucket.createFile(keyName,
data.length(), ReplicationType.RATIS, ReplicationFactor.ONE,
overwrite, recursive);

ozoneOutputStream.write(data.getBytes(), 0, data.length());
ozoneOutputStream.close();

OzoneKeyDetails ozoneKeyDetails = ozoneBucket.getKey(keyName);

Assert.assertEquals(keyName, ozoneKeyDetails.getName());
Assert.assertEquals(ozoneBucket.getName(), ozoneKeyDetails.getBucketName());
Assert.assertEquals(ozoneBucket.getVolumeName(),
ozoneKeyDetails.getVolumeName());
Assert.assertEquals(data.length(), ozoneKeyDetails.getDataSize());

OzoneInputStream ozoneInputStream = ozoneBucket.readKey(keyName);

byte[] fileContent = new byte[data.getBytes().length];
ozoneInputStream.read(fileContent);
Assert.assertEquals(data, new String(fileContent));
}

@Test
public void testMultipartUploadWithOneOmNodeDown() throws Exception {

Expand Down Expand Up @@ -437,7 +578,7 @@ private void createVolumeTest(boolean checkSuccess) throws Exception {
Assert.assertTrue(retVolumeinfo.getAdmin().equals(adminName));
} else {
// Verify that the request failed
Assert.fail("There is no quorum. Request should have failed");
fail("There is no quorum. Request should have failed");
}
} catch (ConnectException | RemoteException e) {
if (!checkSuccess) {
Expand Down Expand Up @@ -566,7 +707,7 @@ public void testOMRetryProxy() throws Exception {

try {
createVolumeTest(true);
Assert.fail("TestOMRetryProxy should fail when there are no OMs running");
fail("TestOMRetryProxy should fail when there are no OMs running");
} catch (ConnectException e) {
// Each retry attempt tries upto 10 times to connect. So there should be
// 10*10 "Retrying connect to server" messages
Expand Down
Expand Up @@ -24,6 +24,7 @@
import org.apache.hadoop.ozone.om.request.bucket.OMBucketSetPropertyRequest;
import org.apache.hadoop.ozone.om.request.OMClientRequest;
import org.apache.hadoop.ozone.om.request.file.OMDirectoryCreateRequest;
import org.apache.hadoop.ozone.om.request.file.OMFileCreateRequest;
import org.apache.hadoop.ozone.om.request.key.OMAllocateBlockRequest;
import org.apache.hadoop.ozone.om.request.key.OMKeyCommitRequest;
import org.apache.hadoop.ozone.om.request.key.OMKeyCreateRequest;
Expand Down Expand Up @@ -93,6 +94,8 @@ public static OMClientRequest createClientRequest(OMRequest omRequest) {
return new OMKeyRenameRequest(omRequest);
case CreateDirectory:
return new OMDirectoryCreateRequest(omRequest);
case CreateFile:
return new OMFileCreateRequest(omRequest);
default:
// TODO: will update once all request types are implemented.
return null;
Expand Down
Expand Up @@ -44,6 +44,8 @@
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.apache.hadoop.security.UserGroupInformation;

import javax.annotation.Nonnull;

/**
* OMClientRequest provides methods which every write OM request should
* implement.
Expand Down Expand Up @@ -170,8 +172,8 @@ public InetAddress getRemoteAddress() throws IOException {
* @param ex - IOException
* @return error response need to be returned to client - OMResponse.
*/
protected OMResponse createErrorOMResponse(OMResponse.Builder omResponse,
IOException ex) {
protected OMResponse createErrorOMResponse(
@Nonnull OMResponse.Builder omResponse, @Nonnull IOException ex) {

omResponse.setSuccess(false);
if (ex.getMessage() != null) {
Expand Down
Expand Up @@ -19,7 +19,6 @@
package org.apache.hadoop.ozone.om.request.file;

import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collections;
Expand Down Expand Up @@ -67,6 +66,10 @@
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_ALREADY_EXISTS;
import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.DIRECTORY_EXISTS_IN_GIVENPATH;
import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.FILE_EXISTS_IN_GIVENPATH;
import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.NONE;
import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.FILE_EXISTS;
/**
* Handle create directory request.
*/
Expand Down Expand Up @@ -156,16 +159,17 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,

// Need to check if any files exist in the given path, if they exist we
// cannot create a directory with the given key.
OMDirectoryResult omDirectoryResult = verifyFilesInPath(omMetadataManager,
volumeName, bucketName, omMetadataManager.getOzoneDirKey(volumeName,
bucketName, keyName), Paths.get(keyName));
OMFileRequest.OMDirectoryResult omDirectoryResult =
OMFileRequest.verifyFilesInPath(omMetadataManager,
volumeName, bucketName, keyName, Paths.get(keyName));

if (omDirectoryResult == OMDirectoryResult.FILE_ALREADY_EXISTS) {
if (omDirectoryResult == FILE_EXISTS ||
omDirectoryResult == FILE_EXISTS_IN_GIVENPATH) {
throw new OMException("Unable to create directory: " +keyName
+ " in volume/bucket: " + volumeName + "/" + bucketName,
FILE_ALREADY_EXISTS);
} else if (omDirectoryResult == OMDirectoryResult.SUB_DIRECTORY_EXISTS ||
omDirectoryResult == OMDirectoryResult.NONE) {
} else if (omDirectoryResult == DIRECTORY_EXISTS_IN_GIVENPATH ||
omDirectoryResult == NONE) {
dirKeyInfo = createDirectoryKeyInfo(ozoneManager, omBucketInfo,
volumeName, bucketName, keyName, keyArgs);

Expand Down Expand Up @@ -206,45 +210,6 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
}
}

/**
* Verify any files exist in the given path in the specified volume/bucket.
* @param omMetadataManager
* @param volumeName
* @param bucketName
* @param keyPath
* @return true - if file exist in the given path, else false.
* @throws IOException
*/
private OMDirectoryResult verifyFilesInPath(
OMMetadataManager omMetadataManager, String volumeName, String bucketName,
String directoryName, Path keyPath) throws IOException {

while (keyPath != null) {
String keyName = keyPath.toString();

String dbKeyName = omMetadataManager.getOzoneKey(volumeName,
bucketName, keyName);
String dbDirKeyName = omMetadataManager.getOzoneDirKey(volumeName,
bucketName, keyName);

if (omMetadataManager.getKeyTable().get(dbKeyName) != null) {
// Found a file in the given path.
return OMDirectoryResult.FILE_ALREADY_EXISTS;
} else if (omMetadataManager.getKeyTable().get(dbDirKeyName) != null) {
if (dbDirKeyName.equals(directoryName)) {
return OMDirectoryResult.DIRECTORY_ALREADY_EXISTS;
} else {
return OMDirectoryResult.SUB_DIRECTORY_EXISTS;
}
}
keyPath = keyPath.getParent();
}

// Found no files/ directories in the given path.
return OMDirectoryResult.NONE;
}


private OmKeyInfo createDirectoryKeyInfo(OzoneManager ozoneManager,
OmBucketInfo omBucketInfo, String volumeName, String bucketName,
String keyName, KeyArgs keyArgs)
Expand All @@ -269,14 +234,4 @@ private OmKeyInfo createDirectoryKeyInfo(OzoneManager ozoneManager,
.build();
}

/**
* Return codes used by verifyFilesInPath method.
*/
enum OMDirectoryResult {
DIRECTORY_ALREADY_EXISTS,
FILE_ALREADY_EXISTS,
SUB_DIRECTORY_EXISTS,
NONE
}

}

0 comments on commit 8965ddc

Please sign in to comment.