HDFS-12063. Ozone: Ozone shell: Multiple RPC calls for put/get key. Contributed by Yiqun Lin.
This commit is contained in:
parent
0a8c903d3f
commit
b49c165d28
|
@ -249,8 +249,8 @@ public class OzoneBucket {
|
|||
* @throws OzoneException
|
||||
* @throws IOException
|
||||
*/
|
||||
private void executePutKey(HttpPut putRequest, CloseableHttpClient httpClient)
|
||||
throws OzoneException, IOException {
|
||||
public static void executePutKey(HttpPut putRequest,
|
||||
CloseableHttpClient httpClient) throws OzoneException, IOException {
|
||||
HttpEntity entity = null;
|
||||
try {
|
||||
HttpResponse response = httpClient.execute(putRequest);
|
||||
|
@ -354,8 +354,8 @@ public class OzoneBucket {
|
|||
* @throws IOException
|
||||
* @throws OzoneException
|
||||
*/
|
||||
private void executeGetKey(HttpGet getRequest, CloseableHttpClient httpClient,
|
||||
OutputStream stream)
|
||||
public static void executeGetKey(HttpGet getRequest,
|
||||
CloseableHttpClient httpClient, OutputStream stream)
|
||||
throws IOException, OzoneException {
|
||||
|
||||
HttpEntity entity = null;
|
||||
|
@ -477,9 +477,8 @@ public class OzoneBucket {
|
|||
* @throws IOException
|
||||
* @throws OzoneException
|
||||
*/
|
||||
private List<OzoneKey> executeListKeys(HttpGet getRequest,
|
||||
CloseableHttpClient httpClient)
|
||||
throws IOException, OzoneException {
|
||||
public static List<OzoneKey> executeListKeys(HttpGet getRequest,
|
||||
CloseableHttpClient httpClient) throws IOException, OzoneException {
|
||||
HttpEntity entity = null;
|
||||
List<OzoneKey> ozoneKeyList = new LinkedList<OzoneKey>();
|
||||
try {
|
||||
|
|
|
@ -18,6 +18,10 @@
|
|||
package org.apache.hadoop.ozone.web.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import org.apache.commons.codec.digest.DigestUtils;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.ozone.OzoneClientUtils;
|
||||
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
|
||||
import org.apache.hadoop.ozone.web.headers.Header;
|
||||
|
@ -33,14 +37,20 @@ import org.apache.http.client.methods.HttpPost;
|
|||
import org.apache.http.client.methods.HttpPut;
|
||||
import org.apache.http.client.methods.HttpRequestBase;
|
||||
import org.apache.http.client.utils.URIBuilder;
|
||||
import org.apache.http.entity.ContentType;
|
||||
import org.apache.http.entity.FileEntity;
|
||||
import org.apache.http.impl.client.CloseableHttpClient;
|
||||
import org.apache.http.util.EntityUtils;
|
||||
|
||||
import javax.ws.rs.core.HttpHeaders;
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.file.Path;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.LinkedList;
|
||||
|
@ -144,7 +154,7 @@ public class OzoneRestClient implements Closeable {
|
|||
String quota) throws OzoneException {
|
||||
HttpPost httpPost = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(volumeName);
|
||||
OzoneUtils.verifyResourceName(volumeName);
|
||||
|
||||
URIBuilder builder = new URIBuilder(endPointURI);
|
||||
builder.setPath("/" + volumeName);
|
||||
|
@ -173,7 +183,7 @@ public class OzoneRestClient implements Closeable {
|
|||
public OzoneVolume getVolume(String volumeName) throws OzoneException {
|
||||
HttpGet httpGet = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(volumeName);
|
||||
OzoneUtils.verifyResourceName(volumeName);
|
||||
URIBuilder builder = new URIBuilder(endPointURI);
|
||||
builder.setPath("/" + volumeName)
|
||||
.setParameter(Header.OZONE_LIST_QUERY_TAG,
|
||||
|
@ -301,7 +311,7 @@ public class OzoneRestClient implements Closeable {
|
|||
public void deleteVolume(String volumeName) throws OzoneException {
|
||||
HttpDelete httpDelete = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(volumeName);
|
||||
OzoneUtils.verifyResourceName(volumeName);
|
||||
URIBuilder builder = new URIBuilder(endPointURI);
|
||||
builder.setPath("/" + volumeName).build();
|
||||
|
||||
|
@ -328,7 +338,7 @@ public class OzoneRestClient implements Closeable {
|
|||
throw new OzoneRestClientException("Invalid new owner name");
|
||||
}
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(volumeName);
|
||||
OzoneUtils.verifyResourceName(volumeName);
|
||||
URIBuilder builder = new URIBuilder(endPointURI);
|
||||
builder.setPath("/" + volumeName).build();
|
||||
|
||||
|
@ -360,7 +370,7 @@ public class OzoneRestClient implements Closeable {
|
|||
}
|
||||
HttpPut putRequest = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(volumeName);
|
||||
OzoneUtils.verifyResourceName(volumeName);
|
||||
URIBuilder builder = new URIBuilder(endPointURI);
|
||||
builder.setPath("/" + volumeName)
|
||||
.setParameter(Header.OZONE_QUOTA_QUERY_TAG, quota)
|
||||
|
@ -537,6 +547,121 @@ public class OzoneRestClient implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Puts a Key in Ozone Bucket.
|
||||
*
|
||||
* @param volumeName - Name of the Volume
|
||||
* @param bucketName - Name of the Bucket
|
||||
* @param keyName - Name of the Key
|
||||
* @param file - Stream that gets read to be put into Ozone.
|
||||
* @throws OzoneException
|
||||
*/
|
||||
public void putKey(String volumeName, String bucketName, String keyName,
|
||||
File file) throws OzoneException {
|
||||
OzoneUtils.verifyResourceName(volumeName);
|
||||
OzoneUtils.verifyResourceName(bucketName);
|
||||
|
||||
if (StringUtils.isEmpty(keyName)) {
|
||||
throw new OzoneRestClientException("Invalid key Name");
|
||||
}
|
||||
|
||||
if (file == null) {
|
||||
throw new OzoneRestClientException("Invalid data stream");
|
||||
}
|
||||
|
||||
HttpPut putRequest = null;
|
||||
FileInputStream fis = null;
|
||||
try (CloseableHttpClient httpClient = OzoneClientUtils.newHttpClient()) {
|
||||
URIBuilder builder = new URIBuilder(getEndPointURI());
|
||||
builder.setPath("/" + volumeName + "/" + bucketName + "/" + keyName)
|
||||
.build();
|
||||
|
||||
putRequest = getHttpPut(builder.toString());
|
||||
|
||||
FileEntity fileEntity = new FileEntity(file, ContentType
|
||||
.APPLICATION_OCTET_STREAM);
|
||||
putRequest.setEntity(fileEntity);
|
||||
|
||||
fis = new FileInputStream(file);
|
||||
putRequest.setHeader(Header.CONTENT_MD5, DigestUtils.md5Hex(fis));
|
||||
OzoneBucket.executePutKey(putRequest, httpClient);
|
||||
} catch (IOException | URISyntaxException ex) {
|
||||
throw new OzoneRestClientException(ex.getMessage());
|
||||
} finally {
|
||||
IOUtils.closeStream(fis);
|
||||
OzoneClientUtils.releaseConnection(putRequest);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a key from the Ozone server and writes to the file pointed by the
|
||||
* downloadTo Path.
|
||||
*
|
||||
* @param volumeName - Volume Name in Ozone.
|
||||
* @param bucketName - Bucket Name in Ozone.
|
||||
* @param keyName - Key Name in Ozone.
|
||||
* @param downloadTo File Name to download the Key's Data to
|
||||
*/
|
||||
public void getKey(String volumeName, String bucketName, String keyName,
|
||||
Path downloadTo) throws OzoneException {
|
||||
OzoneUtils.verifyResourceName(volumeName);
|
||||
OzoneUtils.verifyResourceName(bucketName);
|
||||
|
||||
if (StringUtils.isEmpty(keyName)) {
|
||||
throw new OzoneRestClientException("Invalid key Name");
|
||||
}
|
||||
|
||||
if (downloadTo == null) {
|
||||
throw new OzoneRestClientException("Invalid download path");
|
||||
}
|
||||
|
||||
FileOutputStream outPutFile = null;
|
||||
HttpGet getRequest = null;
|
||||
try (CloseableHttpClient httpClient = OzoneClientUtils.newHttpClient()) {
|
||||
outPutFile = new FileOutputStream(downloadTo.toFile());
|
||||
|
||||
URIBuilder builder = new URIBuilder(getEndPointURI());
|
||||
builder.setPath("/" + volumeName + "/" + bucketName + "/" + keyName)
|
||||
.build();
|
||||
|
||||
getRequest = getHttpGet(builder.toString());
|
||||
OzoneBucket.executeGetKey(getRequest, httpClient, outPutFile);
|
||||
outPutFile.flush();
|
||||
} catch (IOException | URISyntaxException ex) {
|
||||
throw new OzoneRestClientException(ex.getMessage());
|
||||
} finally {
|
||||
IOUtils.closeStream(outPutFile);
|
||||
OzoneClientUtils.releaseConnection(getRequest);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* List all keys in the given bucket.
|
||||
*
|
||||
* @param volumeName - Volume name
|
||||
* @param bucketName - Bucket name
|
||||
*
|
||||
* @return List of OzoneKeys
|
||||
*/
|
||||
public List<OzoneKey> listKeys(String volumeName, String bucketName)
|
||||
throws OzoneException {
|
||||
OzoneUtils.verifyResourceName(volumeName);
|
||||
OzoneUtils.verifyResourceName(bucketName);
|
||||
|
||||
HttpGet getRequest = null;
|
||||
try (CloseableHttpClient httpClient = OzoneClientUtils.newHttpClient()) {
|
||||
URIBuilder builder = new URIBuilder(getEndPointURI());
|
||||
builder.setPath("/" + volumeName + "/" + bucketName).build();
|
||||
|
||||
getRequest = getHttpGet(builder.toString());
|
||||
return OzoneBucket.executeListKeys(getRequest, httpClient);
|
||||
} catch (IOException | URISyntaxException e) {
|
||||
throw new OzoneRestClientException(e.getMessage());
|
||||
} finally {
|
||||
OzoneClientUtils.releaseConnection(getRequest);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a standard HttpPost Object to use for ozone post requests.
|
||||
*
|
||||
|
|
|
@ -167,7 +167,7 @@ public class OzoneVolume {
|
|||
|
||||
HttpPost httpPost = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(bucketName);
|
||||
OzoneUtils.verifyResourceName(bucketName);
|
||||
URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
|
||||
builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
|
||||
|
||||
|
@ -277,7 +277,7 @@ public class OzoneVolume {
|
|||
public void addAcls(String bucketName, String[] acls) throws OzoneException {
|
||||
HttpPut putRequest = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(bucketName);
|
||||
OzoneUtils.verifyResourceName(bucketName);
|
||||
URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
|
||||
builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
|
||||
putRequest = client.getHttpPut(builder.toString());
|
||||
|
@ -306,7 +306,7 @@ public class OzoneVolume {
|
|||
throws OzoneException {
|
||||
HttpPut putRequest = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(bucketName);
|
||||
OzoneUtils.verifyResourceName(bucketName);
|
||||
URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
|
||||
builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
|
||||
putRequest = client.getHttpPut(builder.toString());
|
||||
|
@ -333,7 +333,7 @@ public class OzoneVolume {
|
|||
public OzoneBucket getBucket(String bucketName) throws OzoneException {
|
||||
HttpGet getRequest = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(bucketName);
|
||||
OzoneUtils.verifyResourceName(bucketName);
|
||||
URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
|
||||
builder.setPath("/" + getVolumeName() + "/" + bucketName)
|
||||
.setParameter(Header.OZONE_LIST_QUERY_TAG,
|
||||
|
@ -498,7 +498,7 @@ public class OzoneVolume {
|
|||
public void deleteBucket(String bucketName) throws OzoneException {
|
||||
HttpDelete delRequest = null;
|
||||
try (CloseableHttpClient httpClient = newHttpClient()) {
|
||||
OzoneUtils.verifyBucketName(bucketName);
|
||||
OzoneUtils.verifyResourceName(bucketName);
|
||||
URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
|
||||
builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
|
||||
|
||||
|
|
|
@ -67,6 +67,10 @@ public final class ErrorTable {
|
|||
new OzoneException(HTTP_BAD_REQUEST, "invalidQueryParam",
|
||||
"Invalid query parameter.");
|
||||
|
||||
public static final OzoneException INVALID_RESOURCE_NAME =
|
||||
new OzoneException(HTTP_BAD_REQUEST, "invalidResourceName",
|
||||
"Invalid volume, bucket or key name.");
|
||||
|
||||
public static final OzoneException INVALID_BUCKET_NAME =
|
||||
new OzoneException(HTTP_BAD_REQUEST, "invalidBucketName",
|
||||
"Invalid bucket name.");
|
||||
|
|
|
@ -85,7 +85,7 @@ public abstract class BucketProcessTemplate {
|
|||
userArgs = new UserArgs(reqID, hostName, request, uriInfo, headers);
|
||||
|
||||
OzoneUtils.validate(request, headers, reqID, bucket, hostName);
|
||||
OzoneUtils.verifyBucketName(bucket);
|
||||
OzoneUtils.verifyResourceName(bucket);
|
||||
|
||||
UserAuth auth = UserHandlerBuilder.getAuthHandler();
|
||||
userArgs.setUserName(auth.getUser(userArgs));
|
||||
|
|
|
@ -79,7 +79,7 @@ public abstract class KeyProcessTemplate {
|
|||
try {
|
||||
userArgs = new UserArgs(reqID, hostName, request, info, headers);
|
||||
OzoneUtils.validate(request, headers, reqID, bucket, hostName);
|
||||
OzoneUtils.verifyBucketName(bucket);
|
||||
OzoneUtils.verifyResourceName(bucket);
|
||||
|
||||
UserAuth auth = UserHandlerBuilder.getAuthHandler();
|
||||
userArgs.setUserName(auth.getUser(userArgs));
|
||||
|
|
|
@ -82,7 +82,7 @@ public abstract class VolumeProcessTemplate {
|
|||
OzoneUtils.validate(request, headers, reqID, volume, hostName);
|
||||
|
||||
// we use the same logic for both bucket and volume names
|
||||
OzoneUtils.verifyBucketName(volume);
|
||||
OzoneUtils.verifyResourceName(volume);
|
||||
UserAuth auth = UserHandlerBuilder.getAuthHandler();
|
||||
|
||||
userArgs.setUserName(auth.getUser(userArgs));
|
||||
|
|
|
@ -923,7 +923,7 @@ public final class OzoneMetadataManager {
|
|||
byte[] bucketInfo = metadataDB.get(args.getParentName()
|
||||
.getBytes(encoding));
|
||||
if (bucketInfo == null) {
|
||||
throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
|
||||
throw ErrorTable.newError(ErrorTable.INVALID_RESOURCE_NAME, args);
|
||||
}
|
||||
BucketInfo bInfo = BucketInfo.parse(new String(bucketInfo, encoding));
|
||||
bInfo.setKeyCount(bInfo.getKeyCount() + 1);
|
||||
|
@ -1066,7 +1066,7 @@ public final class OzoneMetadataManager {
|
|||
String fullPath = storageRoot + OBJECT_DIR + fileNameHash;
|
||||
File f = new File(fullPath);
|
||||
if (!f.exists()) {
|
||||
throw ErrorTable.newError(ErrorTable.INVALID_KEY, args);
|
||||
throw ErrorTable.newError(ErrorTable.INVALID_RESOURCE_NAME, args);
|
||||
}
|
||||
long size = f.length();
|
||||
|
||||
|
@ -1094,13 +1094,13 @@ public final class OzoneMetadataManager {
|
|||
byte[] bucketInfo = metadataDB.get(bArgs.getResourceName()
|
||||
.getBytes(encoding));
|
||||
if (bucketInfo == null) {
|
||||
throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, bArgs);
|
||||
throw ErrorTable.newError(ErrorTable.INVALID_RESOURCE_NAME, bArgs);
|
||||
}
|
||||
|
||||
byte[] bucketListBytes = userDB.get(bArgs.getResourceName()
|
||||
.getBytes(encoding));
|
||||
if (bucketListBytes == null) {
|
||||
throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, bArgs);
|
||||
throw ErrorTable.newError(ErrorTable.INVALID_RESOURCE_NAME, bArgs);
|
||||
}
|
||||
return ListKeys.parse(new String(bucketListBytes, encoding));
|
||||
} finally {
|
||||
|
|
|
@ -107,11 +107,7 @@ public class GetKeyHandler extends Handler {
|
|||
client.setEndPointURI(ozoneURI);
|
||||
client.setUserAuth(userName);
|
||||
|
||||
|
||||
OzoneVolume vol = client.getVolume(volumeName);
|
||||
OzoneBucket bucket = vol.getBucket(bucketName);
|
||||
bucket.getKey(keyName, dataFilePath);
|
||||
|
||||
client.getKey(volumeName, bucketName, keyName, dataFilePath);
|
||||
if(cmd.hasOption(Shell.VERBOSE)) {
|
||||
FileInputStream stream = new FileInputStream(dataFile);
|
||||
String hash = DigestUtils.md5Hex(stream);
|
||||
|
|
|
@ -87,11 +87,7 @@ public class ListKeyHandler extends Handler {
|
|||
client.setEndPointURI(ozoneURI);
|
||||
client.setUserAuth(userName);
|
||||
|
||||
|
||||
OzoneVolume vol = client.getVolume(volumeName);
|
||||
OzoneBucket bucket = vol.getBucket(bucketName);
|
||||
List<OzoneKey> keys = bucket.listKeys();
|
||||
|
||||
List<OzoneKey> keys = client.listKeys(volumeName, bucketName);
|
||||
for (OzoneKey key : keys) {
|
||||
System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
|
||||
key.getObjectInfo().toJsonString()));
|
||||
|
|
|
@ -102,9 +102,7 @@ public class PutKeyHandler extends Handler {
|
|||
client.setEndPointURI(ozoneURI);
|
||||
client.setUserAuth(userName);
|
||||
|
||||
OzoneVolume vol = client.getVolume(volumeName);
|
||||
OzoneBucket bucket = vol.getBucket(bucketName);
|
||||
bucket.putKey(keyName, dataFile);
|
||||
client.putKey(volumeName, bucketName, keyName, dataFile);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -61,31 +61,31 @@ public final class OzoneUtils {
|
|||
/**
|
||||
* verifies that bucket name / volume name is a valid DNS name.
|
||||
*
|
||||
* @param bucketName Bucket Name to be validated
|
||||
* @param resName Bucket or volume Name to be validated
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public static void verifyBucketName(String bucketName)
|
||||
public static void verifyResourceName(String resName)
|
||||
throws IllegalArgumentException {
|
||||
|
||||
if (bucketName == null) {
|
||||
if (resName == null) {
|
||||
throw new IllegalArgumentException("Bucket or Volume name is null");
|
||||
}
|
||||
|
||||
if ((bucketName.length() < OzoneConsts.OZONE_MIN_BUCKET_NAME_LENGTH) ||
|
||||
(bucketName.length() > OzoneConsts.OZONE_MAX_BUCKET_NAME_LENGTH)) {
|
||||
if ((resName.length() < OzoneConsts.OZONE_MIN_BUCKET_NAME_LENGTH) ||
|
||||
(resName.length() > OzoneConsts.OZONE_MAX_BUCKET_NAME_LENGTH)) {
|
||||
throw new IllegalArgumentException(
|
||||
"Bucket or Volume length is illegal, " +
|
||||
"valid length is 3-63 characters");
|
||||
}
|
||||
|
||||
if ((bucketName.charAt(0) == '.') || (bucketName.charAt(0) == '-')) {
|
||||
if ((resName.charAt(0) == '.') || (resName.charAt(0) == '-')) {
|
||||
throw new IllegalArgumentException(
|
||||
"Bucket or Volume name cannot start with a period or dash");
|
||||
}
|
||||
|
||||
if ((bucketName.charAt(bucketName.length() - 1) == '.') ||
|
||||
(bucketName.charAt(bucketName.length() - 1) == '-')) {
|
||||
if ((resName.charAt(resName.length() - 1) == '.') ||
|
||||
(resName.charAt(resName.length() - 1) == '-')) {
|
||||
throw new IllegalArgumentException(
|
||||
"Bucket or Volume name cannot end with a period or dash");
|
||||
}
|
||||
|
@ -93,8 +93,8 @@ public final class OzoneUtils {
|
|||
boolean isIPv4 = true;
|
||||
char prev = (char) 0;
|
||||
|
||||
for (int index = 0; index < bucketName.length(); index++) {
|
||||
char currChar = bucketName.charAt(index);
|
||||
for (int index = 0; index < resName.length(); index++) {
|
||||
char currChar = resName.charAt(index);
|
||||
|
||||
if (currChar != '.') {
|
||||
isIPv4 = ((currChar >= '0') && (currChar <= '9')) && isIPv4;
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.HashSet;
|
|||
import java.util.Set;
|
||||
|
||||
import static org.apache.hadoop.ozone.web.utils.OzoneUtils.getRequestID;
|
||||
import static org.apache.hadoop.ozone.web.utils.OzoneUtils.verifyBucketName;
|
||||
import static org.apache.hadoop.ozone.web.utils.OzoneUtils.verifyResourceName;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
|
@ -69,10 +69,10 @@ public class TestUtils {
|
|||
if(testMatrix.get(key)) {
|
||||
|
||||
// For valid names there should be no exceptions at all
|
||||
verifyBucketName(key);
|
||||
verifyResourceName(key);
|
||||
} else {
|
||||
try {
|
||||
verifyBucketName(key);
|
||||
verifyResourceName(key);
|
||||
// should never get here since the isValid call will throw
|
||||
fail("An exception was expected but did not happen.");
|
||||
} catch(IllegalArgumentException e){
|
||||
|
|
|
@ -170,6 +170,36 @@ public class TestKeys {
|
|||
helper.putKey();
|
||||
assertNotNull(helper.getBucket());
|
||||
assertNotNull(helper.getFile());
|
||||
List<OzoneKey> keyList = helper.getBucket().listKeys();
|
||||
Assert.assertEquals(keyList.size(), 1);
|
||||
|
||||
// test list key using a more efficient call
|
||||
String newkeyName = OzoneUtils.getRequestID().toLowerCase();
|
||||
client.putKey(helper.getVol().getVolumeName(),
|
||||
helper.getBucket().getBucketName(), newkeyName, helper.getFile());
|
||||
keyList = helper.getBucket().listKeys();
|
||||
Assert.assertEquals(keyList.size(), 2);
|
||||
|
||||
// test new put key with invalid volume/bucket name
|
||||
try{
|
||||
client.putKey("invalid-volume",
|
||||
helper.getBucket().getBucketName(), newkeyName, helper.getFile());
|
||||
fail("Put key should have thrown"
|
||||
+ " when using invalid volume name.");
|
||||
} catch(OzoneException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
ErrorTable.INVALID_RESOURCE_NAME.getMessage(), e);
|
||||
}
|
||||
|
||||
try {
|
||||
client.putKey(helper.getVol().getVolumeName(), "invalid-bucket",
|
||||
newkeyName, helper.getFile());
|
||||
fail("Put key should have thrown "
|
||||
+ "when using invalid bucket name.");
|
||||
} catch (OzoneException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
ErrorTable.INVALID_RESOURCE_NAME.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private void restartDatanode(int datanodeIdx)
|
||||
|
@ -219,21 +249,52 @@ public class TestKeys {
|
|||
assertNotNull(helper.getBucket());
|
||||
assertNotNull(helper.getFile());
|
||||
|
||||
String newFileName = path + "/" +OzoneUtils.getRequestID().toLowerCase();
|
||||
Path newPath = Paths.get(newFileName);
|
||||
helper.getBucket().getKey(keyName, newPath);
|
||||
String newFileName1 = path + "/" +OzoneUtils.getRequestID().toLowerCase();
|
||||
String newFileName2 = path + "/" +OzoneUtils.getRequestID().toLowerCase();
|
||||
|
||||
Path newPath1 = Paths.get(newFileName1);
|
||||
Path newPath2 = Paths.get(newFileName2);
|
||||
|
||||
helper.getBucket().getKey(keyName, newPath1);
|
||||
// test get key using a more efficient call
|
||||
client.getKey(helper.getVol().getVolumeName(),
|
||||
helper.getBucket().getBucketName(), keyName, newPath2);
|
||||
|
||||
FileInputStream original = new FileInputStream(helper.getFile());
|
||||
FileInputStream downloaded = new FileInputStream(newPath.toFile());
|
||||
|
||||
FileInputStream downloaded1 = new FileInputStream(newPath1.toFile());
|
||||
FileInputStream downloaded2 = new FileInputStream(newPath1.toFile());
|
||||
|
||||
String originalHash = DigestUtils.sha256Hex(original);
|
||||
String downloadedHash = DigestUtils.sha256Hex(downloaded);
|
||||
String downloadedHash1 = DigestUtils.sha256Hex(downloaded1);
|
||||
String downloadedHash2 = DigestUtils.sha256Hex(downloaded2);
|
||||
|
||||
assertEquals(
|
||||
"Sha256 does not match between original file and downloaded file.",
|
||||
originalHash, downloadedHash);
|
||||
originalHash, downloadedHash1);
|
||||
assertEquals(
|
||||
"Sha256 does not match between original file and downloaded file.",
|
||||
originalHash, downloadedHash2);
|
||||
|
||||
// test new get key with invalid volume/bucket name
|
||||
try {
|
||||
client.getKey("invalid-volume",
|
||||
helper.getBucket().getBucketName(), keyName, newPath1);
|
||||
fail("Get key should have thrown "
|
||||
+ "when using invalid volume name.");
|
||||
} catch (OzoneException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
ErrorTable.INVALID_RESOURCE_NAME.getMessage(), e);
|
||||
}
|
||||
|
||||
try {
|
||||
client.getKey(helper.getVol().getVolumeName(),
|
||||
"invalid-bucket", keyName, newPath1);
|
||||
fail("Get key should have thrown "
|
||||
+ "when using invalid bucket name.");
|
||||
} catch (OzoneException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
ErrorTable.INVALID_RESOURCE_NAME.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -250,7 +311,7 @@ public class TestKeys {
|
|||
fail("Get Key on a deleted key should have thrown");
|
||||
} catch (OzoneException ex) {
|
||||
assertEquals(ex.getShortMessage(),
|
||||
ErrorTable.INVALID_KEY.getShortMessage());
|
||||
ErrorTable.INVALID_RESOURCE_NAME.getShortMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -267,7 +328,29 @@ public class TestKeys {
|
|||
helper.getBucket().putKey(newkeyName, helper.getFile());
|
||||
}
|
||||
|
||||
List<OzoneKey> keyList = helper.getBucket().listKeys();
|
||||
Assert.assertEquals(keyList.size(), 11);
|
||||
List<OzoneKey> keyList1 = helper.getBucket().listKeys();
|
||||
// test list key using a more efficient call
|
||||
List<OzoneKey> keyList2 = client.listKeys(helper.getVol().getVolumeName(),
|
||||
helper.getBucket().getBucketName());
|
||||
|
||||
Assert.assertEquals(keyList1.size(), 11);
|
||||
Assert.assertEquals(keyList2.size(), 11);
|
||||
|
||||
// test new list keys with invalid volume/bucket name
|
||||
try {
|
||||
client.listKeys("invalid-volume", helper.getBucket().getBucketName());
|
||||
fail("List keys should have thrown when using invalid volume name.");
|
||||
} catch (OzoneException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
ErrorTable.INVALID_RESOURCE_NAME.getMessage(), e);
|
||||
}
|
||||
|
||||
try {
|
||||
client.listKeys(helper.getVol().getVolumeName(), "invalid-bucket");
|
||||
fail("List keys should have thrown when using invalid bucket name.");
|
||||
} catch (OzoneException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
ErrorTable.INVALID_RESOURCE_NAME.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue