|
@@ -20,16 +20,29 @@ package org.apache.hadoop.ozone.web.client;
|
|
|
import com.google.common.collect.Lists;
|
|
|
import com.google.common.collect.Maps;
|
|
|
import org.apache.commons.codec.digest.DigestUtils;
|
|
|
+import org.apache.commons.collections.IteratorUtils;
|
|
|
import org.apache.commons.io.FileUtils;
|
|
|
+import org.apache.commons.io.IOUtils;
|
|
|
import org.apache.commons.lang.RandomStringUtils;
|
|
|
import org.apache.commons.lang.math.RandomUtils;
|
|
|
import org.apache.commons.lang3.tuple.ImmutablePair;
|
|
|
import org.apache.commons.lang3.tuple.Pair;
|
|
|
-import org.apache.hadoop.fs.StorageType;
|
|
|
+import org.apache.hadoop.hdds.client.ReplicationFactor;
|
|
|
+import org.apache.hadoop.hdds.client.ReplicationType;
|
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
|
+import org.apache.hadoop.ozone.OzoneAcl;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
|
+import org.apache.hadoop.ozone.client.BucketArgs;
|
|
|
+import org.apache.hadoop.ozone.client.VolumeArgs;
|
|
|
+import org.apache.hadoop.ozone.client.OzoneBucket;
|
|
|
+import org.apache.hadoop.ozone.client.OzoneKey;
|
|
|
+import org.apache.hadoop.ozone.client.OzoneVolume;
|
|
|
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
|
|
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
|
|
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
|
|
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
|
@@ -59,15 +72,18 @@ import org.junit.rules.Timeout;
|
|
|
import java.io.File;
|
|
|
import java.io.FileInputStream;
|
|
|
import java.io.FileOutputStream;
|
|
|
+import java.io.InputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.nio.file.Path;
|
|
|
import java.nio.file.Paths;
|
|
|
+import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.Random;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.stream.Collectors;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
@@ -84,10 +100,13 @@ public class TestKeys {
|
|
|
@Rule
|
|
|
public Timeout testTimeout = new Timeout(300000);
|
|
|
|
|
|
+ private static OzoneConfiguration conf;
|
|
|
private static MiniOzoneCluster ozoneCluster = null;
|
|
|
private static String path;
|
|
|
- private static OzoneRestClient ozoneRestClient = null;
|
|
|
+ private static ClientProtocol client = null;
|
|
|
private static long currentTime;
|
|
|
+ private static ReplicationFactor replicationFactor = ReplicationFactor.ONE;
|
|
|
+ private static ReplicationType replicationType = ReplicationType.STAND_ALONE;
|
|
|
|
|
|
/**
|
|
|
* Create a MiniDFSCluster for testing.
|
|
@@ -96,7 +115,7 @@ public class TestKeys {
|
|
|
*/
|
|
|
@BeforeClass
|
|
|
public static void init() throws Exception {
|
|
|
- OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
+ conf = new OzoneConfiguration();
|
|
|
|
|
|
// Set short block deleting service interval to speed up deletions.
|
|
|
conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
|
|
@@ -105,12 +124,9 @@ public class TestKeys {
|
|
|
path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
|
|
|
Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
|
|
|
|
|
|
- ozoneCluster = MiniOzoneCluster.newBuilder(conf).build();
|
|
|
+ ozoneCluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
|
|
|
ozoneCluster.waitForClusterToBeReady();
|
|
|
- final int port = ozoneCluster.getHddsDatanodes().get(0)
|
|
|
- .getDatanodeDetails().getOzoneRestPort();
|
|
|
- ozoneRestClient = new OzoneRestClient(
|
|
|
- String.format("http://localhost:%d", port));
|
|
|
+ client = new RpcClient(conf);
|
|
|
currentTime = Time.now();
|
|
|
}
|
|
|
|
|
@@ -166,7 +182,7 @@ public class TestKeys {
|
|
|
}
|
|
|
|
|
|
static class PutHelper {
|
|
|
- private final OzoneRestClient client;
|
|
|
+ private final ClientProtocol client;
|
|
|
private final String dir;
|
|
|
private final String keyName;
|
|
|
|
|
@@ -174,11 +190,11 @@ public class TestKeys {
|
|
|
private OzoneBucket bucket;
|
|
|
private File file;
|
|
|
|
|
|
- PutHelper(OzoneRestClient client, String dir) {
|
|
|
+ PutHelper(ClientProtocol client, String dir) {
|
|
|
this(client, dir, OzoneUtils.getRequestID().toLowerCase());
|
|
|
}
|
|
|
|
|
|
- PutHelper(OzoneRestClient client, String dir, String key) {
|
|
|
+ PutHelper(ClientProtocol client, String dir, String key) {
|
|
|
this.client = client;
|
|
|
this.dir = dir;
|
|
|
this.keyName = key;
|
|
@@ -202,111 +218,139 @@ public class TestKeys {
|
|
|
* @return Returns the name of the new key that was created.
|
|
|
* @throws OzoneException
|
|
|
*/
|
|
|
- private KsmKeyArgs putKey() throws Exception {
|
|
|
+ private String putKey() throws Exception {
|
|
|
String volumeName = OzoneUtils.getRequestID().toLowerCase();
|
|
|
- client.setUserAuth("hdfs");
|
|
|
|
|
|
- vol = client.createVolume(volumeName, "bilbo", "100TB");
|
|
|
+ VolumeArgs volumeArgs = VolumeArgs.newBuilder()
|
|
|
+ .setOwner("bilbo")
|
|
|
+ .setQuota("100TB")
|
|
|
+ .setAdmin("hdfs")
|
|
|
+ .build();
|
|
|
+ client.createVolume(volumeName, volumeArgs);
|
|
|
+ vol = client.getVolumeDetails(volumeName);
|
|
|
String[] acls = {"user:frodo:rw", "user:samwise:rw"};
|
|
|
|
|
|
String bucketName = OzoneUtils.getRequestID().toLowerCase();
|
|
|
- bucket = vol.createBucket(bucketName, acls, StorageType.DEFAULT);
|
|
|
+ List<OzoneAcl> aclList =
|
|
|
+ Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
|
|
|
+ .collect(Collectors.toList());
|
|
|
+ BucketArgs bucketArgs = BucketArgs.newBuilder()
|
|
|
+ .setAcls(aclList)
|
|
|
+ .build();
|
|
|
+ vol.createBucket(bucketName, bucketArgs);
|
|
|
+ bucket = vol.getBucket(bucketName);
|
|
|
|
|
|
String fileName = OzoneUtils.getRequestID().toLowerCase();
|
|
|
|
|
|
file = createRandomDataFile(dir, fileName, 1024);
|
|
|
|
|
|
- bucket.putKey(keyName, file);
|
|
|
- return new KsmKeyArgs.Builder()
|
|
|
- .setKeyName(keyName)
|
|
|
- .setVolumeName(volumeName)
|
|
|
- .setBucketName(bucketName)
|
|
|
- .setDataSize(1024)
|
|
|
- .build();
|
|
|
+ try (
|
|
|
+ OzoneOutputStream ozoneOutputStream = bucket
|
|
|
+ .createKey(keyName, 0, replicationType, replicationFactor);
|
|
|
+ InputStream fileInputStream = new FileInputStream(file)) {
|
|
|
+ IOUtils.copy(fileInputStream, ozoneOutputStream);
|
|
|
+ }
|
|
|
+ return keyName;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testPutKey() throws Exception {
|
|
|
// Test non-delimited keys
|
|
|
- runTestPutKey(new PutHelper(ozoneRestClient, path));
|
|
|
+ runTestPutKey(new PutHelper(client, path));
|
|
|
// Test key delimited by a random delimiter
|
|
|
String delimiter = RandomStringUtils.randomAscii(1);
|
|
|
- runTestPutKey(new PutHelper(ozoneRestClient, path,
|
|
|
+ runTestPutKey(new PutHelper(client, path,
|
|
|
getMultiPartKey(delimiter)));
|
|
|
}
|
|
|
|
|
|
static void runTestPutKey(PutHelper helper) throws Exception {
|
|
|
- final OzoneRestClient client = helper.client;
|
|
|
+ final ClientProtocol client = helper.client;
|
|
|
helper.putKey();
|
|
|
assertNotNull(helper.getBucket());
|
|
|
assertNotNull(helper.getFile());
|
|
|
- List<OzoneKey> keyList = helper.getBucket().listKeys("100", null, null);
|
|
|
+ List<OzoneKey> keyList = client
|
|
|
+ .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
|
|
|
+ null, 10);
|
|
|
Assert.assertEquals(1, keyList.size());
|
|
|
|
|
|
// 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("100", null, null);
|
|
|
+ OzoneOutputStream ozoneOutputStream = client
|
|
|
+ .createKey(helper.getVol().getName(), helper.getBucket().getName(),
|
|
|
+ newkeyName, 0, replicationType, replicationFactor);
|
|
|
+ ozoneOutputStream.close();
|
|
|
+ keyList = client
|
|
|
+ .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
|
|
|
+ null, 10);
|
|
|
Assert.assertEquals(2, keyList.size());
|
|
|
|
|
|
// test new put key with invalid volume/bucket name
|
|
|
try {
|
|
|
- client.putKey("invalid-volume",
|
|
|
- helper.getBucket().getBucketName(), newkeyName, helper.getFile());
|
|
|
+ ozoneOutputStream = client
|
|
|
+ .createKey("invalid-volume", helper.getBucket().getName(), newkeyName,
|
|
|
+ 0, replicationType, replicationFactor);
|
|
|
+ ozoneOutputStream.close();
|
|
|
fail("Put key should have thrown"
|
|
|
+ " when using invalid volume name.");
|
|
|
- } catch (OzoneException e) {
|
|
|
+ } catch (IOException e) {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
Status.VOLUME_NOT_FOUND.toString(), e);
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- client.putKey(helper.getVol().getVolumeName(), "invalid-bucket",
|
|
|
- newkeyName, helper.getFile());
|
|
|
+ ozoneOutputStream = client
|
|
|
+ .createKey(helper.getVol().getName(), "invalid-bucket", newkeyName, 0,
|
|
|
+ replicationType, replicationFactor);
|
|
|
+ ozoneOutputStream.close();
|
|
|
fail("Put key should have thrown "
|
|
|
+ "when using invalid bucket name.");
|
|
|
- } catch (OzoneException e) {
|
|
|
+ } catch (IOException e) {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
Status.BUCKET_NOT_FOUND.toString(), e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static void restartDatanode(
|
|
|
- MiniOzoneCluster cluster, int datanodeIdx, OzoneRestClient client)
|
|
|
+ private static void restartDatanode(MiniOzoneCluster cluster, int datanodeIdx)
|
|
|
throws OzoneException, URISyntaxException {
|
|
|
cluster.restartHddsDatanode(datanodeIdx);
|
|
|
- // refresh the datanode endpoint uri after datanode restart
|
|
|
- final int port = ozoneCluster.getHddsDatanodes().get(0)
|
|
|
- .getDatanodeDetails().getOzoneRestPort();
|
|
|
- client.setEndPoint(String.format("http://localhost:%d", port));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testPutAndGetKeyWithDnRestart() throws Exception {
|
|
|
runTestPutAndGetKeyWithDnRestart(
|
|
|
- new PutHelper(ozoneRestClient, path), ozoneCluster);
|
|
|
+ new PutHelper(client, path), ozoneCluster);
|
|
|
String delimiter = RandomStringUtils.randomAscii(1);
|
|
|
runTestPutAndGetKeyWithDnRestart(
|
|
|
- new PutHelper(ozoneRestClient, path,
|
|
|
+ new PutHelper(client, path,
|
|
|
getMultiPartKey(delimiter)), ozoneCluster);
|
|
|
}
|
|
|
|
|
|
static void runTestPutAndGetKeyWithDnRestart(
|
|
|
PutHelper helper, MiniOzoneCluster cluster) throws Exception {
|
|
|
- String keyName = helper.putKey().getKeyName();
|
|
|
+ String keyName = helper.putKey();
|
|
|
assertNotNull(helper.getBucket());
|
|
|
assertNotNull(helper.getFile());
|
|
|
|
|
|
// restart the datanode
|
|
|
- restartDatanode(cluster, 0, helper.client);
|
|
|
+ restartDatanode(cluster, 0);
|
|
|
+ // TODO: Try removing sleep and adding a join for the MiniOzoneCluster start
|
|
|
+ // The ozoneContainer is not started and its metrics are not initialized
|
|
|
+ // which leads to NullPointerException in Dispatcher.
|
|
|
+ Thread.sleep(1000);
|
|
|
+ ozoneCluster.waitForClusterToBeReady();
|
|
|
// verify getKey after the datanode restart
|
|
|
String newFileName = helper.dir + "/"
|
|
|
+ OzoneUtils.getRequestID().toLowerCase();
|
|
|
Path newPath = Paths.get(newFileName);
|
|
|
-
|
|
|
- helper.getBucket().getKey(keyName, newPath);
|
|
|
+ try (
|
|
|
+ FileOutputStream newOutputStream = new FileOutputStream(
|
|
|
+ newPath.toString());
|
|
|
+ OzoneInputStream ozoneInputStream = helper.client
|
|
|
+ .getKey(helper.getVol().getName(), helper.getBucket().getName(),
|
|
|
+ keyName)) {
|
|
|
+ IOUtils.copy(ozoneInputStream, newOutputStream);
|
|
|
+ }
|
|
|
|
|
|
try (
|
|
|
FileInputStream original = new FileInputStream(helper.getFile());
|
|
@@ -321,16 +365,16 @@ public class TestKeys {
|
|
|
|
|
|
@Test
|
|
|
public void testPutAndGetKey() throws Exception {
|
|
|
- runTestPutAndGetKey(new PutHelper(ozoneRestClient, path));
|
|
|
+ runTestPutAndGetKey(new PutHelper(client, path));
|
|
|
String delimiter = RandomStringUtils.randomAscii(1);
|
|
|
- runTestPutAndGetKey(new PutHelper(ozoneRestClient, path,
|
|
|
+ runTestPutAndGetKey(new PutHelper(client, path,
|
|
|
getMultiPartKey(delimiter)));
|
|
|
}
|
|
|
|
|
|
static void runTestPutAndGetKey(PutHelper helper) throws Exception {
|
|
|
- final OzoneRestClient client = helper.client;
|
|
|
+ final ClientProtocol client = helper.client;
|
|
|
|
|
|
- String keyName = helper.putKey().getKeyName();
|
|
|
+ String keyName = helper.putKey();
|
|
|
assertNotNull(helper.getBucket());
|
|
|
assertNotNull(helper.getFile());
|
|
|
|
|
@@ -342,10 +386,22 @@ public class TestKeys {
|
|
|
Path newPath1 = Paths.get(newFileName1);
|
|
|
Path newPath2 = Paths.get(newFileName2);
|
|
|
|
|
|
- helper.getBucket().getKey(keyName, newPath1);
|
|
|
+ try (
|
|
|
+ FileOutputStream newOutputStream = new FileOutputStream(
|
|
|
+ newPath1.toString());
|
|
|
+ OzoneInputStream ozoneInputStream = helper.getBucket()
|
|
|
+ .readKey(keyName)) {
|
|
|
+ IOUtils.copy(ozoneInputStream, newOutputStream);
|
|
|
+ }
|
|
|
+
|
|
|
// test get key using a more efficient call
|
|
|
- client.getKey(helper.getVol().getVolumeName(),
|
|
|
- helper.getBucket().getBucketName(), keyName, newPath2);
|
|
|
+ try (
|
|
|
+ FileOutputStream newOutputStream = new FileOutputStream(
|
|
|
+ newPath2.toString());
|
|
|
+ OzoneInputStream ozoneInputStream = helper.getBucket()
|
|
|
+ .readKey(keyName)) {
|
|
|
+ IOUtils.copy(ozoneInputStream, newOutputStream);
|
|
|
+ }
|
|
|
|
|
|
try (FileInputStream original = new FileInputStream(helper.getFile());
|
|
|
FileInputStream downloaded1 = new FileInputStream(newPath1.toFile());
|
|
@@ -363,19 +419,17 @@ public class TestKeys {
|
|
|
|
|
|
// test new get key with invalid volume/bucket name
|
|
|
try {
|
|
|
- client.getKey("invalid-volume", helper.getBucket().getBucketName(),
|
|
|
- keyName, newPath1);
|
|
|
+ client.getKey("invalid-volume", helper.getBucket().getName(), keyName);
|
|
|
fail("Get key should have thrown " + "when using invalid volume name.");
|
|
|
- } catch (OzoneException e) {
|
|
|
+ } catch (IOException e) {
|
|
|
GenericTestUtils
|
|
|
.assertExceptionContains(Status.KEY_NOT_FOUND.toString(), e);
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- client.getKey(helper.getVol().getVolumeName(), "invalid-bucket",
|
|
|
- keyName, newPath1);
|
|
|
+ client.getKey(helper.getVol().getName(), "invalid-bucket", keyName);
|
|
|
fail("Get key should have thrown " + "when using invalid bucket name.");
|
|
|
- } catch (OzoneException e) {
|
|
|
+ } catch (IOException e) {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
Status.KEY_NOT_FOUND.toString(), e);
|
|
|
}
|
|
@@ -384,14 +438,14 @@ public class TestKeys {
|
|
|
|
|
|
@Test
|
|
|
public void testPutAndDeleteKey() throws Exception {
|
|
|
- runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path));
|
|
|
+ runTestPutAndDeleteKey(new PutHelper(client, path));
|
|
|
String delimiter = RandomStringUtils.randomAscii(1);
|
|
|
- runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path,
|
|
|
+ runTestPutAndDeleteKey(new PutHelper(client, path,
|
|
|
getMultiPartKey(delimiter)));
|
|
|
}
|
|
|
|
|
|
static void runTestPutAndDeleteKey(PutHelper helper) throws Exception {
|
|
|
- String keyName = helper.putKey().getKeyName();
|
|
|
+ String keyName = helper.putKey();
|
|
|
assertNotNull(helper.getBucket());
|
|
|
assertNotNull(helper.getFile());
|
|
|
helper.getBucket().deleteKey(keyName);
|
|
@@ -399,7 +453,7 @@ public class TestKeys {
|
|
|
try {
|
|
|
helper.getBucket().getKey(keyName);
|
|
|
fail("Get Key on a deleted key should have thrown");
|
|
|
- } catch (OzoneException ex) {
|
|
|
+ } catch (IOException ex) {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
Status.KEY_NOT_FOUND.toString(), ex);
|
|
|
}
|
|
@@ -407,14 +461,14 @@ public class TestKeys {
|
|
|
|
|
|
@Test
|
|
|
public void testPutAndListKey() throws Exception {
|
|
|
- runTestPutAndListKey(new PutHelper(ozoneRestClient, path));
|
|
|
+ runTestPutAndListKey(new PutHelper(client, path));
|
|
|
String delimiter = RandomStringUtils.randomAscii(1);
|
|
|
- runTestPutAndListKey(new PutHelper(ozoneRestClient, path,
|
|
|
+ runTestPutAndListKey(new PutHelper(client, path,
|
|
|
getMultiPartKey(delimiter)));
|
|
|
}
|
|
|
|
|
|
static void runTestPutAndListKey(PutHelper helper) throws Exception {
|
|
|
- final OzoneRestClient client = helper.client;
|
|
|
+ ClientProtocol client = helper.client;
|
|
|
helper.putKey();
|
|
|
assertNotNull(helper.getBucket());
|
|
|
assertNotNull(helper.getFile());
|
|
@@ -422,13 +476,20 @@ public class TestKeys {
|
|
|
// add keys [list-key0, list-key1, ..., list-key9]
|
|
|
for (int x = 0; x < 10; x++) {
|
|
|
String newkeyName = "list-key" + x;
|
|
|
- helper.getBucket().putKey(newkeyName, helper.getFile());
|
|
|
+ try (
|
|
|
+ OzoneOutputStream ozoneOutputStream = helper.getBucket()
|
|
|
+ .createKey(newkeyName, 0, replicationType, replicationFactor);
|
|
|
+ InputStream fileInputStream = new FileInputStream(helper.getFile())) {
|
|
|
+ IOUtils.copy(fileInputStream, ozoneOutputStream);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- List<OzoneKey> keyList1 = helper.getBucket().listKeys("100", null, null);
|
|
|
+ List<OzoneKey> keyList1 =
|
|
|
+ IteratorUtils.toList(helper.getBucket().listKeys(null, null));
|
|
|
// test list key using a more efficient call
|
|
|
- List<OzoneKey> keyList2 = client.listKeys(helper.getVol().getVolumeName(),
|
|
|
- helper.getBucket().getBucketName(), "100", null, null);
|
|
|
+ List<OzoneKey> keyList2 = client
|
|
|
+ .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
|
|
|
+ null, 100);
|
|
|
|
|
|
Assert.assertEquals(11, keyList1.size());
|
|
|
Assert.assertEquals(11, keyList2.size());
|
|
@@ -436,57 +497,56 @@ public class TestKeys {
|
|
|
// second unit since the date string reparsed to millisecond will
|
|
|
// lose precision.
|
|
|
for (OzoneKey key : keyList1) {
|
|
|
- assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getCreatedOn())
|
|
|
- / 1000) >= (currentTime / 1000));
|
|
|
- assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getModifiedOn())
|
|
|
- / 1000) >= (currentTime / 1000));
|
|
|
+ assertTrue((key.getCreationTime() / 1000) >= (currentTime / 1000));
|
|
|
+ assertTrue((key.getModificationTime() / 1000) >= (currentTime / 1000));
|
|
|
}
|
|
|
|
|
|
for (OzoneKey key : keyList2) {
|
|
|
- assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getCreatedOn())
|
|
|
- / 1000) >= (currentTime / 1000));
|
|
|
- assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getModifiedOn())
|
|
|
- / 1000) >= (currentTime / 1000));
|
|
|
+ assertTrue((key.getCreationTime() / 1000) >= (currentTime / 1000));
|
|
|
+ assertTrue((key.getModificationTime() / 1000) >= (currentTime / 1000));
|
|
|
}
|
|
|
|
|
|
// test maxLength parameter of list keys
|
|
|
- keyList1 = helper.getBucket().listKeys("1", null, null);
|
|
|
- keyList2 = client.listKeys(helper.getVol().getVolumeName(),
|
|
|
- helper.getBucket().getBucketName(), "1", null, null);
|
|
|
- Assert.assertEquals(1, keyList1.size());
|
|
|
+ keyList2 = client
|
|
|
+ .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
|
|
|
+ null, 1);
|
|
|
Assert.assertEquals(1, keyList2.size());
|
|
|
|
|
|
// test startKey parameter of list keys
|
|
|
- keyList1 = helper.getBucket().listKeys("100", "list-key4", "list-key");
|
|
|
- keyList2 = client.listKeys(helper.getVol().getVolumeName(),
|
|
|
- helper.getBucket().getBucketName(), "100", "list-key4", "list-key");
|
|
|
+ keyList1 = IteratorUtils
|
|
|
+ .toList(helper.getBucket().listKeys("list-key", "list-key4"));
|
|
|
+ keyList2 = client
|
|
|
+ .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
|
|
|
+ "list-key", "list-key4", 100);
|
|
|
Assert.assertEquals(5, keyList1.size());
|
|
|
Assert.assertEquals(5, keyList2.size());
|
|
|
|
|
|
// test prefix parameter of list keys
|
|
|
- keyList1 = helper.getBucket().listKeys("100", null, "list-key2");
|
|
|
- keyList2 = client.listKeys(helper.getVol().getVolumeName(),
|
|
|
- helper.getBucket().getBucketName(), "100", null, "list-key2");
|
|
|
- Assert.assertTrue(keyList1.size() == 1
|
|
|
- && keyList1.get(0).getObjectInfo().getKeyName().equals("list-key2"));
|
|
|
- Assert.assertTrue(keyList2.size() == 1
|
|
|
- && keyList2.get(0).getObjectInfo().getKeyName().equals("list-key2"));
|
|
|
+ keyList1 =
|
|
|
+ IteratorUtils.toList(helper.getBucket().listKeys("list-key2", null));
|
|
|
+ keyList2 = client
|
|
|
+ .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
|
|
|
+ "list-key2", null, 100);
|
|
|
+ Assert.assertTrue(
|
|
|
+ keyList1.size() == 1 && keyList1.get(0).getName().equals("list-key2"));
|
|
|
+ Assert.assertTrue(
|
|
|
+ keyList2.size() == 1 && keyList2.get(0).getName().equals("list-key2"));
|
|
|
|
|
|
// test new list keys with invalid volume/bucket name
|
|
|
try {
|
|
|
- client.listKeys("invalid-volume", helper.getBucket().getBucketName(),
|
|
|
- "100", null, null);
|
|
|
+ client.listKeys("invalid-volume", helper.getBucket().getName(),
|
|
|
+ null, null, 100);
|
|
|
fail("List keys should have thrown when using invalid volume name.");
|
|
|
- } catch (OzoneException e) {
|
|
|
+ } catch (IOException e) {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
Status.BUCKET_NOT_FOUND.toString(), e);
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- client.listKeys(helper.getVol().getVolumeName(), "invalid-bucket", "100",
|
|
|
- null, null);
|
|
|
+ client.listKeys(helper.getVol().getName(), "invalid-bucket", null,
|
|
|
+ null, 100);
|
|
|
fail("List keys should have thrown when using invalid bucket name.");
|
|
|
- } catch (OzoneException e) {
|
|
|
+ } catch (IOException e) {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
Status.BUCKET_NOT_FOUND.toString(), e);
|
|
|
}
|
|
@@ -494,29 +554,27 @@ public class TestKeys {
|
|
|
|
|
|
@Test
|
|
|
public void testGetKeyInfo() throws Exception {
|
|
|
- runTestGetKeyInfo(new PutHelper(ozoneRestClient, path));
|
|
|
+ runTestGetKeyInfo(new PutHelper(client, path));
|
|
|
String delimiter = RandomStringUtils.randomAscii(1);
|
|
|
- runTestGetKeyInfo(new PutHelper(ozoneRestClient, path,
|
|
|
+ runTestGetKeyInfo(new PutHelper(client, path,
|
|
|
getMultiPartKey(delimiter)));
|
|
|
}
|
|
|
|
|
|
static void runTestGetKeyInfo(PutHelper helper) throws Exception {
|
|
|
- String keyName = helper.putKey().getKeyName();
|
|
|
+ String keyName = helper.putKey();
|
|
|
assertNotNull(helper.getBucket());
|
|
|
assertNotNull(helper.getFile());
|
|
|
|
|
|
- OzoneKey keyInfo = helper.getBucket().getKeyInfo(keyName);
|
|
|
- assertNotNull(keyInfo.getObjectInfo());
|
|
|
- assertEquals(keyName, keyInfo.getObjectInfo().getKeyName());
|
|
|
+ OzoneKey keyInfo = helper.getBucket().getKey(keyName);
|
|
|
+ assertNotNull(keyInfo);
|
|
|
+ assertEquals(keyName, keyInfo.getName());
|
|
|
|
|
|
// Compare the time in second unit since the date string reparsed to
|
|
|
// millisecond will lose precision.
|
|
|
+ Assert
|
|
|
+ .assertTrue((keyInfo.getCreationTime() / 1000) >= (currentTime / 1000));
|
|
|
Assert.assertTrue(
|
|
|
- (OzoneUtils.formatDate(keyInfo.getObjectInfo().getCreatedOn())
|
|
|
- / 1000) >= (currentTime / 1000));
|
|
|
- Assert.assertTrue(
|
|
|
- (OzoneUtils.formatDate(keyInfo.getObjectInfo().getModifiedOn())
|
|
|
- / 1000) >= (currentTime / 1000));
|
|
|
+ (keyInfo.getModificationTime() / 1000) >= (currentTime / 1000));
|
|
|
}
|
|
|
|
|
|
// Volume, bucket, keys info that helps for test create/delete keys.
|
|
@@ -593,12 +651,12 @@ public class TestKeys {
|
|
|
int numOfExistedKeys = countKsmKeys(ksm);
|
|
|
|
|
|
// Keep tracking bucket keys info while creating them
|
|
|
- PutHelper helper = new PutHelper(ozoneRestClient, path);
|
|
|
+ PutHelper helper = new PutHelper(client, path);
|
|
|
BucketKeys bucketKeys = new BucketKeys();
|
|
|
for (int i = 0; i < 20; i++) {
|
|
|
- KsmKeyArgs keyArgs = helper.putKey();
|
|
|
- bucketKeys.addKey(keyArgs.getVolumeName(), keyArgs.getBucketName(),
|
|
|
- keyArgs.getKeyName());
|
|
|
+ String keyName = helper.putKey();
|
|
|
+ bucketKeys.addKey(helper.getVol().getName(), helper.getBucket().getName(),
|
|
|
+ keyName);
|
|
|
}
|
|
|
|
|
|
// There should be 20 keys in the buckets we just created.
|