diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java index 47922cb8dc..9d652b461a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java @@ -18,87 +18,17 @@ package org.apache.hadoop.ozone.client.rpc; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.RandomStringUtils; -import org.apache.commons.lang3.RandomUtils; -import org.apache.hadoop.hdds.protocol.StorageType; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.scm.container.ContainerID; -import org.apache.hadoop.hdds.scm.container.ContainerInfo; -import org.apache.hadoop.hdds.scm.XceiverClientManager; -import org.apache.hadoop.hdds.scm.XceiverClientRatis; -import org.apache.hadoop.hdds.scm.XceiverClientSpi; - -import org.apache.hadoop.hdds.scm.pipeline.Pipeline; -import org.apache.hadoop.hdfs.DFSUtil; -import org.apache.hadoop.ozone.*; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.client.*; -import org.apache.hadoop.hdds.client.OzoneQuota; -import org.apache.hadoop.hdds.client.ReplicationFactor; -import org.apache.hadoop.hdds.client.ReplicationType; -import org.apache.hadoop.ozone.client.VolumeArgs; -import org.apache.hadoop.ozone.client.io.KeyOutputStream; -import org.apache.hadoop.ozone.client.io.OzoneInputStream; -import org.apache.hadoop.ozone.client.io.OzoneOutputStream; -import org.apache.hadoop.ozone.common.OzoneChecksumException; -import org.apache.hadoop.ozone.container.common.helpers.BlockData; -import org.apache.hadoop.ozone.container.common.interfaces.Container; -import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator; -import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; -import org.apache.hadoop.ozone.container.keyvalue.helpers - .KeyValueContainerLocationUtil; -import org.apache.hadoop.ozone.om.OzoneManager; -import org.apache.hadoop.ozone.om.helpers.*; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.ozone.client.rest.OzoneException; import org.apache.hadoop.hdds.scm.ScmConfigKeys; -import org.apache.hadoop.hdds.scm.protocolPB. - StorageContainerLocationProtocolClientSideTranslatorPB; -import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.util.Time; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.UUID; - -import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.CoreMatchers.either; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; /** * This class is to test all the public facing APIs of Ozone Client. */ -public class TestOzoneRpcClient { - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static MiniOzoneCluster cluster = null; - private static OzoneClient ozClient = null; - private static ObjectStore store = null; - private static OzoneManager ozoneManager; - private static StorageContainerLocationProtocolClientSideTranslatorPB - storageContainerLocationClient; - - private static final String SCM_ID = UUID.randomUUID().toString(); +public class TestOzoneRpcClient extends TestOzoneRpcClientAbstract { /** * Create a MiniOzoneCluster for testing. @@ -111,1744 +41,14 @@ public class TestOzoneRpcClient { public static void init() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 1); - cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(10) - .setScmId(SCM_ID) - .build(); - cluster.waitForClusterToBeReady(); - ozClient = OzoneClientFactory.getRpcClient(conf); - store = ozClient.getObjectStore(); - storageContainerLocationClient = - cluster.getStorageContainerLocationClient(); - ozoneManager = cluster.getOzoneManager(); + startCluster(conf); } - @Test - public void testCreateVolume() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - Assert.assertEquals(volumeName, volume.getName()); - } - - @Test - public void testCreateVolumeWithOwner() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - VolumeArgs.Builder argsBuilder = VolumeArgs.newBuilder(); - argsBuilder.setOwner("test"); - store.createVolume(volumeName, argsBuilder.build()); - OzoneVolume volume = store.getVolume(volumeName); - Assert.assertEquals(volumeName, volume.getName()); - Assert.assertEquals("test", volume.getOwner()); - } - - @Test - public void testCreateVolumeWithQuota() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - VolumeArgs.Builder argsBuilder = VolumeArgs.newBuilder(); - argsBuilder.setOwner("test").setQuota("1000000000 BYTES"); - store.createVolume(volumeName, argsBuilder.build()); - OzoneVolume volume = store.getVolume(volumeName); - Assert.assertEquals(volumeName, volume.getName()); - Assert.assertEquals("test", volume.getOwner()); - Assert.assertEquals(1000000000L, volume.getQuota()); - } - - @Test - public void testInvalidVolumeCreation() throws IOException { - thrown.expectMessage("Bucket or Volume name has an unsupported" + - " character : #"); - String volumeName = "invalid#name"; - store.createVolume(volumeName); - } - - @Test - public void testVolumeAlreadyExist() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - try { - store.createVolume(volumeName); - } catch (IOException ex) { - Assert.assertEquals( - "Volume creation failed, error:VOLUME_ALREADY_EXISTS", - ex.getMessage()); - } - } - - @Test - public void testSetVolumeOwner() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - store.getVolume(volumeName).setOwner("test"); - OzoneVolume volume = store.getVolume(volumeName); - Assert.assertEquals("test", volume.getOwner()); - } - - @Test - public void testSetVolumeQuota() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - store.getVolume(volumeName).setQuota( - OzoneQuota.parseQuota("100000000 BYTES")); - OzoneVolume volume = store.getVolume(volumeName); - Assert.assertEquals(100000000L, volume.getQuota()); - } - - @Test - public void testDeleteVolume() - throws IOException, OzoneException { - thrown.expectMessage("Info Volume failed, error"); - String volumeName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - Assert.assertNotNull(volume); - store.deleteVolume(volumeName); - store.getVolume(volumeName); - } - - @Test - public void testCreateBucket() - throws IOException, OzoneException { - long currentTime = Time.now(); - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, bucket.getName()); - Assert.assertTrue(bucket.getCreationTime() >= currentTime); - Assert.assertTrue(volume.getCreationTime() >= currentTime); - } - - @Test - public void testCreateS3Bucket() - throws IOException, OzoneException { - long currentTime = Time.now(); - String userName = "ozone"; - String bucketName = UUID.randomUUID().toString(); - store.createS3Bucket(userName, bucketName); - String volumeName = store.getOzoneVolumeName(bucketName); - OzoneVolume volume = store.getVolume(volumeName); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, bucket.getName()); - Assert.assertTrue(bucket.getCreationTime() >= currentTime); - Assert.assertTrue(volume.getCreationTime() >= currentTime); - } - - - @Test - public void testListS3Buckets() - throws IOException, OzoneException { - String userName = "ozone100"; - String bucketName1 = UUID.randomUUID().toString(); - String bucketName2 = UUID.randomUUID().toString(); - store.createS3Bucket(userName, bucketName1); - store.createS3Bucket(userName, bucketName2); - Iterator iterator = store.listS3Buckets(userName, - null); - - while (iterator.hasNext()) { - assertThat(iterator.next().getName(), either(containsString(bucketName1)) - .or(containsString(bucketName2))); - } - - } - - @Test - public void testListS3BucketsFail() - throws IOException, OzoneException { - String userName = "randomUser"; - Iterator iterator = store.listS3Buckets(userName, - null); - - Assert.assertFalse(iterator.hasNext()); - - } - - @Test - public void testDeleteS3Bucket() - throws IOException, OzoneException { - long currentTime = Time.now(); - String userName = "ozone1"; - String bucketName = UUID.randomUUID().toString(); - store.createS3Bucket(userName, bucketName); - String volumeName = store.getOzoneVolumeName(bucketName); - OzoneVolume volume = store.getVolume(volumeName); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, bucket.getName()); - Assert.assertTrue(bucket.getCreationTime() >= currentTime); - Assert.assertTrue(volume.getCreationTime() >= currentTime); - store.deleteS3Bucket(bucketName); - thrown.expect(IOException.class); - store.getOzoneVolumeName(bucketName); - } - - @Test - public void testDeleteS3NonExistingBucket() { - try { - store.deleteS3Bucket(UUID.randomUUID().toString()); - } catch (IOException ex) { - GenericTestUtils.assertExceptionContains("NOT_FOUND", ex); - } - } - - @Test - public void testCreateS3BucketMapping() - throws IOException, OzoneException { - long currentTime = Time.now(); - String userName = "ozone"; - String bucketName = UUID.randomUUID().toString(); - store.createS3Bucket(userName, bucketName); - String volumeName = store.getOzoneVolumeName(bucketName); - OzoneVolume volume = store.getVolume(volumeName); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, bucket.getName()); - - String mapping = store.getOzoneBucketMapping(bucketName); - Assert.assertEquals("s3"+userName+"/"+bucketName, mapping); - Assert.assertEquals(bucketName, store.getOzoneBucketName(bucketName)); - Assert.assertEquals("s3"+userName, store.getOzoneVolumeName(bucketName)); - - } - - @Test - public void testCreateBucketWithVersioning() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - BucketArgs.Builder builder = BucketArgs.newBuilder(); - builder.setVersioning(true); - volume.createBucket(bucketName, builder.build()); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, bucket.getName()); - Assert.assertEquals(true, bucket.getVersioning()); - } - - @Test - public void testCreateBucketWithStorageType() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - BucketArgs.Builder builder = BucketArgs.newBuilder(); - builder.setStorageType(StorageType.SSD); - volume.createBucket(bucketName, builder.build()); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, bucket.getName()); - Assert.assertEquals(StorageType.SSD, bucket.getStorageType()); - } - - @Test - public void testCreateBucketWithAcls() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test", - OzoneAcl.OzoneACLRights.READ_WRITE); - List acls = new ArrayList<>(); - acls.add(userAcl); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - BucketArgs.Builder builder = BucketArgs.newBuilder(); - builder.setAcls(acls); - volume.createBucket(bucketName, builder.build()); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, bucket.getName()); - Assert.assertTrue(bucket.getAcls().contains(userAcl)); - } - - @Test - public void testCreateBucketWithAllArgument() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test", - OzoneAcl.OzoneACLRights.READ_WRITE); - List acls = new ArrayList<>(); - acls.add(userAcl); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - BucketArgs.Builder builder = BucketArgs.newBuilder(); - builder.setVersioning(true) - .setStorageType(StorageType.SSD) - .setAcls(acls); - volume.createBucket(bucketName, builder.build()); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, bucket.getName()); - Assert.assertEquals(true, bucket.getVersioning()); - Assert.assertEquals(StorageType.SSD, bucket.getStorageType()); - Assert.assertTrue(bucket.getAcls().contains(userAcl)); - } - - @Test - public void testInvalidBucketCreation() throws IOException { - thrown.expectMessage("Bucket or Volume name has an unsupported" + - " character : #"); - String volumeName = UUID.randomUUID().toString(); - String bucketName = "invalid#bucket"; - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - } - - @Test - public void testAddBucketAcl() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - List acls = new ArrayList<>(); - acls.add(new OzoneAcl( - OzoneAcl.OzoneACLType.USER, "test", - OzoneAcl.OzoneACLRights.READ_WRITE)); - OzoneBucket bucket = volume.getBucket(bucketName); - bucket.addAcls(acls); - OzoneBucket newBucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, newBucket.getName()); - Assert.assertTrue(bucket.getAcls().contains(acls.get(0))); - } - - @Test - public void testRemoveBucketAcl() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test", - OzoneAcl.OzoneACLRights.READ_WRITE); - List acls = new ArrayList<>(); - acls.add(userAcl); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - BucketArgs.Builder builder = BucketArgs.newBuilder(); - builder.setAcls(acls); - volume.createBucket(bucketName, builder.build()); - OzoneBucket bucket = volume.getBucket(bucketName); - bucket.removeAcls(acls); - OzoneBucket newBucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, newBucket.getName()); - Assert.assertTrue(!bucket.getAcls().contains(acls.get(0))); - } - - @Test - public void testSetBucketVersioning() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - bucket.setVersioning(true); - OzoneBucket newBucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, newBucket.getName()); - Assert.assertEquals(true, newBucket.getVersioning()); - } - - @Test - public void testSetBucketStorageType() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - bucket.setStorageType(StorageType.SSD); - OzoneBucket newBucket = volume.getBucket(bucketName); - Assert.assertEquals(bucketName, newBucket.getName()); - Assert.assertEquals(StorageType.SSD, newBucket.getStorageType()); - } - - - @Test - public void testDeleteBucket() - throws IOException, OzoneException { - thrown.expectMessage("Info Bucket failed, error"); - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - Assert.assertNotNull(bucket); - volume.deleteBucket(bucketName); - volume.getBucket(bucketName); - } - - private boolean verifyRatisReplication(String volumeName, String bucketName, - String keyName, ReplicationType type, ReplicationFactor factor) - throws IOException { - OmKeyArgs keyArgs = new OmKeyArgs.Builder() - .setVolumeName(volumeName) - .setBucketName(bucketName) - .setKeyName(keyName) - .build(); - HddsProtos.ReplicationType replicationType = - HddsProtos.ReplicationType.valueOf(type.toString()); - HddsProtos.ReplicationFactor replicationFactor = - HddsProtos.ReplicationFactor.valueOf(factor.getValue()); - OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs); - for (OmKeyLocationInfo info: - keyInfo.getLatestVersionLocations().getLocationList()) { - ContainerInfo container = - storageContainerLocationClient.getContainer(info.getContainerID()); - if (!container.getReplicationFactor().equals(replicationFactor) || ( - container.getReplicationType() != replicationType)) { - return false; - } - } - return true; - } - - @Test - public void testPutKey() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - long currentTime = Time.now(); - - String value = "sample value"; - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - for (int i = 0; i < 10; i++) { - String keyName = UUID.randomUUID().toString(); - - OzoneOutputStream out = bucket.createKey(keyName, - value.getBytes().length, ReplicationType.STAND_ALONE, - ReplicationFactor.ONE); - out.write(value.getBytes()); - out.close(); - OzoneKey key = bucket.getKey(keyName); - Assert.assertEquals(keyName, key.getName()); - OzoneInputStream is = bucket.readKey(keyName); - byte[] fileContent = new byte[value.getBytes().length]; - is.read(fileContent); - Assert.assertTrue(verifyRatisReplication(volumeName, bucketName, - keyName, ReplicationType.STAND_ALONE, - ReplicationFactor.ONE)); - Assert.assertEquals(value, new String(fileContent)); - Assert.assertTrue(key.getCreationTime() >= currentTime); - Assert.assertTrue(key.getModificationTime() >= currentTime); - } - } - - @Test - public void testValidateBlockLengthWithCommitKey() throws IOException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - - String value = RandomStringUtils.random(RandomUtils.nextInt(0, 1024)); - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - String keyName = UUID.randomUUID().toString(); - - // create the initial key with size 0, write will allocate the first block. - OzoneOutputStream out = bucket.createKey(keyName, 0, - ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - out.write(value.getBytes()); - out.close(); - OmKeyArgs.Builder builder = new OmKeyArgs.Builder(); - builder.setVolumeName(volumeName).setBucketName(bucketName) - .setKeyName(keyName); - OmKeyInfo keyInfo = ozoneManager.lookupKey(builder.build()); - - List locationInfoList = - keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly(); - // LocationList should have only 1 block - Assert.assertEquals(1, locationInfoList.size()); - // make sure the data block size is updated - Assert.assertEquals(value.getBytes().length, - locationInfoList.get(0).getLength()); - // make sure the total data size is set correctly - Assert.assertEquals(value.getBytes().length, keyInfo.getDataSize()); - } - - - @Test - public void testPutKeyRatisOneNode() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - long currentTime = Time.now(); - - String value = "sample value"; - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - for (int i = 0; i < 10; i++) { - String keyName = UUID.randomUUID().toString(); - - OzoneOutputStream out = bucket.createKey(keyName, - value.getBytes().length, ReplicationType.RATIS, - ReplicationFactor.ONE); - out.write(value.getBytes()); - out.close(); - OzoneKey key = bucket.getKey(keyName); - Assert.assertEquals(keyName, key.getName()); - OzoneInputStream is = bucket.readKey(keyName); - byte[] fileContent = new byte[value.getBytes().length]; - is.read(fileContent); - is.close(); - Assert.assertTrue(verifyRatisReplication(volumeName, bucketName, - keyName, ReplicationType.RATIS, ReplicationFactor.ONE)); - Assert.assertEquals(value, new String(fileContent)); - Assert.assertTrue(key.getCreationTime() >= currentTime); - Assert.assertTrue(key.getModificationTime() >= currentTime); - } - } - - @Test - public void testPutKeyRatisThreeNodes() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - long currentTime = Time.now(); - - String value = "sample value"; - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - for (int i = 0; i < 10; i++) { - String keyName = UUID.randomUUID().toString(); - - OzoneOutputStream out = bucket.createKey(keyName, - value.getBytes().length, ReplicationType.RATIS, - ReplicationFactor.THREE); - out.write(value.getBytes()); - out.close(); - OzoneKey key = bucket.getKey(keyName); - Assert.assertEquals(keyName, key.getName()); - OzoneInputStream is = bucket.readKey(keyName); - byte[] fileContent = new byte[value.getBytes().length]; - is.read(fileContent); - is.close(); - Assert.assertTrue(verifyRatisReplication(volumeName, bucketName, - keyName, ReplicationType.RATIS, - ReplicationFactor.THREE)); - Assert.assertEquals(value, new String(fileContent)); - Assert.assertTrue(key.getCreationTime() >= currentTime); - Assert.assertTrue(key.getModificationTime() >= currentTime); - } - } - - @Test - public void testPutKeyAndGetKeyThreeNodes() - throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - - String value = "sample value"; - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - String keyName = UUID.randomUUID().toString(); - - OzoneOutputStream out = bucket - .createKey(keyName, value.getBytes().length, ReplicationType.RATIS, - ReplicationFactor.THREE); - KeyOutputStream groupOutputStream = - (KeyOutputStream) out.getOutputStream(); - XceiverClientManager manager = groupOutputStream.getXceiverClientManager(); - out.write(value.getBytes()); - out.close(); - // First, confirm the key info from the client matches the info in OM. - OmKeyArgs.Builder builder = new OmKeyArgs.Builder(); - builder.setVolumeName(volumeName).setBucketName(bucketName) - .setKeyName(keyName); - OmKeyLocationInfo keyInfo = ozoneManager.lookupKey(builder.build()). - getKeyLocationVersions().get(0).getBlocksLatestVersionOnly().get(0); - long containerID = keyInfo.getContainerID(); - long localID = keyInfo.getLocalID(); - OzoneKeyDetails keyDetails = bucket.getKey(keyName); - Assert.assertEquals(keyName, keyDetails.getName()); - - List keyLocations = keyDetails.getOzoneKeyLocations(); - Assert.assertEquals(1, keyLocations.size()); - Assert.assertEquals(containerID, keyLocations.get(0).getContainerID()); - Assert.assertEquals(localID, keyLocations.get(0).getLocalID()); - - // Make sure that the data size matched. - Assert - .assertEquals(value.getBytes().length, keyLocations.get(0).getLength()); - - ContainerInfo container = cluster.getStorageContainerManager() - .getContainerManager().getContainer(ContainerID.valueof(containerID)); - Pipeline pipeline = cluster.getStorageContainerManager() - .getPipelineManager().getPipeline(container.getPipelineID()); - List datanodes = pipeline.getNodes(); - - DatanodeDetails datanodeDetails = datanodes.get(0); - Assert.assertNotNull(datanodeDetails); - - XceiverClientSpi clientSpi = manager.acquireClient(pipeline); - Assert.assertTrue(clientSpi instanceof XceiverClientRatis); - XceiverClientRatis ratisClient = (XceiverClientRatis)clientSpi; - - ratisClient.watchForCommit(keyInfo.getBlockCommitSequenceId(), 5000); - // shutdown the datanode - cluster.shutdownHddsDatanode(datanodeDetails); - - Assert.assertTrue(container.getState() - == HddsProtos.LifeCycleState.OPEN); - // try to read, this shouls be successful - readKey(bucket, keyName, value); - - Assert.assertTrue(container.getState() - == HddsProtos.LifeCycleState.OPEN); - // shutdown the second datanode - datanodeDetails = datanodes.get(1); - cluster.shutdownHddsDatanode(datanodeDetails); - Assert.assertTrue(container.getState() - == HddsProtos.LifeCycleState.OPEN); - - // the container is open and with loss of 2 nodes we still should be able - // to read via Standalone protocol - // try to read - readKey(bucket, keyName, value); - - // shutdown the 3rd datanode - datanodeDetails = datanodes.get(2); - cluster.shutdownHddsDatanode(datanodeDetails); - try { - // try to read - readKey(bucket, keyName, value); - fail("Expected exception not thrown"); - } catch (IOException e) { - Assert.assertTrue(e.getMessage().contains("Failed to execute command")); - Assert.assertTrue( - e.getMessage().contains("on the pipeline " + pipeline.getId())); - } - manager.releaseClient(clientSpi); - } - - private void readKey(OzoneBucket bucket, String keyName, String data) - throws IOException { - OzoneKey key = bucket.getKey(keyName); - Assert.assertEquals(keyName, key.getName()); - OzoneInputStream is = bucket.readKey(keyName); - byte[] fileContent = new byte[data.getBytes().length]; - is.read(fileContent); - is.close(); - } - - @Test - public void testGetKeyDetails() throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - String keyName = UUID.randomUUID().toString(); - String keyValue = RandomStringUtils.random(128); - //String keyValue = "this is a test value.glx"; - // create the initial key with size 0, write will allocate the first block. - OzoneOutputStream out = bucket.createKey(keyName, - keyValue.getBytes().length, ReplicationType.STAND_ALONE, - ReplicationFactor.ONE); - out.write(keyValue.getBytes()); - out.close(); - - OzoneInputStream is = bucket.readKey(keyName); - byte[] fileContent = new byte[32]; - is.read(fileContent); - - // First, confirm the key info from the client matches the info in OM. - OmKeyArgs.Builder builder = new OmKeyArgs.Builder(); - builder.setVolumeName(volumeName).setBucketName(bucketName) - .setKeyName(keyName); - OmKeyLocationInfo keyInfo = ozoneManager.lookupKey(builder.build()). - getKeyLocationVersions().get(0).getBlocksLatestVersionOnly().get(0); - long containerID = keyInfo.getContainerID(); - long localID = keyInfo.getLocalID(); - OzoneKeyDetails keyDetails = (OzoneKeyDetails)bucket.getKey(keyName); - Assert.assertEquals(keyName, keyDetails.getName()); - - List keyLocations = keyDetails.getOzoneKeyLocations(); - Assert.assertEquals(1, keyLocations.size()); - Assert.assertEquals(containerID, keyLocations.get(0).getContainerID()); - Assert.assertEquals(localID, keyLocations.get(0).getLocalID()); - - // Make sure that the data size matched. - Assert.assertEquals(keyValue.getBytes().length, - keyLocations.get(0).getLength()); - - // Second, sum the data size from chunks in Container via containerID - // and localID, make sure the size equals to the size from keyDetails. - ContainerInfo container = cluster.getStorageContainerManager() - .getContainerManager().getContainer(ContainerID.valueof(containerID)); - Pipeline pipeline = cluster.getStorageContainerManager() - .getPipelineManager().getPipeline(container.getPipelineID()); - List datanodes = pipeline.getNodes(); - Assert.assertEquals(datanodes.size(), 1); - - DatanodeDetails datanodeDetails = datanodes.get(0); - Assert.assertNotNull(datanodeDetails); - HddsDatanodeService datanodeService = null; - for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) { - if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) { - datanodeService = datanodeServiceItr; - break; - } - } - KeyValueContainerData containerData = - (KeyValueContainerData)(datanodeService.getDatanodeStateMachine() - .getContainer().getContainerSet().getContainer(containerID) - .getContainerData()); - String containerPath = new File(containerData.getMetadataPath()) - .getParent(); - KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( - containerID, new File(containerPath)); - while (keyValueBlockIterator.hasNext()) { - BlockData blockData = keyValueBlockIterator.nextBlock(); - if (blockData.getBlockID().getLocalID() == localID) { - long length = 0; - List chunks = blockData.getChunks(); - for (ContainerProtos.ChunkInfo chunk : chunks) { - length += chunk.getLen(); - } - Assert.assertEquals(length, keyValue.getBytes().length); - break; - } - } - } - - /** - * Tests reading a corrputed chunk file throws checksum exception. - * @throws IOException - */ - @Test - public void testReadKeyWithCorruptedData() throws IOException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - - String value = "sample value"; - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - String keyName = UUID.randomUUID().toString(); - - // Write data into a key - OzoneOutputStream out = bucket.createKey(keyName, - value.getBytes().length, ReplicationType.STAND_ALONE, - ReplicationFactor.ONE); - out.write(value.getBytes()); - out.close(); - - // We need to find the location of the chunk file corresponding to the - // data we just wrote. - OzoneKey key = bucket.getKey(keyName); - long containerID = ((OzoneKeyDetails) key).getOzoneKeyLocations().get(0) - .getContainerID(); - long localID = ((OzoneKeyDetails) key).getOzoneKeyLocations().get(0) - .getLocalID(); - - // Get the container by traversing the datanodes. Atleast one of the - // datanode must have this container. - Container container = null; - for (HddsDatanodeService hddsDatanode : cluster.getHddsDatanodes()) { - container = hddsDatanode.getDatanodeStateMachine().getContainer() - .getContainerSet().getContainer(containerID); - if (container != null) { - break; - } - } - Assert.assertNotNull("Container not found", container); - - // From the containerData, get the block iterator for all the blocks in - // the container. - KeyValueContainerData containerData = - (KeyValueContainerData) container.getContainerData(); - String containerPath = new File(containerData.getMetadataPath()) - .getParent(); - KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( - containerID, new File(containerPath)); - - // Find the block corresponding to the key we put. We use the localID of - // the BlockData to identify out key. - BlockData blockData = null; - while (keyValueBlockIterator.hasNext()) { - blockData = keyValueBlockIterator.nextBlock(); - if (blockData.getBlockID().getLocalID() == localID) { - break; - } - } - Assert.assertNotNull("Block not found", blockData); - - // Get the location of the chunk file - String chunkName = blockData.getChunks().get(0).getChunkName(); - String containreBaseDir = container.getContainerData().getVolume() - .getHddsRootDir().getPath(); - File chunksLocationPath = KeyValueContainerLocationUtil - .getChunksLocationPath(containreBaseDir, SCM_ID, containerID); - File chunkFile = new File(chunksLocationPath, chunkName); - - // Corrupt the contents of the chunk file - String newData = new String("corrupted data"); - FileUtils.writeByteArrayToFile(chunkFile, newData.getBytes()); - - // Try reading the key. Since the chunk file is corrupted, it should - // throw a checksum mismatch exception. - try { - OzoneInputStream is = bucket.readKey(keyName); - is.read(new byte[100]); - fail("Reading corrupted data should fail."); - } catch (OzoneChecksumException e) { - GenericTestUtils.assertExceptionContains("Checksum mismatch", e); - } - } - - @Test - public void testDeleteKey() - throws IOException, OzoneException { - thrown.expectMessage("Lookup key failed, error"); - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - String value = "sample value"; - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - OzoneOutputStream out = bucket.createKey(keyName, - value.getBytes().length, ReplicationType.STAND_ALONE, - ReplicationFactor.ONE); - out.write(value.getBytes()); - out.close(); - OzoneKey key = bucket.getKey(keyName); - Assert.assertEquals(keyName, key.getName()); - bucket.deleteKey(keyName); - bucket.getKey(keyName); - } - - @Test - public void testRenameKey() - throws IOException, OzoneException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String fromKeyName = UUID.randomUUID().toString(); - String value = "sample value"; - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - OzoneOutputStream out = bucket.createKey(fromKeyName, - value.getBytes().length, ReplicationType.STAND_ALONE, - ReplicationFactor.ONE); - out.write(value.getBytes()); - out.close(); - OzoneKey key = bucket.getKey(fromKeyName); - Assert.assertEquals(fromKeyName, key.getName()); - - // Rename to empty string should fail. - IOException ioe = null; - String toKeyName = ""; - try { - bucket.renameKey(fromKeyName, toKeyName); - } catch (IOException e) { - ioe = e; - } - Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error")); - - toKeyName = UUID.randomUUID().toString(); - bucket.renameKey(fromKeyName, toKeyName); - - // Lookup for old key should fail. - try { - bucket.getKey(fromKeyName); - } catch (IOException e) { - ioe = e; - } - Assert.assertTrue(ioe.getMessage().contains("Lookup key failed, error")); - - key = bucket.getKey(toKeyName); - Assert.assertEquals(toKeyName, key.getName()); - } - - // Listing all volumes in the cluster feature has to be fixed after HDDS-357. - // TODO: fix this - @Ignore - @Test - public void testListVolume() throws IOException, OzoneException { - String volBase = "vol-" + RandomStringUtils.randomNumeric(3); - //Create 10 volume vol--a-0- to vol--a-9- - String volBaseNameA = volBase + "-a-"; - for(int i = 0; i < 10; i++) { - store.createVolume( - volBaseNameA + i + "-" + RandomStringUtils.randomNumeric(5)); - } - //Create 10 volume vol--b-0- to vol--b-9- - String volBaseNameB = volBase + "-b-"; - for(int i = 0; i < 10; i++) { - store.createVolume( - volBaseNameB + i + "-" + RandomStringUtils.randomNumeric(5)); - } - Iterator volIterator = store.listVolumes(volBase); - int totalVolumeCount = 0; - while(volIterator.hasNext()) { - volIterator.next(); - totalVolumeCount++; - } - Assert.assertEquals(20, totalVolumeCount); - Iterator volAIterator = store.listVolumes( - volBaseNameA); - for(int i = 0; i < 10; i++) { - Assert.assertTrue(volAIterator.next().getName() - .startsWith(volBaseNameA + i + "-")); - } - Assert.assertFalse(volAIterator.hasNext()); - Iterator volBIterator = store.listVolumes( - volBaseNameB); - for(int i = 0; i < 10; i++) { - Assert.assertTrue(volBIterator.next().getName() - .startsWith(volBaseNameB + i + "-")); - } - Assert.assertFalse(volBIterator.hasNext()); - Iterator iter = store.listVolumes(volBaseNameA + - "1-"); - Assert.assertTrue(iter.next().getName().startsWith(volBaseNameA + "1-")); - Assert.assertFalse(iter.hasNext()); - } - - @Test - public void testListBucket() - throws IOException, OzoneException { - String volumeA = "vol-a-" + RandomStringUtils.randomNumeric(5); - String volumeB = "vol-b-" + RandomStringUtils.randomNumeric(5); - store.createVolume(volumeA); - store.createVolume(volumeB); - OzoneVolume volA = store.getVolume(volumeA); - OzoneVolume volB = store.getVolume(volumeB); - - //Create 10 buckets in vol-a- and 10 in vol-b- - String bucketBaseNameA = "bucket-a-"; - for(int i = 0; i < 10; i++) { - volA.createBucket( - bucketBaseNameA + i + "-" + RandomStringUtils.randomNumeric(5)); - volB.createBucket( - bucketBaseNameA + i + "-" + RandomStringUtils.randomNumeric(5)); - } - //Create 10 buckets in vol-a- and 10 in vol-b- - String bucketBaseNameB = "bucket-b-"; - for(int i = 0; i < 10; i++) { - volA.createBucket( - bucketBaseNameB + i + "-" + RandomStringUtils.randomNumeric(5)); - volB.createBucket( - bucketBaseNameB + i + "-" + RandomStringUtils.randomNumeric(5)); - } - Iterator volABucketIter = - volA.listBuckets("bucket-"); - int volABucketCount = 0; - while(volABucketIter.hasNext()) { - volABucketIter.next(); - volABucketCount++; - } - Assert.assertEquals(20, volABucketCount); - Iterator volBBucketIter = - volA.listBuckets("bucket-"); - int volBBucketCount = 0; - while(volBBucketIter.hasNext()) { - volBBucketIter.next(); - volBBucketCount++; - } - Assert.assertEquals(20, volBBucketCount); - - Iterator volABucketAIter = - volA.listBuckets("bucket-a-"); - int volABucketACount = 0; - while(volABucketAIter.hasNext()) { - volABucketAIter.next(); - volABucketACount++; - } - Assert.assertEquals(10, volABucketACount); - Iterator volBBucketBIter = - volA.listBuckets("bucket-b-"); - int volBBucketBCount = 0; - while(volBBucketBIter.hasNext()) { - volBBucketBIter.next(); - volBBucketBCount++; - } - Assert.assertEquals(10, volBBucketBCount); - Iterator volABucketBIter = volA.listBuckets( - "bucket-b-"); - for(int i = 0; i < 10; i++) { - Assert.assertTrue(volABucketBIter.next().getName() - .startsWith(bucketBaseNameB + i + "-")); - } - Assert.assertFalse(volABucketBIter.hasNext()); - Iterator volBBucketAIter = volB.listBuckets( - "bucket-a-"); - for(int i = 0; i < 10; i++) { - Assert.assertTrue(volBBucketAIter.next().getName() - .startsWith(bucketBaseNameA + i + "-")); - } - Assert.assertFalse(volBBucketAIter.hasNext()); - - } - - @Test - public void testListBucketsOnEmptyVolume() - throws IOException, OzoneException { - String volume = "vol-" + RandomStringUtils.randomNumeric(5); - store.createVolume(volume); - OzoneVolume vol = store.getVolume(volume); - Iterator buckets = vol.listBuckets(""); - while(buckets.hasNext()) { - fail(); - } - } - - @Test - public void testListKey() - throws IOException, OzoneException { - String volumeA = "vol-a-" + RandomStringUtils.randomNumeric(5); - String volumeB = "vol-b-" + RandomStringUtils.randomNumeric(5); - String bucketA = "buc-a-" + RandomStringUtils.randomNumeric(5); - String bucketB = "buc-b-" + RandomStringUtils.randomNumeric(5); - store.createVolume(volumeA); - store.createVolume(volumeB); - OzoneVolume volA = store.getVolume(volumeA); - OzoneVolume volB = store.getVolume(volumeB); - volA.createBucket(bucketA); - volA.createBucket(bucketB); - volB.createBucket(bucketA); - volB.createBucket(bucketB); - OzoneBucket volAbucketA = volA.getBucket(bucketA); - OzoneBucket volAbucketB = volA.getBucket(bucketB); - OzoneBucket volBbucketA = volB.getBucket(bucketA); - OzoneBucket volBbucketB = volB.getBucket(bucketB); - - /* - Create 10 keys in vol-a-/buc-a-, - vol-a-/buc-b-, vol-b-/buc-a- and - vol-b-/buc-b- - */ - String keyBaseA = "key-a-"; - for (int i = 0; i < 10; i++) { - byte[] value = RandomStringUtils.randomAscii(10240).getBytes(); - OzoneOutputStream one = volAbucketA.createKey( - keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5), - value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - one.write(value); - one.close(); - OzoneOutputStream two = volAbucketB.createKey( - keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5), - value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - two.write(value); - two.close(); - OzoneOutputStream three = volBbucketA.createKey( - keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5), - value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - three.write(value); - three.close(); - OzoneOutputStream four = volBbucketB.createKey( - keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5), - value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - four.write(value); - four.close(); - } - /* - Create 10 keys in vol-a-/buc-a-, - vol-a-/buc-b-, vol-b-/buc-a- and - vol-b-/buc-b- - */ - String keyBaseB = "key-b-"; - for (int i = 0; i < 10; i++) { - byte[] value = RandomStringUtils.randomAscii(10240).getBytes(); - OzoneOutputStream one = volAbucketA.createKey( - keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5), - value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - one.write(value); - one.close(); - OzoneOutputStream two = volAbucketB.createKey( - keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5), - value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - two.write(value); - two.close(); - OzoneOutputStream three = volBbucketA.createKey( - keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5), - value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - three.write(value); - three.close(); - OzoneOutputStream four = volBbucketB.createKey( - keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5), - value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - four.write(value); - four.close(); - } - Iterator volABucketAIter = - volAbucketA.listKeys("key-"); - int volABucketAKeyCount = 0; - while(volABucketAIter.hasNext()) { - volABucketAIter.next(); - volABucketAKeyCount++; - } - Assert.assertEquals(20, volABucketAKeyCount); - Iterator volABucketBIter = - volAbucketB.listKeys("key-"); - int volABucketBKeyCount = 0; - while(volABucketBIter.hasNext()) { - volABucketBIter.next(); - volABucketBKeyCount++; - } - Assert.assertEquals(20, volABucketBKeyCount); - Iterator volBBucketAIter = - volBbucketA.listKeys("key-"); - int volBBucketAKeyCount = 0; - while(volBBucketAIter.hasNext()) { - volBBucketAIter.next(); - volBBucketAKeyCount++; - } - Assert.assertEquals(20, volBBucketAKeyCount); - Iterator volBBucketBIter = - volBbucketB.listKeys("key-"); - int volBBucketBKeyCount = 0; - while(volBBucketBIter.hasNext()) { - volBBucketBIter.next(); - volBBucketBKeyCount++; - } - Assert.assertEquals(20, volBBucketBKeyCount); - Iterator volABucketAKeyAIter = - volAbucketA.listKeys("key-a-"); - int volABucketAKeyACount = 0; - while(volABucketAKeyAIter.hasNext()) { - volABucketAKeyAIter.next(); - volABucketAKeyACount++; - } - Assert.assertEquals(10, volABucketAKeyACount); - Iterator volABucketAKeyBIter = - volAbucketA.listKeys("key-b-"); - for(int i = 0; i < 10; i++) { - Assert.assertTrue(volABucketAKeyBIter.next().getName() - .startsWith("key-b-" + i + "-")); - } - Assert.assertFalse(volABucketBIter.hasNext()); - } - - @Test - public void testListKeyOnEmptyBucket() - throws IOException, OzoneException { - String volume = "vol-" + RandomStringUtils.randomNumeric(5); - String bucket = "buc-" + RandomStringUtils.randomNumeric(5); - store.createVolume(volume); - OzoneVolume vol = store.getVolume(volume); - vol.createBucket(bucket); - OzoneBucket buc = vol.getBucket(bucket); - Iterator keys = buc.listKeys(""); - while(keys.hasNext()) { - fail(); - } - } - - @Test - public void testInitiateMultipartUploadWithReplicationInformationSet() throws - IOException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, - ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - - assertNotNull(multipartInfo); - String uploadID = multipartInfo.getUploadID(); - Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); - Assert.assertEquals(bucketName, multipartInfo.getBucketName()); - Assert.assertEquals(keyName, multipartInfo.getKeyName()); - assertNotNull(multipartInfo.getUploadID()); - - // Call initiate multipart upload for the same key again, this should - // generate a new uploadID. - multipartInfo = bucket.initiateMultipartUpload(keyName, - ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - - assertNotNull(multipartInfo); - Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); - Assert.assertEquals(bucketName, multipartInfo.getBucketName()); - Assert.assertEquals(keyName, multipartInfo.getKeyName()); - assertNotEquals(multipartInfo.getUploadID(), uploadID); - assertNotNull(multipartInfo.getUploadID()); - } - - - @Test - public void testInitiateMultipartUploadWithDefaultReplication() throws - IOException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName); - - assertNotNull(multipartInfo); - String uploadID = multipartInfo.getUploadID(); - Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); - Assert.assertEquals(bucketName, multipartInfo.getBucketName()); - Assert.assertEquals(keyName, multipartInfo.getKeyName()); - assertNotNull(multipartInfo.getUploadID()); - - // Call initiate multipart upload for the same key again, this should - // generate a new uploadID. - multipartInfo = bucket.initiateMultipartUpload(keyName); - - assertNotNull(multipartInfo); - Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); - Assert.assertEquals(bucketName, multipartInfo.getBucketName()); - Assert.assertEquals(keyName, multipartInfo.getKeyName()); - assertNotEquals(multipartInfo.getUploadID(), uploadID); - assertNotNull(multipartInfo.getUploadID()); - } - - - @Test - public void testUploadPartWithNoOverride() throws IOException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - String sampleData = "sample Value"; - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, - ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - - assertNotNull(multipartInfo); - String uploadID = multipartInfo.getUploadID(); - Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); - Assert.assertEquals(bucketName, multipartInfo.getBucketName()); - Assert.assertEquals(keyName, multipartInfo.getKeyName()); - assertNotNull(multipartInfo.getUploadID()); - - OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName, - sampleData.length(), 1, uploadID); - ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, - sampleData.length()); - ozoneOutputStream.close(); - - OmMultipartCommitUploadPartInfo commitUploadPartInfo = ozoneOutputStream - .getCommitUploadPartInfo(); - - assertNotNull(commitUploadPartInfo); - String partName = commitUploadPartInfo.getPartName(); - assertNotNull(commitUploadPartInfo.getPartName()); - - } - - @Test - public void testUploadPartOverrideWithStandAlone() throws IOException { - - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - String sampleData = "sample Value"; - int partNumber = 1; - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, - ReplicationType.STAND_ALONE, ReplicationFactor.ONE); - - assertNotNull(multipartInfo); - String uploadID = multipartInfo.getUploadID(); - Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); - Assert.assertEquals(bucketName, multipartInfo.getBucketName()); - Assert.assertEquals(keyName, multipartInfo.getKeyName()); - assertNotNull(multipartInfo.getUploadID()); - - OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName, - sampleData.length(), partNumber, uploadID); - ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, - sampleData.length()); - ozoneOutputStream.close(); - - OmMultipartCommitUploadPartInfo commitUploadPartInfo = ozoneOutputStream - .getCommitUploadPartInfo(); - - assertNotNull(commitUploadPartInfo); - String partName = commitUploadPartInfo.getPartName(); - assertNotNull(commitUploadPartInfo.getPartName()); - - //Overwrite the part by creating part key with same part number. - sampleData = "sample Data Changed"; - ozoneOutputStream = bucket.createMultipartKey(keyName, - sampleData.length(), partNumber, uploadID); - ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, "name" - .length()); - ozoneOutputStream.close(); - - commitUploadPartInfo = ozoneOutputStream - .getCommitUploadPartInfo(); - - assertNotNull(commitUploadPartInfo); - assertNotNull(commitUploadPartInfo.getPartName()); - - // PartName should be different from old part Name. - assertNotEquals("Part names should be different", partName, - commitUploadPartInfo.getPartName()); - } - - @Test - public void testUploadPartOverrideWithRatis() throws IOException { - - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - String sampleData = "sample Value"; - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, - ReplicationType.RATIS, ReplicationFactor.THREE); - - assertNotNull(multipartInfo); - String uploadID = multipartInfo.getUploadID(); - Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); - Assert.assertEquals(bucketName, multipartInfo.getBucketName()); - Assert.assertEquals(keyName, multipartInfo.getKeyName()); - assertNotNull(multipartInfo.getUploadID()); - - int partNumber = 1; - - OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName, - sampleData.length(), partNumber, uploadID); - ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, - sampleData.length()); - ozoneOutputStream.close(); - - OmMultipartCommitUploadPartInfo commitUploadPartInfo = ozoneOutputStream - .getCommitUploadPartInfo(); - - assertNotNull(commitUploadPartInfo); - String partName = commitUploadPartInfo.getPartName(); - assertNotNull(commitUploadPartInfo.getPartName()); - - //Overwrite the part by creating part key with same part number. - sampleData = "sample Data Changed"; - ozoneOutputStream = bucket.createMultipartKey(keyName, - sampleData.length(), partNumber, uploadID); - ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, "name" - .length()); - ozoneOutputStream.close(); - - commitUploadPartInfo = ozoneOutputStream - .getCommitUploadPartInfo(); - - assertNotNull(commitUploadPartInfo); - assertNotNull(commitUploadPartInfo.getPartName()); - - // PartName should be different from old part Name. - assertNotEquals("Part names should be different", partName, - commitUploadPartInfo.getPartName()); - } - - @Test - public void testNoSuchUploadError() throws IOException { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - String sampleData = "sample Value"; - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - String uploadID = "random"; - try { - bucket.createMultipartKey(keyName, sampleData.length(), 1, uploadID); - fail("testNoSuchUploadError failed"); - } catch (IOException ex) { - GenericTestUtils.assertExceptionContains("NO_SUCH_MULTIPART_UPLOAD_ERROR", - ex); - } - } - - @Test - public void testMultipartUpload() throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - doMultipartUpload(bucket, keyName, (byte)98); - - } - - - @Test - public void testMultipartUploadOverride() throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - doMultipartUpload(bucket, keyName, (byte)96); - - // Initiate Multipart upload again, now we should read latest version, as - // read always reads latest blocks. - doMultipartUpload(bucket, keyName, (byte)97); - - } - - - @Test - public void testMultipartUploadWithPartsLessThanMinSize() throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - // Initiate multipart upload - String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType - .STAND_ALONE, ReplicationFactor.ONE); - - // Upload Parts - Map partsMap = new TreeMap<>(); - // Uploading part 1 with less than min size - String partName = uploadPart(bucket, keyName, uploadID, 1, "data".getBytes( - "UTF-8")); - partsMap.put(1, partName); - - partName = uploadPart(bucket, keyName, uploadID, 2, "data".getBytes( - "UTF-8")); - partsMap.put(2, partName); - - - // Complete multipart upload - - try { - completeMultipartUpload(bucket, keyName, uploadID, partsMap); - fail("testMultipartUploadWithPartsLessThanMinSize failed"); - } catch (IOException ex) { - GenericTestUtils.assertExceptionContains("ENTITY_TOO_SMALL", ex); - } - - } - - - - @Test - public void testMultipartUploadWithPartsMisMatchWithListSizeDifferent() - throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType - .STAND_ALONE, ReplicationFactor.ONE); - - // We have not uploaded any parts, but passing some list it should throw - // error. - TreeMap partsMap = new TreeMap<>(); - partsMap.put(1, UUID.randomUUID().toString()); - - try { - completeMultipartUpload(bucket, keyName, uploadID, partsMap); - fail("testMultipartUploadWithPartsMisMatch"); - } catch (IOException ex) { - GenericTestUtils.assertExceptionContains("MISMATCH_MULTIPART_LIST", ex); - } - - } - - @Test - public void testMultipartUploadWithPartsMisMatchWithIncorrectPartName() - throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType - .STAND_ALONE, ReplicationFactor.ONE); - - uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8")); - // We have not uploaded any parts, but passing some list it should throw - // error. - TreeMap partsMap = new TreeMap<>(); - partsMap.put(1, UUID.randomUUID().toString()); - - try { - completeMultipartUpload(bucket, keyName, uploadID, partsMap); - fail("testMultipartUploadWithPartsMisMatch"); - } catch (IOException ex) { - GenericTestUtils.assertExceptionContains("MISMATCH_MULTIPART_LIST", ex); - } - - } - - @Test - public void testMultipartUploadWithMissingParts() throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType - .STAND_ALONE, ReplicationFactor.ONE); - - uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8")); - // We have not uploaded any parts, but passing some list it should throw - // error. - TreeMap partsMap = new TreeMap<>(); - partsMap.put(3, "random"); - - try { - completeMultipartUpload(bucket, keyName, uploadID, partsMap); - fail("testMultipartUploadWithPartsMisMatch"); - } catch (IOException ex) { - GenericTestUtils.assertExceptionContains("MISSING_UPLOAD_PARTS", ex); - } - } - - @Test - public void testAbortUploadFail() throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - try { - bucket.abortMultipartUpload(keyName, "random"); - fail("testAbortUploadFail failed"); - } catch (IOException ex) { - GenericTestUtils.assertExceptionContains( - "NO_SUCH_MULTIPART_UPLOAD_ERROR", ex); - } - } - - - @Test - public void testAbortUploadSuccessWithOutAnyParts() throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - try { - String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType - .STAND_ALONE, ReplicationFactor.ONE); - bucket.abortMultipartUpload(keyName, uploadID); - } catch (IOException ex) { - fail("testAbortUploadSuccess failed"); - } - } - - @Test - public void testAbortUploadSuccessWithParts() throws Exception { - String volumeName = UUID.randomUUID().toString(); - String bucketName = UUID.randomUUID().toString(); - String keyName = UUID.randomUUID().toString(); - - store.createVolume(volumeName); - OzoneVolume volume = store.getVolume(volumeName); - volume.createBucket(bucketName); - OzoneBucket bucket = volume.getBucket(bucketName); - - try { - String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType - .STAND_ALONE, ReplicationFactor.ONE); - uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8")); - bucket.abortMultipartUpload(keyName, uploadID); - } catch (IOException ex) { - fail("testAbortUploadSuccess failed"); - } - } - - - private byte[] generateData(int size, byte val) { - byte[] chars = new byte[size]; - Arrays.fill(chars, val); - return chars; - } - - - private void doMultipartUpload(OzoneBucket bucket, String keyName, byte val) - throws Exception { - // Initiate Multipart upload request - String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType - .RATIS, ReplicationFactor.THREE); - - // Upload parts - Map partsMap = new TreeMap<>(); - - // get 5mb data, as each part should be of min 5mb, last part can be less - // than 5mb - int length = 0; - byte[] data = generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, val); - String partName = uploadPart(bucket, keyName, uploadID, 1, data); - partsMap.put(1, partName); - length += data.length; - - - partName = uploadPart(bucket, keyName, uploadID, 2, data); - partsMap.put(2, partName); - length += data.length; - - String part3 = UUID.randomUUID().toString(); - partName = uploadPart(bucket, keyName, uploadID, 3, part3.getBytes( - "UTF-8")); - partsMap.put(3, partName); - length += part3.getBytes("UTF-8").length; - - - // Complete multipart upload request - completeMultipartUpload(bucket, keyName, uploadID, partsMap); - - - //Now Read the key which has been completed multipart upload. - byte[] fileContent = new byte[data.length + data.length + part3.getBytes( - "UTF-8").length]; - OzoneInputStream inputStream = bucket.readKey(keyName); - inputStream.read(fileContent); - - Assert.assertTrue(verifyRatisReplication(bucket.getVolumeName(), - bucket.getName(), keyName, ReplicationType.RATIS, - ReplicationFactor.THREE)); - - StringBuilder sb = new StringBuilder(length); - - // Combine all parts data, and check is it matching with get key data. - String part1 = new String(data); - String part2 = new String(data); - sb.append(part1); - sb.append(part2); - sb.append(part3); - Assert.assertEquals(sb.toString(), new String(fileContent)); - } - - - private String initiateMultipartUpload(OzoneBucket bucket, String keyName, - ReplicationType replicationType, ReplicationFactor replicationFactor) - throws Exception { - OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, - replicationType, replicationFactor); - - String uploadID = multipartInfo.getUploadID(); - Assert.assertNotNull(uploadID); - return uploadID; - } - - private String uploadPart(OzoneBucket bucket, String keyName, String - uploadID, int partNumber, byte[] data) throws Exception { - OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName, - data.length, partNumber, uploadID); - ozoneOutputStream.write(data, 0, - data.length); - ozoneOutputStream.close(); - - OmMultipartCommitUploadPartInfo omMultipartCommitUploadPartInfo = - ozoneOutputStream.getCommitUploadPartInfo(); - - Assert.assertNotNull(omMultipartCommitUploadPartInfo); - Assert.assertNotNull(omMultipartCommitUploadPartInfo.getPartName()); - return omMultipartCommitUploadPartInfo.getPartName(); - - } - - private void completeMultipartUpload(OzoneBucket bucket, String keyName, - String uploadID, Map partsMap) throws Exception { - OmMultipartUploadCompleteInfo omMultipartUploadCompleteInfo = bucket - .completeMultipartUpload(keyName, uploadID, partsMap); - - Assert.assertNotNull(omMultipartUploadCompleteInfo); - Assert.assertEquals(omMultipartUploadCompleteInfo.getBucket(), bucket - .getName()); - Assert.assertEquals(omMultipartUploadCompleteInfo.getVolume(), bucket - .getVolumeName()); - Assert.assertEquals(omMultipartUploadCompleteInfo.getKey(), keyName); - Assert.assertNotNull(omMultipartUploadCompleteInfo.getHash()); - } - - /** * Close OzoneClient and shutdown MiniOzoneCluster. */ @AfterClass public static void shutdown() throws IOException { - if(ozClient != null) { - ozClient.close(); - } - - if (storageContainerLocationClient != null) { - storageContainerLocationClient.close(); - } - - if (cluster != null) { - cluster.shutdown(); - } + shutdownCluster(); } - } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java new file mode 100644 index 0000000000..e7bca5e78d --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java @@ -0,0 +1,1791 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.ozone.client.rpc; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.UUID; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.RandomUtils; +import org.apache.hadoop.hdds.client.OzoneQuota; +import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientRatis; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.protocolPB + .StorageContainerLocationProtocolClientSideTranslatorPB; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.ozone.HddsDatanodeService; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneAcl; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.client.BucketArgs; +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.OzoneClientFactory; +import org.apache.hadoop.ozone.client.OzoneKey; +import org.apache.hadoop.ozone.client.OzoneKeyDetails; +import org.apache.hadoop.ozone.client.OzoneKeyLocation; +import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.client.io.KeyOutputStream; +import org.apache.hadoop.ozone.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.client.rest.OzoneException; +import org.apache.hadoop.ozone.common.OzoneChecksumException; +import org.apache.hadoop.ozone.container.common.helpers.BlockData; +import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; +import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.Time; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.either; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * This is an abstract class to test all the public facing APIs of Ozone + * Client, w/o OM Ratis server. + * {@link TestOzoneRpcClient} tests the Ozone Client by submitting the + * requests directly to OzoneManager. {@link TestOzoneRpcClientWithRatis} + * tests the Ozone Client by submitting requests to OM's Ratis server. + */ +public abstract class TestOzoneRpcClientAbstract { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static MiniOzoneCluster cluster = null; + private static OzoneClient ozClient = null; + private static ObjectStore store = null; + private static OzoneManager ozoneManager; + private static StorageContainerLocationProtocolClientSideTranslatorPB + storageContainerLocationClient; + + private static final String SCM_ID = UUID.randomUUID().toString(); + + /** + * Create a MiniOzoneCluster for testing. + * @param conf Configurations to start the cluster. + * @throws Exception + */ + static void startCluster(OzoneConfiguration conf) throws Exception { + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(10) + .setScmId(SCM_ID) + .build(); + cluster.waitForClusterToBeReady(); + ozClient = OzoneClientFactory.getRpcClient(conf); + store = ozClient.getObjectStore(); + storageContainerLocationClient = + cluster.getStorageContainerLocationClient(); + ozoneManager = cluster.getOzoneManager(); + } + + /** + * Close OzoneClient and shutdown MiniOzoneCluster. + */ + static void shutdownCluster() throws IOException { + if(ozClient != null) { + ozClient.close(); + } + + if (storageContainerLocationClient != null) { + storageContainerLocationClient.close(); + } + + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testSetVolumeQuota() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + store.getVolume(volumeName).setQuota( + OzoneQuota.parseQuota("100000000 BYTES")); + OzoneVolume volume = store.getVolume(volumeName); + Assert.assertEquals(100000000L, volume.getQuota()); + } + + @Test + public void testDeleteVolume() + throws IOException, OzoneException { + thrown.expectMessage("Info Volume failed, error"); + String volumeName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + Assert.assertNotNull(volume); + store.deleteVolume(volumeName); + store.getVolume(volumeName); + } + + @Test + public void testCreateBucket() + throws IOException, OzoneException { + long currentTime = Time.now(); + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, bucket.getName()); + Assert.assertTrue(bucket.getCreationTime() >= currentTime); + Assert.assertTrue(volume.getCreationTime() >= currentTime); + } + + @Test + public void testCreateS3Bucket() + throws IOException, OzoneException { + long currentTime = Time.now(); + String userName = "ozone"; + String bucketName = UUID.randomUUID().toString(); + store.createS3Bucket(userName, bucketName); + String volumeName = store.getOzoneVolumeName(bucketName); + OzoneVolume volume = store.getVolume(volumeName); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, bucket.getName()); + Assert.assertTrue(bucket.getCreationTime() >= currentTime); + Assert.assertTrue(volume.getCreationTime() >= currentTime); + } + + + @Test + public void testListS3Buckets() + throws IOException, OzoneException { + String userName = "ozone100"; + String bucketName1 = UUID.randomUUID().toString(); + String bucketName2 = UUID.randomUUID().toString(); + store.createS3Bucket(userName, bucketName1); + store.createS3Bucket(userName, bucketName2); + Iterator iterator = store.listS3Buckets(userName, + null); + + while (iterator.hasNext()) { + assertThat(iterator.next().getName(), either(containsString(bucketName1)) + .or(containsString(bucketName2))); + } + + } + + @Test + public void testListS3BucketsFail() + throws IOException, OzoneException { + String userName = "randomUser"; + Iterator iterator = store.listS3Buckets(userName, + null); + + Assert.assertFalse(iterator.hasNext()); + + } + + @Test + public void testDeleteS3Bucket() + throws IOException, OzoneException { + long currentTime = Time.now(); + String userName = "ozone1"; + String bucketName = UUID.randomUUID().toString(); + store.createS3Bucket(userName, bucketName); + String volumeName = store.getOzoneVolumeName(bucketName); + OzoneVolume volume = store.getVolume(volumeName); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, bucket.getName()); + Assert.assertTrue(bucket.getCreationTime() >= currentTime); + Assert.assertTrue(volume.getCreationTime() >= currentTime); + store.deleteS3Bucket(bucketName); + thrown.expect(IOException.class); + store.getOzoneVolumeName(bucketName); + } + + @Test + public void testDeleteS3NonExistingBucket() { + try { + store.deleteS3Bucket(UUID.randomUUID().toString()); + } catch (IOException ex) { + GenericTestUtils.assertExceptionContains("NOT_FOUND", ex); + } + } + + @Test + public void testCreateS3BucketMapping() + throws IOException, OzoneException { + long currentTime = Time.now(); + String userName = "ozone"; + String bucketName = UUID.randomUUID().toString(); + store.createS3Bucket(userName, bucketName); + String volumeName = store.getOzoneVolumeName(bucketName); + OzoneVolume volume = store.getVolume(volumeName); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, bucket.getName()); + + String mapping = store.getOzoneBucketMapping(bucketName); + Assert.assertEquals("s3"+userName+"/"+bucketName, mapping); + Assert.assertEquals(bucketName, store.getOzoneBucketName(bucketName)); + Assert.assertEquals("s3"+userName, store.getOzoneVolumeName(bucketName)); + + } + + @Test + public void testCreateBucketWithVersioning() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + BucketArgs.Builder builder = BucketArgs.newBuilder(); + builder.setVersioning(true); + volume.createBucket(bucketName, builder.build()); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, bucket.getName()); + Assert.assertEquals(true, bucket.getVersioning()); + } + + @Test + public void testCreateBucketWithStorageType() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + BucketArgs.Builder builder = BucketArgs.newBuilder(); + builder.setStorageType(StorageType.SSD); + volume.createBucket(bucketName, builder.build()); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, bucket.getName()); + Assert.assertEquals(StorageType.SSD, bucket.getStorageType()); + } + + @Test + public void testCreateBucketWithAcls() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test", + OzoneAcl.OzoneACLRights.READ_WRITE); + List acls = new ArrayList<>(); + acls.add(userAcl); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + BucketArgs.Builder builder = BucketArgs.newBuilder(); + builder.setAcls(acls); + volume.createBucket(bucketName, builder.build()); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, bucket.getName()); + Assert.assertTrue(bucket.getAcls().contains(userAcl)); + } + + @Test + public void testCreateBucketWithAllArgument() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test", + OzoneAcl.OzoneACLRights.READ_WRITE); + List acls = new ArrayList<>(); + acls.add(userAcl); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + BucketArgs.Builder builder = BucketArgs.newBuilder(); + builder.setVersioning(true) + .setStorageType(StorageType.SSD) + .setAcls(acls); + volume.createBucket(bucketName, builder.build()); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, bucket.getName()); + Assert.assertEquals(true, bucket.getVersioning()); + Assert.assertEquals(StorageType.SSD, bucket.getStorageType()); + Assert.assertTrue(bucket.getAcls().contains(userAcl)); + } + + @Test + public void testInvalidBucketCreation() throws IOException { + thrown.expectMessage("Bucket or Volume name has an unsupported" + + " character : #"); + String volumeName = UUID.randomUUID().toString(); + String bucketName = "invalid#bucket"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + } + + @Test + public void testAddBucketAcl() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + List acls = new ArrayList<>(); + acls.add(new OzoneAcl( + OzoneAcl.OzoneACLType.USER, "test", + OzoneAcl.OzoneACLRights.READ_WRITE)); + OzoneBucket bucket = volume.getBucket(bucketName); + bucket.addAcls(acls); + OzoneBucket newBucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, newBucket.getName()); + Assert.assertTrue(bucket.getAcls().contains(acls.get(0))); + } + + @Test + public void testRemoveBucketAcl() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test", + OzoneAcl.OzoneACLRights.READ_WRITE); + List acls = new ArrayList<>(); + acls.add(userAcl); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + BucketArgs.Builder builder = BucketArgs.newBuilder(); + builder.setAcls(acls); + volume.createBucket(bucketName, builder.build()); + OzoneBucket bucket = volume.getBucket(bucketName); + bucket.removeAcls(acls); + OzoneBucket newBucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, newBucket.getName()); + Assert.assertTrue(!bucket.getAcls().contains(acls.get(0))); + } + + @Test + public void testSetBucketVersioning() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + bucket.setVersioning(true); + OzoneBucket newBucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, newBucket.getName()); + Assert.assertEquals(true, newBucket.getVersioning()); + } + + @Test + public void testSetBucketStorageType() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + bucket.setStorageType(StorageType.SSD); + OzoneBucket newBucket = volume.getBucket(bucketName); + Assert.assertEquals(bucketName, newBucket.getName()); + Assert.assertEquals(StorageType.SSD, newBucket.getStorageType()); + } + + + @Test + public void testDeleteBucket() + throws IOException, OzoneException { + thrown.expectMessage("Info Bucket failed, error"); + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + Assert.assertNotNull(bucket); + volume.deleteBucket(bucketName); + volume.getBucket(bucketName); + } + + private boolean verifyRatisReplication(String volumeName, String bucketName, + String keyName, ReplicationType type, ReplicationFactor factor) + throws IOException { + OmKeyArgs keyArgs = new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .build(); + HddsProtos.ReplicationType replicationType = + HddsProtos.ReplicationType.valueOf(type.toString()); + HddsProtos.ReplicationFactor replicationFactor = + HddsProtos.ReplicationFactor.valueOf(factor.getValue()); + OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs); + for (OmKeyLocationInfo info: + keyInfo.getLatestVersionLocations().getLocationList()) { + ContainerInfo container = + storageContainerLocationClient.getContainer(info.getContainerID()); + if (!container.getReplicationFactor().equals(replicationFactor) || ( + container.getReplicationType() != replicationType)) { + return false; + } + } + return true; + } + + @Test + public void testPutKey() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + long currentTime = Time.now(); + + String value = "sample value"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + for (int i = 0; i < 10; i++) { + String keyName = UUID.randomUUID().toString(); + + OzoneOutputStream out = bucket.createKey(keyName, + value.getBytes().length, ReplicationType.STAND_ALONE, + ReplicationFactor.ONE); + out.write(value.getBytes()); + out.close(); + OzoneKey key = bucket.getKey(keyName); + Assert.assertEquals(keyName, key.getName()); + OzoneInputStream is = bucket.readKey(keyName); + byte[] fileContent = new byte[value.getBytes().length]; + is.read(fileContent); + Assert.assertTrue(verifyRatisReplication(volumeName, bucketName, + keyName, ReplicationType.STAND_ALONE, + ReplicationFactor.ONE)); + Assert.assertEquals(value, new String(fileContent)); + Assert.assertTrue(key.getCreationTime() >= currentTime); + Assert.assertTrue(key.getModificationTime() >= currentTime); + } + } + + @Test + public void testValidateBlockLengthWithCommitKey() throws IOException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + + String value = RandomStringUtils.random(RandomUtils.nextInt(0, 1024)); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + String keyName = UUID.randomUUID().toString(); + + // create the initial key with size 0, write will allocate the first block. + OzoneOutputStream out = bucket.createKey(keyName, 0, + ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + out.write(value.getBytes()); + out.close(); + OmKeyArgs.Builder builder = new OmKeyArgs.Builder(); + builder.setVolumeName(volumeName).setBucketName(bucketName) + .setKeyName(keyName); + OmKeyInfo keyInfo = ozoneManager.lookupKey(builder.build()); + + List locationInfoList = + keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly(); + // LocationList should have only 1 block + Assert.assertEquals(1, locationInfoList.size()); + // make sure the data block size is updated + Assert.assertEquals(value.getBytes().length, + locationInfoList.get(0).getLength()); + // make sure the total data size is set correctly + Assert.assertEquals(value.getBytes().length, keyInfo.getDataSize()); + } + + + @Test + public void testPutKeyRatisOneNode() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + long currentTime = Time.now(); + + String value = "sample value"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + for (int i = 0; i < 10; i++) { + String keyName = UUID.randomUUID().toString(); + + OzoneOutputStream out = bucket.createKey(keyName, + value.getBytes().length, ReplicationType.RATIS, + ReplicationFactor.ONE); + out.write(value.getBytes()); + out.close(); + OzoneKey key = bucket.getKey(keyName); + Assert.assertEquals(keyName, key.getName()); + OzoneInputStream is = bucket.readKey(keyName); + byte[] fileContent = new byte[value.getBytes().length]; + is.read(fileContent); + is.close(); + Assert.assertTrue(verifyRatisReplication(volumeName, bucketName, + keyName, ReplicationType.RATIS, ReplicationFactor.ONE)); + Assert.assertEquals(value, new String(fileContent)); + Assert.assertTrue(key.getCreationTime() >= currentTime); + Assert.assertTrue(key.getModificationTime() >= currentTime); + } + } + + @Test + public void testPutKeyRatisThreeNodes() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + long currentTime = Time.now(); + + String value = "sample value"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + for (int i = 0; i < 10; i++) { + String keyName = UUID.randomUUID().toString(); + + OzoneOutputStream out = bucket.createKey(keyName, + value.getBytes().length, ReplicationType.RATIS, + ReplicationFactor.THREE); + out.write(value.getBytes()); + out.close(); + OzoneKey key = bucket.getKey(keyName); + Assert.assertEquals(keyName, key.getName()); + OzoneInputStream is = bucket.readKey(keyName); + byte[] fileContent = new byte[value.getBytes().length]; + is.read(fileContent); + is.close(); + Assert.assertTrue(verifyRatisReplication(volumeName, bucketName, + keyName, ReplicationType.RATIS, + ReplicationFactor.THREE)); + Assert.assertEquals(value, new String(fileContent)); + Assert.assertTrue(key.getCreationTime() >= currentTime); + Assert.assertTrue(key.getModificationTime() >= currentTime); + } + } + + @Test + public void testPutKeyAndGetKeyThreeNodes() + throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + + String value = "sample value"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + String keyName = UUID.randomUUID().toString(); + + OzoneOutputStream out = bucket + .createKey(keyName, value.getBytes().length, ReplicationType.RATIS, + ReplicationFactor.THREE); + KeyOutputStream groupOutputStream = + (KeyOutputStream) out.getOutputStream(); + XceiverClientManager manager = groupOutputStream.getXceiverClientManager(); + out.write(value.getBytes()); + out.close(); + // First, confirm the key info from the client matches the info in OM. + OmKeyArgs.Builder builder = new OmKeyArgs.Builder(); + builder.setVolumeName(volumeName).setBucketName(bucketName) + .setKeyName(keyName); + OmKeyLocationInfo keyInfo = ozoneManager.lookupKey(builder.build()). + getKeyLocationVersions().get(0).getBlocksLatestVersionOnly().get(0); + long containerID = keyInfo.getContainerID(); + long localID = keyInfo.getLocalID(); + OzoneKeyDetails keyDetails = bucket.getKey(keyName); + Assert.assertEquals(keyName, keyDetails.getName()); + + List keyLocations = keyDetails.getOzoneKeyLocations(); + Assert.assertEquals(1, keyLocations.size()); + Assert.assertEquals(containerID, keyLocations.get(0).getContainerID()); + Assert.assertEquals(localID, keyLocations.get(0).getLocalID()); + + // Make sure that the data size matched. + Assert + .assertEquals(value.getBytes().length, keyLocations.get(0).getLength()); + + ContainerInfo container = cluster.getStorageContainerManager() + .getContainerManager().getContainer(ContainerID.valueof(containerID)); + Pipeline pipeline = cluster.getStorageContainerManager() + .getPipelineManager().getPipeline(container.getPipelineID()); + List datanodes = pipeline.getNodes(); + + DatanodeDetails datanodeDetails = datanodes.get(0); + Assert.assertNotNull(datanodeDetails); + + XceiverClientSpi clientSpi = manager.acquireClient(pipeline); + Assert.assertTrue(clientSpi instanceof XceiverClientRatis); + XceiverClientRatis ratisClient = (XceiverClientRatis)clientSpi; + + ratisClient.watchForCommit(keyInfo.getBlockCommitSequenceId(), 5000); + // shutdown the datanode + cluster.shutdownHddsDatanode(datanodeDetails); + + Assert.assertTrue(container.getState() + == HddsProtos.LifeCycleState.OPEN); + // try to read, this shouls be successful + readKey(bucket, keyName, value); + + Assert.assertTrue(container.getState() + == HddsProtos.LifeCycleState.OPEN); + // shutdown the second datanode + datanodeDetails = datanodes.get(1); + cluster.shutdownHddsDatanode(datanodeDetails); + Assert.assertTrue(container.getState() + == HddsProtos.LifeCycleState.OPEN); + + // the container is open and with loss of 2 nodes we still should be able + // to read via Standalone protocol + // try to read + readKey(bucket, keyName, value); + + // shutdown the 3rd datanode + datanodeDetails = datanodes.get(2); + cluster.shutdownHddsDatanode(datanodeDetails); + try { + // try to read + readKey(bucket, keyName, value); + fail("Expected exception not thrown"); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("Failed to execute command")); + Assert.assertTrue( + e.getMessage().contains("on the pipeline " + pipeline.getId())); + } + manager.releaseClient(clientSpi); + } + + private void readKey(OzoneBucket bucket, String keyName, String data) + throws IOException { + OzoneKey key = bucket.getKey(keyName); + Assert.assertEquals(keyName, key.getName()); + OzoneInputStream is = bucket.readKey(keyName); + byte[] fileContent = new byte[data.getBytes().length]; + is.read(fileContent); + is.close(); + } + + @Test + public void testGetKeyDetails() throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + String keyName = UUID.randomUUID().toString(); + String keyValue = RandomStringUtils.random(128); + //String keyValue = "this is a test value.glx"; + // create the initial key with size 0, write will allocate the first block. + OzoneOutputStream out = bucket.createKey(keyName, + keyValue.getBytes().length, ReplicationType.STAND_ALONE, + ReplicationFactor.ONE); + out.write(keyValue.getBytes()); + out.close(); + + OzoneInputStream is = bucket.readKey(keyName); + byte[] fileContent = new byte[32]; + is.read(fileContent); + + // First, confirm the key info from the client matches the info in OM. + OmKeyArgs.Builder builder = new OmKeyArgs.Builder(); + builder.setVolumeName(volumeName).setBucketName(bucketName) + .setKeyName(keyName); + OmKeyLocationInfo keyInfo = ozoneManager.lookupKey(builder.build()). + getKeyLocationVersions().get(0).getBlocksLatestVersionOnly().get(0); + long containerID = keyInfo.getContainerID(); + long localID = keyInfo.getLocalID(); + OzoneKeyDetails keyDetails = (OzoneKeyDetails)bucket.getKey(keyName); + Assert.assertEquals(keyName, keyDetails.getName()); + + List keyLocations = keyDetails.getOzoneKeyLocations(); + Assert.assertEquals(1, keyLocations.size()); + Assert.assertEquals(containerID, keyLocations.get(0).getContainerID()); + Assert.assertEquals(localID, keyLocations.get(0).getLocalID()); + + // Make sure that the data size matched. + Assert.assertEquals(keyValue.getBytes().length, + keyLocations.get(0).getLength()); + + // Second, sum the data size from chunks in Container via containerID + // and localID, make sure the size equals to the size from keyDetails. + ContainerInfo container = cluster.getStorageContainerManager() + .getContainerManager().getContainer(ContainerID.valueof(containerID)); + Pipeline pipeline = cluster.getStorageContainerManager() + .getPipelineManager().getPipeline(container.getPipelineID()); + List datanodes = pipeline.getNodes(); + Assert.assertEquals(datanodes.size(), 1); + + DatanodeDetails datanodeDetails = datanodes.get(0); + Assert.assertNotNull(datanodeDetails); + HddsDatanodeService datanodeService = null; + for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) { + if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) { + datanodeService = datanodeServiceItr; + break; + } + } + KeyValueContainerData containerData = + (KeyValueContainerData)(datanodeService.getDatanodeStateMachine() + .getContainer().getContainerSet().getContainer(containerID) + .getContainerData()); + String containerPath = new File(containerData.getMetadataPath()) + .getParent(); + KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( + containerID, new File(containerPath)); + while (keyValueBlockIterator.hasNext()) { + BlockData blockData = keyValueBlockIterator.nextBlock(); + if (blockData.getBlockID().getLocalID() == localID) { + long length = 0; + List chunks = blockData.getChunks(); + for (ContainerProtos.ChunkInfo chunk : chunks) { + length += chunk.getLen(); + } + Assert.assertEquals(length, keyValue.getBytes().length); + break; + } + } + } + + /** + * Tests reading a corrputed chunk file throws checksum exception. + * @throws IOException + */ + @Test + public void testReadKeyWithCorruptedData() throws IOException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + + String value = "sample value"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + String keyName = UUID.randomUUID().toString(); + + // Write data into a key + OzoneOutputStream out = bucket.createKey(keyName, + value.getBytes().length, ReplicationType.STAND_ALONE, + ReplicationFactor.ONE); + out.write(value.getBytes()); + out.close(); + + // We need to find the location of the chunk file corresponding to the + // data we just wrote. + OzoneKey key = bucket.getKey(keyName); + long containerID = ((OzoneKeyDetails) key).getOzoneKeyLocations().get(0) + .getContainerID(); + long localID = ((OzoneKeyDetails) key).getOzoneKeyLocations().get(0) + .getLocalID(); + + // Get the container by traversing the datanodes. Atleast one of the + // datanode must have this container. + Container container = null; + for (HddsDatanodeService hddsDatanode : cluster.getHddsDatanodes()) { + container = hddsDatanode.getDatanodeStateMachine().getContainer() + .getContainerSet().getContainer(containerID); + if (container != null) { + break; + } + } + Assert.assertNotNull("Container not found", container); + + // From the containerData, get the block iterator for all the blocks in + // the container. + KeyValueContainerData containerData = + (KeyValueContainerData) container.getContainerData(); + String containerPath = new File(containerData.getMetadataPath()) + .getParent(); + KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator( + containerID, new File(containerPath)); + + // Find the block corresponding to the key we put. We use the localID of + // the BlockData to identify out key. + BlockData blockData = null; + while (keyValueBlockIterator.hasNext()) { + blockData = keyValueBlockIterator.nextBlock(); + if (blockData.getBlockID().getLocalID() == localID) { + break; + } + } + Assert.assertNotNull("Block not found", blockData); + + // Get the location of the chunk file + String chunkName = blockData.getChunks().get(0).getChunkName(); + String containreBaseDir = container.getContainerData().getVolume() + .getHddsRootDir().getPath(); + File chunksLocationPath = KeyValueContainerLocationUtil + .getChunksLocationPath(containreBaseDir, SCM_ID, containerID); + File chunkFile = new File(chunksLocationPath, chunkName); + + // Corrupt the contents of the chunk file + String newData = new String("corrupted data"); + FileUtils.writeByteArrayToFile(chunkFile, newData.getBytes()); + + // Try reading the key. Since the chunk file is corrupted, it should + // throw a checksum mismatch exception. + try { + OzoneInputStream is = bucket.readKey(keyName); + is.read(new byte[100]); + fail("Reading corrupted data should fail."); + } catch (OzoneChecksumException e) { + GenericTestUtils.assertExceptionContains("Checksum mismatch", e); + } + } + + @Test + public void testDeleteKey() + throws IOException, OzoneException { + thrown.expectMessage("Lookup key failed, error"); + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + String value = "sample value"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + OzoneOutputStream out = bucket.createKey(keyName, + value.getBytes().length, ReplicationType.STAND_ALONE, + ReplicationFactor.ONE); + out.write(value.getBytes()); + out.close(); + OzoneKey key = bucket.getKey(keyName); + Assert.assertEquals(keyName, key.getName()); + bucket.deleteKey(keyName); + bucket.getKey(keyName); + } + + @Test + public void testRenameKey() + throws IOException, OzoneException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String fromKeyName = UUID.randomUUID().toString(); + String value = "sample value"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + OzoneOutputStream out = bucket.createKey(fromKeyName, + value.getBytes().length, ReplicationType.STAND_ALONE, + ReplicationFactor.ONE); + out.write(value.getBytes()); + out.close(); + OzoneKey key = bucket.getKey(fromKeyName); + Assert.assertEquals(fromKeyName, key.getName()); + + // Rename to empty string should fail. + IOException ioe = null; + String toKeyName = ""; + try { + bucket.renameKey(fromKeyName, toKeyName); + } catch (IOException e) { + ioe = e; + } + Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error")); + + toKeyName = UUID.randomUUID().toString(); + bucket.renameKey(fromKeyName, toKeyName); + + // Lookup for old key should fail. + try { + bucket.getKey(fromKeyName); + } catch (IOException e) { + ioe = e; + } + Assert.assertTrue(ioe.getMessage().contains("Lookup key failed, error")); + + key = bucket.getKey(toKeyName); + Assert.assertEquals(toKeyName, key.getName()); + } + + // Listing all volumes in the cluster feature has to be fixed after HDDS-357. + // TODO: fix this + @Ignore + @Test + public void testListVolume() throws IOException, OzoneException { + String volBase = "vol-" + RandomStringUtils.randomNumeric(3); + //Create 10 volume vol--a-0- to vol--a-9- + String volBaseNameA = volBase + "-a-"; + for(int i = 0; i < 10; i++) { + store.createVolume( + volBaseNameA + i + "-" + RandomStringUtils.randomNumeric(5)); + } + //Create 10 volume vol--b-0- to vol--b-9- + String volBaseNameB = volBase + "-b-"; + for(int i = 0; i < 10; i++) { + store.createVolume( + volBaseNameB + i + "-" + RandomStringUtils.randomNumeric(5)); + } + Iterator volIterator = store.listVolumes(volBase); + int totalVolumeCount = 0; + while(volIterator.hasNext()) { + volIterator.next(); + totalVolumeCount++; + } + Assert.assertEquals(20, totalVolumeCount); + Iterator volAIterator = store.listVolumes( + volBaseNameA); + for(int i = 0; i < 10; i++) { + Assert.assertTrue(volAIterator.next().getName() + .startsWith(volBaseNameA + i + "-")); + } + Assert.assertFalse(volAIterator.hasNext()); + Iterator volBIterator = store.listVolumes( + volBaseNameB); + for(int i = 0; i < 10; i++) { + Assert.assertTrue(volBIterator.next().getName() + .startsWith(volBaseNameB + i + "-")); + } + Assert.assertFalse(volBIterator.hasNext()); + Iterator iter = store.listVolumes(volBaseNameA + + "1-"); + Assert.assertTrue(iter.next().getName().startsWith(volBaseNameA + "1-")); + Assert.assertFalse(iter.hasNext()); + } + + @Test + public void testListBucket() + throws IOException, OzoneException { + String volumeA = "vol-a-" + RandomStringUtils.randomNumeric(5); + String volumeB = "vol-b-" + RandomStringUtils.randomNumeric(5); + store.createVolume(volumeA); + store.createVolume(volumeB); + OzoneVolume volA = store.getVolume(volumeA); + OzoneVolume volB = store.getVolume(volumeB); + + //Create 10 buckets in vol-a- and 10 in vol-b- + String bucketBaseNameA = "bucket-a-"; + for(int i = 0; i < 10; i++) { + volA.createBucket( + bucketBaseNameA + i + "-" + RandomStringUtils.randomNumeric(5)); + volB.createBucket( + bucketBaseNameA + i + "-" + RandomStringUtils.randomNumeric(5)); + } + //Create 10 buckets in vol-a- and 10 in vol-b- + String bucketBaseNameB = "bucket-b-"; + for(int i = 0; i < 10; i++) { + volA.createBucket( + bucketBaseNameB + i + "-" + RandomStringUtils.randomNumeric(5)); + volB.createBucket( + bucketBaseNameB + i + "-" + RandomStringUtils.randomNumeric(5)); + } + Iterator volABucketIter = + volA.listBuckets("bucket-"); + int volABucketCount = 0; + while(volABucketIter.hasNext()) { + volABucketIter.next(); + volABucketCount++; + } + Assert.assertEquals(20, volABucketCount); + Iterator volBBucketIter = + volA.listBuckets("bucket-"); + int volBBucketCount = 0; + while(volBBucketIter.hasNext()) { + volBBucketIter.next(); + volBBucketCount++; + } + Assert.assertEquals(20, volBBucketCount); + + Iterator volABucketAIter = + volA.listBuckets("bucket-a-"); + int volABucketACount = 0; + while(volABucketAIter.hasNext()) { + volABucketAIter.next(); + volABucketACount++; + } + Assert.assertEquals(10, volABucketACount); + Iterator volBBucketBIter = + volA.listBuckets("bucket-b-"); + int volBBucketBCount = 0; + while(volBBucketBIter.hasNext()) { + volBBucketBIter.next(); + volBBucketBCount++; + } + Assert.assertEquals(10, volBBucketBCount); + Iterator volABucketBIter = volA.listBuckets( + "bucket-b-"); + for(int i = 0; i < 10; i++) { + Assert.assertTrue(volABucketBIter.next().getName() + .startsWith(bucketBaseNameB + i + "-")); + } + Assert.assertFalse(volABucketBIter.hasNext()); + Iterator volBBucketAIter = volB.listBuckets( + "bucket-a-"); + for(int i = 0; i < 10; i++) { + Assert.assertTrue(volBBucketAIter.next().getName() + .startsWith(bucketBaseNameA + i + "-")); + } + Assert.assertFalse(volBBucketAIter.hasNext()); + + } + + @Test + public void testListBucketsOnEmptyVolume() + throws IOException, OzoneException { + String volume = "vol-" + RandomStringUtils.randomNumeric(5); + store.createVolume(volume); + OzoneVolume vol = store.getVolume(volume); + Iterator buckets = vol.listBuckets(""); + while(buckets.hasNext()) { + fail(); + } + } + + @Test + public void testListKey() + throws IOException, OzoneException { + String volumeA = "vol-a-" + RandomStringUtils.randomNumeric(5); + String volumeB = "vol-b-" + RandomStringUtils.randomNumeric(5); + String bucketA = "buc-a-" + RandomStringUtils.randomNumeric(5); + String bucketB = "buc-b-" + RandomStringUtils.randomNumeric(5); + store.createVolume(volumeA); + store.createVolume(volumeB); + OzoneVolume volA = store.getVolume(volumeA); + OzoneVolume volB = store.getVolume(volumeB); + volA.createBucket(bucketA); + volA.createBucket(bucketB); + volB.createBucket(bucketA); + volB.createBucket(bucketB); + OzoneBucket volAbucketA = volA.getBucket(bucketA); + OzoneBucket volAbucketB = volA.getBucket(bucketB); + OzoneBucket volBbucketA = volB.getBucket(bucketA); + OzoneBucket volBbucketB = volB.getBucket(bucketB); + + /* + Create 10 keys in vol-a-/buc-a-, + vol-a-/buc-b-, vol-b-/buc-a- and + vol-b-/buc-b- + */ + String keyBaseA = "key-a-"; + for (int i = 0; i < 10; i++) { + byte[] value = RandomStringUtils.randomAscii(10240).getBytes(); + OzoneOutputStream one = volAbucketA.createKey( + keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5), + value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + one.write(value); + one.close(); + OzoneOutputStream two = volAbucketB.createKey( + keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5), + value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + two.write(value); + two.close(); + OzoneOutputStream three = volBbucketA.createKey( + keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5), + value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + three.write(value); + three.close(); + OzoneOutputStream four = volBbucketB.createKey( + keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5), + value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + four.write(value); + four.close(); + } + /* + Create 10 keys in vol-a-/buc-a-, + vol-a-/buc-b-, vol-b-/buc-a- and + vol-b-/buc-b- + */ + String keyBaseB = "key-b-"; + for (int i = 0; i < 10; i++) { + byte[] value = RandomStringUtils.randomAscii(10240).getBytes(); + OzoneOutputStream one = volAbucketA.createKey( + keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5), + value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + one.write(value); + one.close(); + OzoneOutputStream two = volAbucketB.createKey( + keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5), + value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + two.write(value); + two.close(); + OzoneOutputStream three = volBbucketA.createKey( + keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5), + value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + three.write(value); + three.close(); + OzoneOutputStream four = volBbucketB.createKey( + keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5), + value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + four.write(value); + four.close(); + } + Iterator volABucketAIter = + volAbucketA.listKeys("key-"); + int volABucketAKeyCount = 0; + while(volABucketAIter.hasNext()) { + volABucketAIter.next(); + volABucketAKeyCount++; + } + Assert.assertEquals(20, volABucketAKeyCount); + Iterator volABucketBIter = + volAbucketB.listKeys("key-"); + int volABucketBKeyCount = 0; + while(volABucketBIter.hasNext()) { + volABucketBIter.next(); + volABucketBKeyCount++; + } + Assert.assertEquals(20, volABucketBKeyCount); + Iterator volBBucketAIter = + volBbucketA.listKeys("key-"); + int volBBucketAKeyCount = 0; + while(volBBucketAIter.hasNext()) { + volBBucketAIter.next(); + volBBucketAKeyCount++; + } + Assert.assertEquals(20, volBBucketAKeyCount); + Iterator volBBucketBIter = + volBbucketB.listKeys("key-"); + int volBBucketBKeyCount = 0; + while(volBBucketBIter.hasNext()) { + volBBucketBIter.next(); + volBBucketBKeyCount++; + } + Assert.assertEquals(20, volBBucketBKeyCount); + Iterator volABucketAKeyAIter = + volAbucketA.listKeys("key-a-"); + int volABucketAKeyACount = 0; + while(volABucketAKeyAIter.hasNext()) { + volABucketAKeyAIter.next(); + volABucketAKeyACount++; + } + Assert.assertEquals(10, volABucketAKeyACount); + Iterator volABucketAKeyBIter = + volAbucketA.listKeys("key-b-"); + for(int i = 0; i < 10; i++) { + Assert.assertTrue(volABucketAKeyBIter.next().getName() + .startsWith("key-b-" + i + "-")); + } + Assert.assertFalse(volABucketBIter.hasNext()); + } + + @Test + public void testListKeyOnEmptyBucket() + throws IOException, OzoneException { + String volume = "vol-" + RandomStringUtils.randomNumeric(5); + String bucket = "buc-" + RandomStringUtils.randomNumeric(5); + store.createVolume(volume); + OzoneVolume vol = store.getVolume(volume); + vol.createBucket(bucket); + OzoneBucket buc = vol.getBucket(bucket); + Iterator keys = buc.listKeys(""); + while(keys.hasNext()) { + fail(); + } + } + + @Test + public void testInitiateMultipartUploadWithReplicationInformationSet() throws + IOException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, + ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + + assertNotNull(multipartInfo); + String uploadID = multipartInfo.getUploadID(); + Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); + Assert.assertEquals(bucketName, multipartInfo.getBucketName()); + Assert.assertEquals(keyName, multipartInfo.getKeyName()); + assertNotNull(multipartInfo.getUploadID()); + + // Call initiate multipart upload for the same key again, this should + // generate a new uploadID. + multipartInfo = bucket.initiateMultipartUpload(keyName, + ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + + assertNotNull(multipartInfo); + Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); + Assert.assertEquals(bucketName, multipartInfo.getBucketName()); + Assert.assertEquals(keyName, multipartInfo.getKeyName()); + assertNotEquals(multipartInfo.getUploadID(), uploadID); + assertNotNull(multipartInfo.getUploadID()); + } + + + @Test + public void testInitiateMultipartUploadWithDefaultReplication() throws + IOException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName); + + assertNotNull(multipartInfo); + String uploadID = multipartInfo.getUploadID(); + Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); + Assert.assertEquals(bucketName, multipartInfo.getBucketName()); + Assert.assertEquals(keyName, multipartInfo.getKeyName()); + assertNotNull(multipartInfo.getUploadID()); + + // Call initiate multipart upload for the same key again, this should + // generate a new uploadID. + multipartInfo = bucket.initiateMultipartUpload(keyName); + + assertNotNull(multipartInfo); + Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); + Assert.assertEquals(bucketName, multipartInfo.getBucketName()); + Assert.assertEquals(keyName, multipartInfo.getKeyName()); + assertNotEquals(multipartInfo.getUploadID(), uploadID); + assertNotNull(multipartInfo.getUploadID()); + } + + + @Test + public void testUploadPartWithNoOverride() throws IOException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + String sampleData = "sample Value"; + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, + ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + + assertNotNull(multipartInfo); + String uploadID = multipartInfo.getUploadID(); + Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); + Assert.assertEquals(bucketName, multipartInfo.getBucketName()); + Assert.assertEquals(keyName, multipartInfo.getKeyName()); + assertNotNull(multipartInfo.getUploadID()); + + OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName, + sampleData.length(), 1, uploadID); + ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, + sampleData.length()); + ozoneOutputStream.close(); + + OmMultipartCommitUploadPartInfo commitUploadPartInfo = ozoneOutputStream + .getCommitUploadPartInfo(); + + assertNotNull(commitUploadPartInfo); + String partName = commitUploadPartInfo.getPartName(); + assertNotNull(commitUploadPartInfo.getPartName()); + + } + + @Test + public void testUploadPartOverrideWithStandAlone() throws IOException { + + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + String sampleData = "sample Value"; + int partNumber = 1; + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, + ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + + assertNotNull(multipartInfo); + String uploadID = multipartInfo.getUploadID(); + Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); + Assert.assertEquals(bucketName, multipartInfo.getBucketName()); + Assert.assertEquals(keyName, multipartInfo.getKeyName()); + assertNotNull(multipartInfo.getUploadID()); + + OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName, + sampleData.length(), partNumber, uploadID); + ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, + sampleData.length()); + ozoneOutputStream.close(); + + OmMultipartCommitUploadPartInfo commitUploadPartInfo = ozoneOutputStream + .getCommitUploadPartInfo(); + + assertNotNull(commitUploadPartInfo); + String partName = commitUploadPartInfo.getPartName(); + assertNotNull(commitUploadPartInfo.getPartName()); + + //Overwrite the part by creating part key with same part number. + sampleData = "sample Data Changed"; + ozoneOutputStream = bucket.createMultipartKey(keyName, + sampleData.length(), partNumber, uploadID); + ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, "name" + .length()); + ozoneOutputStream.close(); + + commitUploadPartInfo = ozoneOutputStream + .getCommitUploadPartInfo(); + + assertNotNull(commitUploadPartInfo); + assertNotNull(commitUploadPartInfo.getPartName()); + + // PartName should be different from old part Name. + assertNotEquals("Part names should be different", partName, + commitUploadPartInfo.getPartName()); + } + + @Test + public void testUploadPartOverrideWithRatis() throws IOException { + + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + String sampleData = "sample Value"; + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, + ReplicationType.RATIS, ReplicationFactor.THREE); + + assertNotNull(multipartInfo); + String uploadID = multipartInfo.getUploadID(); + Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); + Assert.assertEquals(bucketName, multipartInfo.getBucketName()); + Assert.assertEquals(keyName, multipartInfo.getKeyName()); + assertNotNull(multipartInfo.getUploadID()); + + int partNumber = 1; + + OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName, + sampleData.length(), partNumber, uploadID); + ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, + sampleData.length()); + ozoneOutputStream.close(); + + OmMultipartCommitUploadPartInfo commitUploadPartInfo = ozoneOutputStream + .getCommitUploadPartInfo(); + + assertNotNull(commitUploadPartInfo); + String partName = commitUploadPartInfo.getPartName(); + assertNotNull(commitUploadPartInfo.getPartName()); + + //Overwrite the part by creating part key with same part number. + sampleData = "sample Data Changed"; + ozoneOutputStream = bucket.createMultipartKey(keyName, + sampleData.length(), partNumber, uploadID); + ozoneOutputStream.write(DFSUtil.string2Bytes(sampleData), 0, "name" + .length()); + ozoneOutputStream.close(); + + commitUploadPartInfo = ozoneOutputStream + .getCommitUploadPartInfo(); + + assertNotNull(commitUploadPartInfo); + assertNotNull(commitUploadPartInfo.getPartName()); + + // PartName should be different from old part Name. + assertNotEquals("Part names should be different", partName, + commitUploadPartInfo.getPartName()); + } + + @Test + public void testNoSuchUploadError() throws IOException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + String sampleData = "sample Value"; + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + String uploadID = "random"; + try { + bucket.createMultipartKey(keyName, sampleData.length(), 1, uploadID); + fail("testNoSuchUploadError failed"); + } catch (IOException ex) { + GenericTestUtils.assertExceptionContains("NO_SUCH_MULTIPART_UPLOAD_ERROR", + ex); + } + } + + @Test + public void testMultipartUpload() throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + doMultipartUpload(bucket, keyName, (byte)98); + + } + + + @Test + public void testMultipartUploadOverride() throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + doMultipartUpload(bucket, keyName, (byte)96); + + // Initiate Multipart upload again, now we should read latest version, as + // read always reads latest blocks. + doMultipartUpload(bucket, keyName, (byte)97); + + } + + + @Test + public void testMultipartUploadWithPartsLessThanMinSize() throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + // Initiate multipart upload + String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType + .STAND_ALONE, ReplicationFactor.ONE); + + // Upload Parts + Map partsMap = new TreeMap<>(); + // Uploading part 1 with less than min size + String partName = uploadPart(bucket, keyName, uploadID, 1, "data".getBytes( + "UTF-8")); + partsMap.put(1, partName); + + partName = uploadPart(bucket, keyName, uploadID, 2, "data".getBytes( + "UTF-8")); + partsMap.put(2, partName); + + + // Complete multipart upload + + try { + completeMultipartUpload(bucket, keyName, uploadID, partsMap); + fail("testMultipartUploadWithPartsLessThanMinSize failed"); + } catch (IOException ex) { + GenericTestUtils.assertExceptionContains("ENTITY_TOO_SMALL", ex); + } + + } + + + + @Test + public void testMultipartUploadWithPartsMisMatchWithListSizeDifferent() + throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType + .STAND_ALONE, ReplicationFactor.ONE); + + // We have not uploaded any parts, but passing some list it should throw + // error. + TreeMap partsMap = new TreeMap<>(); + partsMap.put(1, UUID.randomUUID().toString()); + + try { + completeMultipartUpload(bucket, keyName, uploadID, partsMap); + fail("testMultipartUploadWithPartsMisMatch"); + } catch (IOException ex) { + GenericTestUtils.assertExceptionContains("MISMATCH_MULTIPART_LIST", ex); + } + + } + + @Test + public void testMultipartUploadWithPartsMisMatchWithIncorrectPartName() + throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType + .STAND_ALONE, ReplicationFactor.ONE); + + uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8")); + // We have not uploaded any parts, but passing some list it should throw + // error. + TreeMap partsMap = new TreeMap<>(); + partsMap.put(1, UUID.randomUUID().toString()); + + try { + completeMultipartUpload(bucket, keyName, uploadID, partsMap); + fail("testMultipartUploadWithPartsMisMatch"); + } catch (IOException ex) { + GenericTestUtils.assertExceptionContains("MISMATCH_MULTIPART_LIST", ex); + } + + } + + @Test + public void testMultipartUploadWithMissingParts() throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType + .STAND_ALONE, ReplicationFactor.ONE); + + uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8")); + // We have not uploaded any parts, but passing some list it should throw + // error. + TreeMap partsMap = new TreeMap<>(); + partsMap.put(3, "random"); + + try { + completeMultipartUpload(bucket, keyName, uploadID, partsMap); + fail("testMultipartUploadWithPartsMisMatch"); + } catch (IOException ex) { + GenericTestUtils.assertExceptionContains("MISSING_UPLOAD_PARTS", ex); + } + } + + @Test + public void testAbortUploadFail() throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + try { + bucket.abortMultipartUpload(keyName, "random"); + fail("testAbortUploadFail failed"); + } catch (IOException ex) { + GenericTestUtils.assertExceptionContains( + "NO_SUCH_MULTIPART_UPLOAD_ERROR", ex); + } + } + + + @Test + public void testAbortUploadSuccessWithOutAnyParts() throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + try { + String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType + .STAND_ALONE, ReplicationFactor.ONE); + bucket.abortMultipartUpload(keyName, uploadID); + } catch (IOException ex) { + fail("testAbortUploadSuccess failed"); + } + } + + @Test + public void testAbortUploadSuccessWithParts() throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + try { + String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType + .STAND_ALONE, ReplicationFactor.ONE); + uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8")); + bucket.abortMultipartUpload(keyName, uploadID); + } catch (IOException ex) { + fail("testAbortUploadSuccess failed"); + } + } + + + private byte[] generateData(int size, byte val) { + byte[] chars = new byte[size]; + Arrays.fill(chars, val); + return chars; + } + + + private void doMultipartUpload(OzoneBucket bucket, String keyName, byte val) + throws Exception { + // Initiate Multipart upload request + String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType + .RATIS, ReplicationFactor.THREE); + + // Upload parts + Map partsMap = new TreeMap<>(); + + // get 5mb data, as each part should be of min 5mb, last part can be less + // than 5mb + int length = 0; + byte[] data = generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, val); + String partName = uploadPart(bucket, keyName, uploadID, 1, data); + partsMap.put(1, partName); + length += data.length; + + + partName = uploadPart(bucket, keyName, uploadID, 2, data); + partsMap.put(2, partName); + length += data.length; + + String part3 = UUID.randomUUID().toString(); + partName = uploadPart(bucket, keyName, uploadID, 3, part3.getBytes( + "UTF-8")); + partsMap.put(3, partName); + length += part3.getBytes("UTF-8").length; + + + // Complete multipart upload request + completeMultipartUpload(bucket, keyName, uploadID, partsMap); + + + //Now Read the key which has been completed multipart upload. + byte[] fileContent = new byte[data.length + data.length + part3.getBytes( + "UTF-8").length]; + OzoneInputStream inputStream = bucket.readKey(keyName); + inputStream.read(fileContent); + + Assert.assertTrue(verifyRatisReplication(bucket.getVolumeName(), + bucket.getName(), keyName, ReplicationType.RATIS, + ReplicationFactor.THREE)); + + StringBuilder sb = new StringBuilder(length); + + // Combine all parts data, and check is it matching with get key data. + String part1 = new String(data); + String part2 = new String(data); + sb.append(part1); + sb.append(part2); + sb.append(part3); + Assert.assertEquals(sb.toString(), new String(fileContent)); + } + + + private String initiateMultipartUpload(OzoneBucket bucket, String keyName, + ReplicationType replicationType, ReplicationFactor replicationFactor) + throws Exception { + OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, + replicationType, replicationFactor); + + String uploadID = multipartInfo.getUploadID(); + Assert.assertNotNull(uploadID); + return uploadID; + } + + private String uploadPart(OzoneBucket bucket, String keyName, String + uploadID, int partNumber, byte[] data) throws Exception { + OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName, + data.length, partNumber, uploadID); + ozoneOutputStream.write(data, 0, + data.length); + ozoneOutputStream.close(); + + OmMultipartCommitUploadPartInfo omMultipartCommitUploadPartInfo = + ozoneOutputStream.getCommitUploadPartInfo(); + + Assert.assertNotNull(omMultipartCommitUploadPartInfo); + Assert.assertNotNull(omMultipartCommitUploadPartInfo.getPartName()); + return omMultipartCommitUploadPartInfo.getPartName(); + + } + + private void completeMultipartUpload(OzoneBucket bucket, String keyName, + String uploadID, Map partsMap) throws Exception { + OmMultipartUploadCompleteInfo omMultipartUploadCompleteInfo = bucket + .completeMultipartUpload(keyName, uploadID, partsMap); + + Assert.assertNotNull(omMultipartUploadCompleteInfo); + Assert.assertEquals(omMultipartUploadCompleteInfo.getBucket(), bucket + .getName()); + Assert.assertEquals(omMultipartUploadCompleteInfo.getVolume(), bucket + .getVolumeName()); + Assert.assertEquals(omMultipartUploadCompleteInfo.getKey(), keyName); + Assert.assertNotNull(omMultipartUploadCompleteInfo.getHash()); + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java new file mode 100644 index 0000000000..83d8fa8096 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.client.rpc; + +import java.io.IOException; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.ScmConfigKeys; +import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +/** + * This class is to test all the public facing APIs of Ozone Client with an + * active OM Ratis server. + */ +public class TestOzoneRpcClientWithRatis extends TestOzoneRpcClientAbstract { + + /** + * Create a MiniOzoneCluster for testing. + * Ozone is made active by setting OZONE_ENABLED = true. + * Ozone OM Ratis server is made active by setting + * OZONE_OM_RATIS_ENABLE = true; + * + * @throws IOException + */ + @BeforeClass + public static void init() throws Exception { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 1); + conf.setBoolean(OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY, true); + startCluster(conf); + } + + /** + * Close OzoneClient and shutdown MiniOzoneCluster. + */ + @AfterClass + public static void shutdown() throws IOException { + shutdownCluster(); + } + +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index d8352471c5..36c14f96c9 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -72,10 +72,8 @@ import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB; import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisClient; import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type; import org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB; import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType; @@ -602,8 +600,8 @@ public void start() throws IOException { OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY, OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT); if (omRatisEnabled) { - omRatisServer = OzoneManagerRatisServer.newOMRatisServer( - omId, omRpcAddress.getAddress(), configuration); + omRatisServer = OzoneManagerRatisServer.newOMRatisServer(this, omId, + omNodeRpcAddr.getAddress(), configuration); omRatisServer.start(); LOG.info("OzoneManager Ratis server started at port {}", @@ -704,22 +702,6 @@ public void join() { } } - /** - * Validates that the incoming OM request has required parameters. - * TODO: Add more validation checks before writing the request to Ratis log. - * @param omRequest client request to OM - * @throws OMException thrown if required parameters are set to null. - */ - public void validateRequest(OMRequest omRequest) throws OMException { - Type cmdType = omRequest.getCmdType(); - if (cmdType == null) { - throw new OMException("CmdType is null", ResultCodes.INVALID_REQUEST); - } - if (omRequest.getClientId() == null) { - throw new OMException("ClientId is null", ResultCodes.INVALID_REQUEST); - } - } - /** * Creates a volume. * diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisHelper.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisHelper.java index bc861acc0e..ee1fee6d9b 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisHelper.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisHelper.java @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.om.ratis; import com.google.protobuf.InvalidProtocolBufferException; -import java.util.concurrent.CompletableFuture; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.ozone.om.OMConfigKeys; @@ -31,6 +30,7 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.GrpcConfigKeys; +import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.retry.RetryPolicy; @@ -44,7 +44,6 @@ * Ratis helper methods for OM Ratis server and client. */ public final class OMRatisHelper { - private static final Logger LOG = LoggerFactory.getLogger( OMRatisHelper.class); @@ -95,9 +94,9 @@ static OMRequest convertByteStringToOMRequest(ByteString byteString) return OMRequest.parseFrom(bytes); } - static ByteString convertResponseToByteString(OMResponse response) { + static Message convertResponseToMessage(OMResponse response) { byte[] requestBytes = response.toByteArray(); - return ByteString.copyFrom(requestBytes); + return Message.valueOf(ByteString.copyFrom(requestBytes)); } static OMResponse convertByteStringToOMResponse(ByteString byteString) @@ -113,10 +112,4 @@ static OMResponse getErrorResponse(Type cmdType, Exception e) { .setMessage(e.getMessage()) .build(); } - - static CompletableFuture completeExceptionally(Exception e) { - final CompletableFuture future = new CompletableFuture<>(); - future.completeExceptionally(e); - return future; - } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java index f28f2ce529..d49d5e6a3b 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java @@ -35,6 +35,7 @@ import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; import org.apache.ratis.RaftConfigKeys; import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; @@ -68,6 +69,7 @@ public final class OzoneManagerRatisServer { private final RaftGroupId raftGroupId; private final RaftGroup raftGroup; private final RaftPeerId raftPeerId; + private final OzoneManagerProtocol ozoneManager; private static final AtomicLong CALL_ID_COUNTER = new AtomicLong(); @@ -75,9 +77,10 @@ private static long nextCallId() { return CALL_ID_COUNTER.getAndIncrement() & Long.MAX_VALUE; } - private OzoneManagerRatisServer(String omId, InetAddress addr, int port, - Configuration conf) throws IOException { - Objects.requireNonNull(omId, "omId == null"); + private OzoneManagerRatisServer(OzoneManagerProtocol om, String omId, + InetAddress addr, int port, Configuration conf) throws IOException { + Objects.requireNonNull(omId, "omId is null"); + this.ozoneManager = om; this.port = port; this.omRatisAddress = new InetSocketAddress(addr.getHostAddress(), port); RaftProperties serverProperties = newRaftProperties(conf); @@ -98,8 +101,9 @@ private OzoneManagerRatisServer(String omId, InetAddress addr, int port, .build(); } - public static OzoneManagerRatisServer newOMRatisServer(String omId, - InetAddress omAddress, Configuration ozoneConf) throws IOException { + public static OzoneManagerRatisServer newOMRatisServer( + OzoneManagerProtocol om, String omId, InetAddress omAddress, + Configuration ozoneConf) throws IOException { int localPort = ozoneConf.getInt( OMConfigKeys.OZONE_OM_RATIS_PORT_KEY, OMConfigKeys.OZONE_OM_RATIS_PORT_DEFAULT); @@ -120,7 +124,8 @@ public static OzoneManagerRatisServer newOMRatisServer(String omId, + "fallback to use default port {}", localPort, e); } } - return new OzoneManagerRatisServer(omId, omAddress, localPort, ozoneConf); + return new OzoneManagerRatisServer(om, omId, omAddress, localPort, + ozoneConf); } public RaftGroup getRaftGroup() { @@ -128,11 +133,10 @@ public RaftGroup getRaftGroup() { } /** - * Return a dummy StateMachine. - * TODO: Implement a state machine on OM. + * Returns OzoneManager StateMachine. */ private BaseStateMachine getStateMachine(RaftGroupId gid) { - return new OzoneManagerStateMachine(null); + return new OzoneManagerStateMachine(ozoneManager); } public void start() throws IOException { @@ -199,8 +203,7 @@ private RaftProperties newRaftProperties(Configuration conf) { SizeInBytes.valueOf(raftSegmentPreallocatedSize)); // For grpc set the maximum message size - // TODO: calculate the max message size based on the max size of a - // PutSmallFileRequest's file size limit + // TODO: calculate the optimal max message size GrpcConfigKeys.setMessageSizeMax(properties, SizeInBytes.valueOf(logAppenderQueueByteLimit)); @@ -263,11 +266,6 @@ private RaftProperties newRaftProperties(Configuration conf) { // TODO: set max write buffer size - /** - * TODO: when state machine is implemented, enable StateMachineData sync - * and set sync timeout and number of sync retries. - */ - /** * TODO: set following ratis leader election related configs when * replicated ratis server is implemented. diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java index 5ea0b49530..701ac16de1 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java @@ -17,14 +17,22 @@ package org.apache.hadoop.ozone.om.ratis; -import com.google.protobuf.InvalidProtocolBufferException; +import com.google.common.base.Preconditions; +import com.google.protobuf.ServiceException; import java.io.IOException; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.ozone.container.common.transport.server.ratis .ContainerStateMachine; import org.apache.hadoop.ozone.om.OzoneManager; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .OMRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .OMResponse; +import org.apache.hadoop.ozone.protocolPB.OzoneManagerRequestHandler; +import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.Message; +import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.storage.RaftStorage; @@ -46,11 +54,11 @@ public class OzoneManagerStateMachine extends BaseStateMachine { LoggerFactory.getLogger(ContainerStateMachine.class); private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage(); - private final OzoneManager ozoneManager; + private final OzoneManagerRequestHandler handler; + private RaftGroupId raftGroupId; - public OzoneManagerStateMachine(OzoneManager om) { - // OzoneManager is required when implementing StateMachine - this.ozoneManager = om; + public OzoneManagerStateMachine(OzoneManagerProtocol om) { + this.handler = new OzoneManagerRequestHandler(om); } /** @@ -62,29 +70,88 @@ public void initialize( RaftServer server, RaftGroupId id, RaftStorage raftStorage) throws IOException { super.initialize(server, id, raftStorage); + this.raftGroupId = id; storage.init(raftStorage); } + /** + * Validate/pre-process the incoming update request in the state machine. + * @return the content to be written to the log entry. Null means the request + * should be rejected. + * @throws IOException thrown by the state machine while validating + */ + public TransactionContext startTransaction( + RaftClientRequest raftClientRequest) throws IOException { + ByteString messageContent = raftClientRequest.getMessage().getContent(); + OMRequest omRequest = OMRatisHelper.convertByteStringToOMRequest( + messageContent); + + Preconditions.checkArgument(raftClientRequest.getRaftGroupId().equals( + raftGroupId)); + try { + handler.validateRequest(omRequest); + } catch (IOException ioe) { + TransactionContext ctxt = TransactionContext.newBuilder() + .setClientRequest(raftClientRequest) + .setStateMachine(this) + .setServerRole(RaftProtos.RaftPeerRole.LEADER) + .build(); + ctxt.setException(ioe); + return ctxt; + } + return TransactionContext.newBuilder() + .setClientRequest(raftClientRequest) + .setStateMachine(this) + .setServerRole(RaftProtos.RaftPeerRole.LEADER) + .setLogData(messageContent) + .build(); + } + /* - * Apply a committed log entry to the state machine. This function - * currently returns a dummy message. - * TODO: Apply transaction to OM state machine + * Apply a committed log entry to the state machine. */ @Override public CompletableFuture applyTransaction(TransactionContext trx) { - String errorMessage; - ByteString logData = trx.getStateMachineLogEntry().getLogData(); try { - OMRequest omRequest = OMRatisHelper.convertByteStringToOMRequest(logData); - LOG.debug("Received request: cmdType={} traceID={} ", - omRequest.getCmdType(), omRequest.getTraceID()); - errorMessage = "Dummy response from Ratis server for command type: " + - omRequest.getCmdType(); - } catch (InvalidProtocolBufferException e) { - errorMessage = e.getMessage(); + OMRequest request = OMRatisHelper.convertByteStringToOMRequest( + trx.getStateMachineLogEntry().getLogData()); + CompletableFuture future = CompletableFuture + .supplyAsync(() -> runCommand(request)); + return future; + } catch (IOException e) { + return completeExceptionally(e); } - - // TODO: When State Machine is implemented, send the actual response back - return OMRatisHelper.completeExceptionally(new IOException(errorMessage)); } + + /** + * Query the state machine. The request must be read-only. + */ + @Override + public CompletableFuture query(Message request) { + try { + OMRequest omRequest = OMRatisHelper.convertByteStringToOMRequest( + request.getContent()); + return CompletableFuture.completedFuture(runCommand(omRequest)); + } catch (IOException e) { + return completeExceptionally(e); + } + } + + /** + * Submits request to OM and returns the response Message. + * @param request OMRequest + * @return response from OM + * @throws ServiceException + */ + private Message runCommand(OMRequest request) { + OMResponse response = handler.handle(request); + return OMRatisHelper.convertResponseToMessage(response); + } + + private static CompletableFuture completeExceptionally(Exception e) { + final CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(e); + return future; + } + } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java index ca54bae407..aff879591a 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java @@ -16,153 +16,20 @@ */ package org.apache.hadoop.ozone.protocolPB; -import com.google.common.collect.Lists; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.ozone.om.exceptions.OMException; -import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; -import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; -import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; -import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; -import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; -import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo; -import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo; -import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo; -import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList; -import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; -import org.apache.hadoop.ozone.om.helpers.OpenKeySession; -import org.apache.hadoop.ozone.om.helpers.ServiceInfo; + import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB; import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisClient; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .AllocateBlockRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .AllocateBlockResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CheckVolumeAccessRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CheckVolumeAccessResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CommitKeyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CommitKeyResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CreateBucketRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CreateBucketResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CreateKeyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CreateKeyResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CreateVolumeRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CreateVolumeResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .DeleteBucketRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .DeleteBucketResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .DeleteKeyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .DeleteKeyResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .DeleteVolumeRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .DeleteVolumeResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .InfoBucketRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .InfoBucketResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .InfoVolumeRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .InfoVolumeResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .KeyArgs; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .ListBucketsRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .ListBucketsResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .ListKeysRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .ListKeysResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .ListVolumeRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .ListVolumeResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .LookupKeyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .LookupKeyResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos. - MultipartUploadAbortRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .MultipartUploadAbortResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .MultipartCommitUploadPartRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .MultipartCommitUploadPartResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .MultipartUploadCompleteRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .MultipartUploadCompleteResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .MultipartInfoInitiateRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .MultipartInfoInitiateResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Part; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .RenameKeyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .RenameKeyResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .S3BucketInfoRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .S3BucketInfoResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .S3CreateBucketRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .S3CreateBucketResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .S3DeleteBucketRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .S3DeleteBucketResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .S3ListBucketsResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .S3ListBucketsRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .ServiceListRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .ServiceListResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .SetBucketPropertyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .SetBucketPropertyResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .SetVolumePropertyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .SetVolumePropertyResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .Status; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .Type; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; -import java.util.TreeMap; -import java.util.stream.Collectors; - /** * This class is the server-side translator that forwards requests received on * {@link OzoneManagerProtocolPB} @@ -172,8 +39,8 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements OzoneManagerProtocolPB { private static final Logger LOG = LoggerFactory .getLogger(OzoneManagerProtocolServerSideTranslatorPB.class); - private final OzoneManagerProtocol impl; private final OzoneManagerRatisClient omRatisClient; + private final OzoneManagerRequestHandler handler; private final boolean isRatisEnabled; /** @@ -184,7 +51,7 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements public OzoneManagerProtocolServerSideTranslatorPB( OzoneManagerProtocol impl, OzoneManagerRatisClient ratisClient, boolean enableRatis) { - this.impl = impl; + handler = new OzoneManagerRequestHandler(impl); this.omRatisClient = ratisClient; this.isRatisEnabled = enableRatis; } @@ -200,7 +67,7 @@ public OMResponse submitRequest(RpcController controller, if (isRatisEnabled) { return submitRequestToRatis(request); } else { - return submitRequestToOM(request); + return submitRequestDirectlyToOM(request); } } @@ -214,738 +81,7 @@ private OMResponse submitRequestToRatis(OMRequest request) { /** * Submits request directly to OM. */ - @SuppressWarnings("methodlength") - private OMResponse submitRequestToOM(OMRequest request) - throws ServiceException { - Type cmdType = request.getCmdType(); - OMResponse.Builder responseBuilder = OMResponse.newBuilder() - .setCmdType(cmdType); - - switch (cmdType) { - case CreateVolume: - CreateVolumeResponse createVolumeResponse = createVolume( - request.getCreateVolumeRequest()); - responseBuilder.setCreateVolumeResponse(createVolumeResponse); - break; - case SetVolumeProperty: - SetVolumePropertyResponse setVolumePropertyResponse = setVolumeProperty( - request.getSetVolumePropertyRequest()); - responseBuilder.setSetVolumePropertyResponse(setVolumePropertyResponse); - break; - case CheckVolumeAccess: - CheckVolumeAccessResponse checkVolumeAccessResponse = checkVolumeAccess( - request.getCheckVolumeAccessRequest()); - responseBuilder.setCheckVolumeAccessResponse(checkVolumeAccessResponse); - break; - case InfoVolume: - InfoVolumeResponse infoVolumeResponse = infoVolume( - request.getInfoVolumeRequest()); - responseBuilder.setInfoVolumeResponse(infoVolumeResponse); - break; - case DeleteVolume: - DeleteVolumeResponse deleteVolumeResponse = deleteVolume( - request.getDeleteVolumeRequest()); - responseBuilder.setDeleteVolumeResponse(deleteVolumeResponse); - break; - case ListVolume: - ListVolumeResponse listVolumeResponse = listVolumes( - request.getListVolumeRequest()); - responseBuilder.setListVolumeResponse(listVolumeResponse); - break; - case CreateBucket: - CreateBucketResponse createBucketResponse = createBucket( - request.getCreateBucketRequest()); - responseBuilder.setCreateBucketResponse(createBucketResponse); - break; - case InfoBucket: - InfoBucketResponse infoBucketResponse = infoBucket( - request.getInfoBucketRequest()); - responseBuilder.setInfoBucketResponse(infoBucketResponse); - break; - case SetBucketProperty: - SetBucketPropertyResponse setBucketPropertyResponse = setBucketProperty( - request.getSetBucketPropertyRequest()); - responseBuilder.setSetBucketPropertyResponse(setBucketPropertyResponse); - break; - case DeleteBucket: - DeleteBucketResponse deleteBucketResponse = deleteBucket( - request.getDeleteBucketRequest()); - responseBuilder.setDeleteBucketResponse(deleteBucketResponse); - break; - case ListBuckets: - ListBucketsResponse listBucketsResponse = listBuckets( - request.getListBucketsRequest()); - responseBuilder.setListBucketsResponse(listBucketsResponse); - break; - case CreateKey: - CreateKeyResponse createKeyResponse = createKey( - request.getCreateKeyRequest()); - responseBuilder.setCreateKeyResponse(createKeyResponse); - break; - case LookupKey: - LookupKeyResponse lookupKeyResponse = lookupKey( - request.getLookupKeyRequest()); - responseBuilder.setLookupKeyResponse(lookupKeyResponse); - break; - case RenameKey: - RenameKeyResponse renameKeyResponse = renameKey( - request.getRenameKeyRequest()); - responseBuilder.setRenameKeyResponse(renameKeyResponse); - break; - case DeleteKey: - DeleteKeyResponse deleteKeyResponse = deleteKey( - request.getDeleteKeyRequest()); - responseBuilder.setDeleteKeyResponse(deleteKeyResponse); - break; - case ListKeys: - ListKeysResponse listKeysResponse = listKeys( - request.getListKeysRequest()); - responseBuilder.setListKeysResponse(listKeysResponse); - break; - case CommitKey: - CommitKeyResponse commitKeyResponse = commitKey( - request.getCommitKeyRequest()); - responseBuilder.setCommitKeyResponse(commitKeyResponse); - break; - case AllocateBlock: - AllocateBlockResponse allocateBlockResponse = allocateBlock( - request.getAllocateBlockRequest()); - responseBuilder.setAllocateBlockResponse(allocateBlockResponse); - break; - case CreateS3Bucket: - S3CreateBucketResponse s3CreateBucketResponse = createS3Bucket( - request.getCreateS3BucketRequest()); - responseBuilder.setCreateS3BucketResponse(s3CreateBucketResponse); - break; - case DeleteS3Bucket: - S3DeleteBucketResponse s3DeleteBucketResponse = deleteS3Bucket( - request.getDeleteS3BucketRequest()); - responseBuilder.setDeleteS3BucketResponse(s3DeleteBucketResponse); - break; - case InfoS3Bucket: - S3BucketInfoResponse s3BucketInfoResponse = getS3Bucketinfo( - request.getInfoS3BucketRequest()); - responseBuilder.setInfoS3BucketResponse(s3BucketInfoResponse); - break; - case ListS3Buckets: - S3ListBucketsResponse s3ListBucketsResponse = listS3Buckets( - request.getListS3BucketsRequest()); - responseBuilder.setListS3BucketsResponse(s3ListBucketsResponse); - break; - case InitiateMultiPartUpload: - MultipartInfoInitiateResponse multipartInfoInitiateResponse = - initiateMultiPartUpload(request.getInitiateMultiPartUploadRequest()); - responseBuilder.setInitiateMultiPartUploadResponse( - multipartInfoInitiateResponse); - break; - case CommitMultiPartUpload: - MultipartCommitUploadPartResponse commitUploadPartResponse = - commitMultipartUploadPart(request.getCommitMultiPartUploadRequest()); - responseBuilder.setCommitMultiPartUploadResponse( - commitUploadPartResponse); - break; - case CompleteMultiPartUpload: - MultipartUploadCompleteResponse completeMultipartUploadResponse = - completeMultipartUpload( - request.getCompleteMultiPartUploadRequest()); - responseBuilder.setCompleteMultiPartUploadResponse( - completeMultipartUploadResponse); - break; - case AbortMultiPartUpload: - MultipartUploadAbortResponse multipartUploadAbortResponse = - abortMultipartUpload(request.getAbortMultiPartUploadRequest()); - responseBuilder.setAbortMultiPartUploadResponse( - multipartUploadAbortResponse); - break; - case ServiceList: - ServiceListResponse serviceListResponse = getServiceList( - request.getServiceListRequest()); - responseBuilder.setServiceListResponse(serviceListResponse); - break; - default: - responseBuilder.setSuccess(false); - responseBuilder.setMessage("Unrecognized Command Type: " + cmdType); - break; - } - return responseBuilder.build(); + private OMResponse submitRequestDirectlyToOM(OMRequest request) { + return handler.handle(request); } - // Convert and exception to corresponding status code - private Status exceptionToResponseStatus(IOException ex) { - if (ex instanceof OMException) { - OMException omException = (OMException)ex; - switch (omException.getResult()) { - case FAILED_VOLUME_ALREADY_EXISTS: - return Status.VOLUME_ALREADY_EXISTS; - case FAILED_TOO_MANY_USER_VOLUMES: - return Status.USER_TOO_MANY_VOLUMES; - case FAILED_VOLUME_NOT_FOUND: - return Status.VOLUME_NOT_FOUND; - case FAILED_VOLUME_NOT_EMPTY: - return Status.VOLUME_NOT_EMPTY; - case FAILED_USER_NOT_FOUND: - return Status.USER_NOT_FOUND; - case FAILED_BUCKET_ALREADY_EXISTS: - return Status.BUCKET_ALREADY_EXISTS; - case FAILED_BUCKET_NOT_FOUND: - return Status.BUCKET_NOT_FOUND; - case FAILED_BUCKET_NOT_EMPTY: - return Status.BUCKET_NOT_EMPTY; - case FAILED_KEY_ALREADY_EXISTS: - return Status.KEY_ALREADY_EXISTS; - case FAILED_KEY_NOT_FOUND: - return Status.KEY_NOT_FOUND; - case FAILED_INVALID_KEY_NAME: - return Status.INVALID_KEY_NAME; - case FAILED_KEY_ALLOCATION: - return Status.KEY_ALLOCATION_ERROR; - case FAILED_KEY_DELETION: - return Status.KEY_DELETION_ERROR; - case FAILED_KEY_RENAME: - return Status.KEY_RENAME_ERROR; - case FAILED_METADATA_ERROR: - return Status.METADATA_ERROR; - case OM_NOT_INITIALIZED: - return Status.OM_NOT_INITIALIZED; - case SCM_VERSION_MISMATCH_ERROR: - return Status.SCM_VERSION_MISMATCH_ERROR; - case S3_BUCKET_ALREADY_EXISTS: - return Status.S3_BUCKET_ALREADY_EXISTS; - case S3_BUCKET_NOT_FOUND: - return Status.S3_BUCKET_NOT_FOUND; - case INITIATE_MULTIPART_UPLOAD_FAILED: - return Status.INITIATE_MULTIPART_UPLOAD_ERROR; - case NO_SUCH_MULTIPART_UPLOAD: - return Status.NO_SUCH_MULTIPART_UPLOAD_ERROR; - case UPLOAD_PART_FAILED: - return Status.MULTIPART_UPLOAD_PARTFILE_ERROR; - case COMPLETE_MULTIPART_UPLOAD_FAILED: - return Status.COMPLETE_MULTIPART_UPLOAD_ERROR; - case MISMATCH_MULTIPART_LIST: - return Status.MISMATCH_MULTIPART_LIST; - case MISSING_UPLOAD_PARTS: - return Status.MISSING_UPLOAD_PARTS; - case ENTITY_TOO_SMALL: - return Status.ENTITY_TOO_SMALL; - case ABORT_MULTIPART_UPLOAD_FAILED: - return Status.ABORT_MULTIPART_UPLOAD_FAILED; - default: - return Status.INTERNAL_ERROR; - } - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Unknown error occurs", ex); - } - return Status.INTERNAL_ERROR; - } - } - - private CreateVolumeResponse createVolume(CreateVolumeRequest request) { - CreateVolumeResponse.Builder resp = CreateVolumeResponse.newBuilder(); - resp.setStatus(Status.OK); - try { - impl.createVolume(OmVolumeArgs.getFromProtobuf(request.getVolumeInfo())); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private SetVolumePropertyResponse setVolumeProperty( - SetVolumePropertyRequest request) { - SetVolumePropertyResponse.Builder resp = - SetVolumePropertyResponse.newBuilder(); - resp.setStatus(Status.OK); - String volume = request.getVolumeName(); - - try { - if (request.hasQuotaInBytes()) { - long quota = request.getQuotaInBytes(); - impl.setQuota(volume, quota); - } else { - String owner = request.getOwnerName(); - impl.setOwner(volume, owner); - } - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private CheckVolumeAccessResponse checkVolumeAccess( - CheckVolumeAccessRequest request) { - CheckVolumeAccessResponse.Builder resp = - CheckVolumeAccessResponse.newBuilder(); - resp.setStatus(Status.OK); - try { - boolean access = impl.checkVolumeAccess(request.getVolumeName(), - request.getUserAcl()); - // if no access, set the response status as access denied - if (!access) { - resp.setStatus(Status.ACCESS_DENIED); - } - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - - return resp.build(); - } - - private InfoVolumeResponse infoVolume(InfoVolumeRequest request) { - InfoVolumeResponse.Builder resp = InfoVolumeResponse.newBuilder(); - resp.setStatus(Status.OK); - String volume = request.getVolumeName(); - try { - OmVolumeArgs ret = impl.getVolumeInfo(volume); - resp.setVolumeInfo(ret.getProtobuf()); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private DeleteVolumeResponse deleteVolume(DeleteVolumeRequest request) { - DeleteVolumeResponse.Builder resp = DeleteVolumeResponse.newBuilder(); - resp.setStatus(Status.OK); - try { - impl.deleteVolume(request.getVolumeName()); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private ListVolumeResponse listVolumes(ListVolumeRequest request) - throws ServiceException { - ListVolumeResponse.Builder resp = ListVolumeResponse.newBuilder(); - List result = Lists.newArrayList(); - try { - if (request.getScope() - == ListVolumeRequest.Scope.VOLUMES_BY_USER) { - result = impl.listVolumeByUser(request.getUserName(), - request.getPrefix(), request.getPrevKey(), request.getMaxKeys()); - } else if (request.getScope() - == ListVolumeRequest.Scope.VOLUMES_BY_CLUSTER) { - result = impl.listAllVolumes(request.getPrefix(), request.getPrevKey(), - request.getMaxKeys()); - } - - if (result == null) { - throw new ServiceException("Failed to get volumes for given scope " - + request.getScope()); - } - - result.forEach(item -> resp.addVolumeInfo(item.getProtobuf())); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private CreateBucketResponse createBucket(CreateBucketRequest request) { - CreateBucketResponse.Builder resp = - CreateBucketResponse.newBuilder(); - try { - impl.createBucket(OmBucketInfo.getFromProtobuf( - request.getBucketInfo())); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private InfoBucketResponse infoBucket(InfoBucketRequest request) { - InfoBucketResponse.Builder resp = - InfoBucketResponse.newBuilder(); - try { - OmBucketInfo omBucketInfo = impl.getBucketInfo( - request.getVolumeName(), request.getBucketName()); - resp.setStatus(Status.OK); - resp.setBucketInfo(omBucketInfo.getProtobuf()); - } catch(IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private CreateKeyResponse createKey(CreateKeyRequest request) { - CreateKeyResponse.Builder resp = - CreateKeyResponse.newBuilder(); - try { - KeyArgs keyArgs = request.getKeyArgs(); - HddsProtos.ReplicationType type = - keyArgs.hasType()? keyArgs.getType() : null; - HddsProtos.ReplicationFactor factor = - keyArgs.hasFactor()? keyArgs.getFactor() : null; - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .setDataSize(keyArgs.getDataSize()) - .setType(type) - .setFactor(factor) - .setIsMultipartKey(keyArgs.getIsMultipartKey()) - .setMultipartUploadID(keyArgs.getMultipartUploadID()) - .setMultipartUploadPartNumber(keyArgs.getMultipartNumber()) - .build(); - if (keyArgs.hasDataSize()) { - omKeyArgs.setDataSize(keyArgs.getDataSize()); - } else { - omKeyArgs.setDataSize(0); - } - OpenKeySession openKey = impl.openKey(omKeyArgs); - resp.setKeyInfo(openKey.getKeyInfo().getProtobuf()); - resp.setID(openKey.getId()); - resp.setOpenVersion(openKey.getOpenVersion()); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private LookupKeyResponse lookupKey(LookupKeyRequest request) { - LookupKeyResponse.Builder resp = - LookupKeyResponse.newBuilder(); - try { - KeyArgs keyArgs = request.getKeyArgs(); - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .build(); - OmKeyInfo keyInfo = impl.lookupKey(omKeyArgs); - resp.setKeyInfo(keyInfo.getProtobuf()); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private RenameKeyResponse renameKey(RenameKeyRequest request) { - RenameKeyResponse.Builder resp = RenameKeyResponse.newBuilder(); - try { - KeyArgs keyArgs = request.getKeyArgs(); - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .build(); - impl.renameKey(omKeyArgs, request.getToKeyName()); - resp.setStatus(Status.OK); - } catch (IOException e){ - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private SetBucketPropertyResponse setBucketProperty( - SetBucketPropertyRequest request) { - SetBucketPropertyResponse.Builder resp = - SetBucketPropertyResponse.newBuilder(); - try { - impl.setBucketProperty(OmBucketArgs.getFromProtobuf( - request.getBucketArgs())); - resp.setStatus(Status.OK); - } catch(IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private DeleteKeyResponse deleteKey(DeleteKeyRequest request) { - DeleteKeyResponse.Builder resp = - DeleteKeyResponse.newBuilder(); - try { - KeyArgs keyArgs = request.getKeyArgs(); - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .build(); - impl.deleteKey(omKeyArgs); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private DeleteBucketResponse deleteBucket(DeleteBucketRequest request) { - DeleteBucketResponse.Builder resp = DeleteBucketResponse.newBuilder(); - resp.setStatus(Status.OK); - try { - impl.deleteBucket(request.getVolumeName(), request.getBucketName()); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private ListBucketsResponse listBuckets(ListBucketsRequest request) { - ListBucketsResponse.Builder resp = - ListBucketsResponse.newBuilder(); - try { - List buckets = impl.listBuckets( - request.getVolumeName(), - request.getStartKey(), - request.getPrefix(), - request.getCount()); - for(OmBucketInfo bucket : buckets) { - resp.addBucketInfo(bucket.getProtobuf()); - } - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private ListKeysResponse listKeys(ListKeysRequest request) { - ListKeysResponse.Builder resp = - ListKeysResponse.newBuilder(); - try { - List keys = impl.listKeys( - request.getVolumeName(), - request.getBucketName(), - request.getStartKey(), - request.getPrefix(), - request.getCount()); - for(OmKeyInfo key : keys) { - resp.addKeyInfo(key.getProtobuf()); - } - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private CommitKeyResponse commitKey(CommitKeyRequest request) { - CommitKeyResponse.Builder resp = - CommitKeyResponse.newBuilder(); - try { - KeyArgs keyArgs = request.getKeyArgs(); - HddsProtos.ReplicationType type = - keyArgs.hasType()? keyArgs.getType() : null; - HddsProtos.ReplicationFactor factor = - keyArgs.hasFactor()? keyArgs.getFactor() : null; - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .setLocationInfoList(keyArgs.getKeyLocationsList().stream() - .map(OmKeyLocationInfo::getFromProtobuf) - .collect(Collectors.toList())) - .setType(type) - .setFactor(factor) - .setDataSize(keyArgs.getDataSize()) - .build(); - impl.commitKey(omKeyArgs, request.getClientID()); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private AllocateBlockResponse allocateBlock(AllocateBlockRequest request) { - AllocateBlockResponse.Builder resp = - AllocateBlockResponse.newBuilder(); - try { - KeyArgs keyArgs = request.getKeyArgs(); - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .build(); - OmKeyLocationInfo newLocation = impl.allocateBlock(omKeyArgs, - request.getClientID()); - resp.setKeyLocation(newLocation.getProtobuf()); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private ServiceListResponse getServiceList(ServiceListRequest request) { - ServiceListResponse.Builder resp = ServiceListResponse.newBuilder(); - try { - resp.addAllServiceInfo(impl.getServiceList().stream() - .map(ServiceInfo::getProtobuf) - .collect(Collectors.toList())); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private S3CreateBucketResponse createS3Bucket(S3CreateBucketRequest request) { - S3CreateBucketResponse.Builder resp = S3CreateBucketResponse.newBuilder(); - try { - impl.createS3Bucket(request.getUserName(), request.getS3Bucketname()); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private S3DeleteBucketResponse deleteS3Bucket(S3DeleteBucketRequest request) { - S3DeleteBucketResponse.Builder resp = S3DeleteBucketResponse.newBuilder(); - try { - impl.deleteS3Bucket(request.getS3BucketName()); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private S3BucketInfoResponse getS3Bucketinfo(S3BucketInfoRequest request) { - S3BucketInfoResponse.Builder resp = S3BucketInfoResponse.newBuilder(); - try { - resp.setOzoneMapping( - impl.getOzoneBucketMapping(request.getS3BucketName())); - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private S3ListBucketsResponse listS3Buckets(S3ListBucketsRequest request) { - S3ListBucketsResponse.Builder resp = S3ListBucketsResponse.newBuilder(); - try { - List buckets = impl.listS3Buckets( - request.getUserName(), - request.getStartKey(), - request.getPrefix(), - request.getCount()); - for(OmBucketInfo bucket : buckets) { - resp.addBucketInfo(bucket.getProtobuf()); - } - resp.setStatus(Status.OK); - } catch (IOException e) { - resp.setStatus(exceptionToResponseStatus(e)); - } - return resp.build(); - } - - private MultipartInfoInitiateResponse initiateMultiPartUpload( - MultipartInfoInitiateRequest request) { - MultipartInfoInitiateResponse.Builder resp = MultipartInfoInitiateResponse - .newBuilder(); - try { - KeyArgs keyArgs = request.getKeyArgs(); - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .setType(keyArgs.getType()) - .setFactor(keyArgs.getFactor()) - .build(); - OmMultipartInfo multipartInfo = impl.initiateMultipartUpload(omKeyArgs); - resp.setVolumeName(multipartInfo.getVolumeName()); - resp.setBucketName(multipartInfo.getBucketName()); - resp.setKeyName(multipartInfo.getKeyName()); - resp.setMultipartUploadID(multipartInfo.getUploadID()); - resp.setStatus(Status.OK); - } catch (IOException ex) { - resp.setStatus(exceptionToResponseStatus(ex)); - } - return resp.build(); - } - - private MultipartCommitUploadPartResponse commitMultipartUploadPart( - MultipartCommitUploadPartRequest request) { - MultipartCommitUploadPartResponse.Builder resp = - MultipartCommitUploadPartResponse.newBuilder(); - try { - KeyArgs keyArgs = request.getKeyArgs(); - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .setMultipartUploadID(keyArgs.getMultipartUploadID()) - .setIsMultipartKey(keyArgs.getIsMultipartKey()) - .setMultipartUploadPartNumber(keyArgs.getMultipartNumber()) - .setDataSize(keyArgs.getDataSize()) - .setLocationInfoList(keyArgs.getKeyLocationsList().stream() - .map(OmKeyLocationInfo::getFromProtobuf) - .collect(Collectors.toList())) - .build(); - OmMultipartCommitUploadPartInfo commitUploadPartInfo = - impl.commitMultipartUploadPart(omKeyArgs, request.getClientID()); - resp.setPartName(commitUploadPartInfo.getPartName()); - resp.setStatus(Status.OK); - } catch (IOException ex) { - resp.setStatus(exceptionToResponseStatus(ex)); - } - return resp.build(); - } - - - private MultipartUploadCompleteResponse completeMultipartUpload( - MultipartUploadCompleteRequest request) { - MultipartUploadCompleteResponse.Builder response = - MultipartUploadCompleteResponse.newBuilder(); - - try { - KeyArgs keyArgs = request.getKeyArgs(); - List partsList = request.getPartsListList(); - - TreeMap partsMap = new TreeMap<>(); - for (Part part : partsList) { - partsMap.put(part.getPartNumber(), part.getPartName()); - } - - OmMultipartUploadList omMultipartUploadList = - new OmMultipartUploadList(partsMap); - - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .setMultipartUploadID(keyArgs.getMultipartUploadID()) - .build(); - OmMultipartUploadCompleteInfo omMultipartUploadCompleteInfo = impl - .completeMultipartUpload(omKeyArgs, omMultipartUploadList); - - response.setVolume(omMultipartUploadCompleteInfo.getVolume()) - .setBucket(omMultipartUploadCompleteInfo.getBucket()) - .setKey(omMultipartUploadCompleteInfo.getKey()) - .setHash(omMultipartUploadCompleteInfo.getHash()); - response.setStatus(Status.OK); - } catch (IOException ex) { - response.setStatus(exceptionToResponseStatus(ex)); - } - return response.build(); - } - - private MultipartUploadAbortResponse abortMultipartUpload( - MultipartUploadAbortRequest multipartUploadAbortRequest) { - MultipartUploadAbortResponse.Builder response = - MultipartUploadAbortResponse.newBuilder(); - - try { - KeyArgs keyArgs = multipartUploadAbortRequest.getKeyArgs(); - OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() - .setVolumeName(keyArgs.getVolumeName()) - .setBucketName(keyArgs.getBucketName()) - .setKeyName(keyArgs.getKeyName()) - .setMultipartUploadID(keyArgs.getMultipartUploadID()) - .build(); - impl.abortMultipartUpload(omKeyArgs); - response.setStatus(Status.OK); - } catch (IOException ex) { - response.setStatus(exceptionToResponseStatus(ex)); - } - return response.build(); - } - } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java new file mode 100644 index 0000000000..894297f52a --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java @@ -0,0 +1,918 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.ozone.protocolPB; + +import com.google.common.collect.Lists; +import java.io.IOException; +import java.util.List; +import java.util.TreeMap; +import java.util.stream.Collectors; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .AllocateBlockRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .AllocateBlockResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CheckVolumeAccessRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CheckVolumeAccessResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CommitKeyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CommitKeyResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CreateBucketRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CreateBucketResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CreateKeyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CreateKeyResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CreateVolumeRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .CreateVolumeResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .DeleteBucketRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .DeleteBucketResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .DeleteKeyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .DeleteKeyResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .DeleteVolumeRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .DeleteVolumeResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .InfoBucketRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .InfoBucketResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .InfoVolumeRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .InfoVolumeResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .KeyArgs; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .ListBucketsRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .ListBucketsResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .ListKeysRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .ListKeysResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .ListVolumeRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .ListVolumeResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .LookupKeyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .LookupKeyResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .MultipartCommitUploadPartRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .MultipartCommitUploadPartResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .MultipartInfoInitiateRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .MultipartInfoInitiateResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .MultipartUploadAbortRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .MultipartUploadAbortResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .MultipartUploadCompleteRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .MultipartUploadCompleteResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .OMRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .OMResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .Part; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .RenameKeyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .RenameKeyResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .S3BucketInfoRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .S3BucketInfoResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .S3CreateBucketRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .S3CreateBucketResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .S3DeleteBucketRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .S3DeleteBucketResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .S3ListBucketsRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .S3ListBucketsResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .ServiceListRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .ServiceListResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .SetBucketPropertyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .SetBucketPropertyResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .SetVolumePropertyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .SetVolumePropertyResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Command Handler for OM requests. OM State Machine calls this handler for + * deserializing the client request and sending it to OM. + */ +public class OzoneManagerRequestHandler { + static final Logger LOG = + LoggerFactory.getLogger(OzoneManagerRequestHandler.class); + private final OzoneManagerProtocol impl; + + public OzoneManagerRequestHandler(OzoneManagerProtocol om) { + this.impl = om; + } + + public OMResponse handle(OMRequest request) { + LOG.debug("Received OMRequest: {}, ", request); + Type cmdType = request.getCmdType(); + OMResponse.Builder responseBuilder = OMResponse.newBuilder() + .setCmdType(cmdType); + + switch (cmdType) { + case CreateVolume: + CreateVolumeResponse createVolumeResponse = createVolume( + request.getCreateVolumeRequest()); + responseBuilder.setCreateVolumeResponse(createVolumeResponse); + break; + case SetVolumeProperty: + SetVolumePropertyResponse setVolumePropertyResponse = setVolumeProperty( + request.getSetVolumePropertyRequest()); + responseBuilder.setSetVolumePropertyResponse(setVolumePropertyResponse); + break; + case CheckVolumeAccess: + CheckVolumeAccessResponse checkVolumeAccessResponse = checkVolumeAccess( + request.getCheckVolumeAccessRequest()); + responseBuilder.setCheckVolumeAccessResponse(checkVolumeAccessResponse); + break; + case InfoVolume: + InfoVolumeResponse infoVolumeResponse = infoVolume( + request.getInfoVolumeRequest()); + responseBuilder.setInfoVolumeResponse(infoVolumeResponse); + break; + case DeleteVolume: + DeleteVolumeResponse deleteVolumeResponse = deleteVolume( + request.getDeleteVolumeRequest()); + responseBuilder.setDeleteVolumeResponse(deleteVolumeResponse); + break; + case ListVolume: + ListVolumeResponse listVolumeResponse = listVolumes( + request.getListVolumeRequest()); + responseBuilder.setListVolumeResponse(listVolumeResponse); + break; + case CreateBucket: + CreateBucketResponse createBucketResponse = createBucket( + request.getCreateBucketRequest()); + responseBuilder.setCreateBucketResponse(createBucketResponse); + break; + case InfoBucket: + InfoBucketResponse infoBucketResponse = infoBucket( + request.getInfoBucketRequest()); + responseBuilder.setInfoBucketResponse(infoBucketResponse); + break; + case SetBucketProperty: + SetBucketPropertyResponse setBucketPropertyResponse = setBucketProperty( + request.getSetBucketPropertyRequest()); + responseBuilder.setSetBucketPropertyResponse(setBucketPropertyResponse); + break; + case DeleteBucket: + DeleteBucketResponse deleteBucketResponse = deleteBucket( + request.getDeleteBucketRequest()); + responseBuilder.setDeleteBucketResponse(deleteBucketResponse); + break; + case ListBuckets: + ListBucketsResponse listBucketsResponse = listBuckets( + request.getListBucketsRequest()); + responseBuilder.setListBucketsResponse(listBucketsResponse); + break; + case CreateKey: + CreateKeyResponse createKeyResponse = createKey( + request.getCreateKeyRequest()); + responseBuilder.setCreateKeyResponse(createKeyResponse); + break; + case LookupKey: + LookupKeyResponse lookupKeyResponse = lookupKey( + request.getLookupKeyRequest()); + responseBuilder.setLookupKeyResponse(lookupKeyResponse); + break; + case RenameKey: + RenameKeyResponse renameKeyResponse = renameKey( + request.getRenameKeyRequest()); + responseBuilder.setRenameKeyResponse(renameKeyResponse); + break; + case DeleteKey: + DeleteKeyResponse deleteKeyResponse = deleteKey( + request.getDeleteKeyRequest()); + responseBuilder.setDeleteKeyResponse(deleteKeyResponse); + break; + case ListKeys: + ListKeysResponse listKeysResponse = listKeys( + request.getListKeysRequest()); + responseBuilder.setListKeysResponse(listKeysResponse); + break; + case CommitKey: + CommitKeyResponse commitKeyResponse = commitKey( + request.getCommitKeyRequest()); + responseBuilder.setCommitKeyResponse(commitKeyResponse); + break; + case AllocateBlock: + AllocateBlockResponse allocateBlockResponse = allocateBlock( + request.getAllocateBlockRequest()); + responseBuilder.setAllocateBlockResponse(allocateBlockResponse); + break; + case CreateS3Bucket: + S3CreateBucketResponse s3CreateBucketResponse = createS3Bucket( + request.getCreateS3BucketRequest()); + responseBuilder.setCreateS3BucketResponse(s3CreateBucketResponse); + break; + case DeleteS3Bucket: + S3DeleteBucketResponse s3DeleteBucketResponse = deleteS3Bucket( + request.getDeleteS3BucketRequest()); + responseBuilder.setDeleteS3BucketResponse(s3DeleteBucketResponse); + break; + case InfoS3Bucket: + S3BucketInfoResponse s3BucketInfoResponse = getS3Bucketinfo( + request.getInfoS3BucketRequest()); + responseBuilder.setInfoS3BucketResponse(s3BucketInfoResponse); + break; + case ListS3Buckets: + S3ListBucketsResponse s3ListBucketsResponse = listS3Buckets( + request.getListS3BucketsRequest()); + responseBuilder.setListS3BucketsResponse(s3ListBucketsResponse); + break; + case InitiateMultiPartUpload: + MultipartInfoInitiateResponse multipartInfoInitiateResponse = + initiateMultiPartUpload(request.getInitiateMultiPartUploadRequest()); + responseBuilder.setInitiateMultiPartUploadResponse( + multipartInfoInitiateResponse); + break; + case CommitMultiPartUpload: + MultipartCommitUploadPartResponse commitUploadPartResponse = + commitMultipartUploadPart(request.getCommitMultiPartUploadRequest()); + responseBuilder.setCommitMultiPartUploadResponse( + commitUploadPartResponse); + break; + case CompleteMultiPartUpload: + MultipartUploadCompleteResponse completeMultiPartUploadResponse = + completeMultipartUpload(request.getCompleteMultiPartUploadRequest()); + responseBuilder.setCompleteMultiPartUploadResponse( + completeMultiPartUploadResponse); + break; + case AbortMultiPartUpload: + MultipartUploadAbortResponse abortMultiPartAbortResponse = + abortMultipartUpload(request.getAbortMultiPartUploadRequest()); + responseBuilder.setAbortMultiPartUploadResponse( + abortMultiPartAbortResponse); + break; + case ServiceList: + ServiceListResponse serviceListResponse = getServiceList( + request.getServiceListRequest()); + responseBuilder.setServiceListResponse(serviceListResponse); + break; + default: + responseBuilder.setSuccess(false); + responseBuilder.setMessage("Unrecognized Command Type: " + cmdType); + break; + } + return responseBuilder.build(); + } + + // Convert and exception to corresponding status code + private Status exceptionToResponseStatus(IOException ex) { + if (ex instanceof OMException) { + OMException omException = (OMException)ex; + switch (omException.getResult()) { + case FAILED_VOLUME_ALREADY_EXISTS: + return Status.VOLUME_ALREADY_EXISTS; + case FAILED_TOO_MANY_USER_VOLUMES: + return Status.USER_TOO_MANY_VOLUMES; + case FAILED_VOLUME_NOT_FOUND: + return Status.VOLUME_NOT_FOUND; + case FAILED_VOLUME_NOT_EMPTY: + return Status.VOLUME_NOT_EMPTY; + case FAILED_USER_NOT_FOUND: + return Status.USER_NOT_FOUND; + case FAILED_BUCKET_ALREADY_EXISTS: + return Status.BUCKET_ALREADY_EXISTS; + case FAILED_BUCKET_NOT_FOUND: + return Status.BUCKET_NOT_FOUND; + case FAILED_BUCKET_NOT_EMPTY: + return Status.BUCKET_NOT_EMPTY; + case FAILED_KEY_ALREADY_EXISTS: + return Status.KEY_ALREADY_EXISTS; + case FAILED_KEY_NOT_FOUND: + return Status.KEY_NOT_FOUND; + case FAILED_INVALID_KEY_NAME: + return Status.INVALID_KEY_NAME; + case FAILED_KEY_ALLOCATION: + return Status.KEY_ALLOCATION_ERROR; + case FAILED_KEY_DELETION: + return Status.KEY_DELETION_ERROR; + case FAILED_KEY_RENAME: + return Status.KEY_RENAME_ERROR; + case FAILED_METADATA_ERROR: + return Status.METADATA_ERROR; + case OM_NOT_INITIALIZED: + return Status.OM_NOT_INITIALIZED; + case SCM_VERSION_MISMATCH_ERROR: + return Status.SCM_VERSION_MISMATCH_ERROR; + case S3_BUCKET_ALREADY_EXISTS: + return Status.S3_BUCKET_ALREADY_EXISTS; + case S3_BUCKET_NOT_FOUND: + return Status.S3_BUCKET_NOT_FOUND; + case INITIATE_MULTIPART_UPLOAD_FAILED: + return Status.INITIATE_MULTIPART_UPLOAD_ERROR; + case NO_SUCH_MULTIPART_UPLOAD: + return Status.NO_SUCH_MULTIPART_UPLOAD_ERROR; + case UPLOAD_PART_FAILED: + return Status.MULTIPART_UPLOAD_PARTFILE_ERROR; + case COMPLETE_MULTIPART_UPLOAD_FAILED: + return Status.COMPLETE_MULTIPART_UPLOAD_ERROR; + case MISMATCH_MULTIPART_LIST: + return Status.MISMATCH_MULTIPART_LIST; + case MISSING_UPLOAD_PARTS: + return Status.MISSING_UPLOAD_PARTS; + case ENTITY_TOO_SMALL: + return Status.ENTITY_TOO_SMALL; + case ABORT_MULTIPART_UPLOAD_FAILED: + return Status.ABORT_MULTIPART_UPLOAD_FAILED; + default: + return Status.INTERNAL_ERROR; + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Unknown error occurs", ex); + } + return Status.INTERNAL_ERROR; + } + } + + /** + * Validates that the incoming OM request has required parameters. + * TODO: Add more validation checks before writing the request to Ratis log. + * @param omRequest client request to OM + * @throws OMException thrown if required parameters are set to null. + */ + public void validateRequest(OMRequest omRequest) throws OMException { + Type cmdType = omRequest.getCmdType(); + if (cmdType == null) { + throw new OMException("CmdType is null", + OMException.ResultCodes.INVALID_REQUEST); + } + if (omRequest.getClientId() == null) { + throw new OMException("ClientId is null", + OMException.ResultCodes.INVALID_REQUEST); + } + } + + private CreateVolumeResponse createVolume(CreateVolumeRequest request) { + CreateVolumeResponse.Builder resp = CreateVolumeResponse.newBuilder(); + resp.setStatus(Status.OK); + try { + impl.createVolume(OmVolumeArgs.getFromProtobuf(request.getVolumeInfo())); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private SetVolumePropertyResponse setVolumeProperty( + SetVolumePropertyRequest request) { + SetVolumePropertyResponse.Builder resp = + SetVolumePropertyResponse.newBuilder(); + resp.setStatus(Status.OK); + String volume = request.getVolumeName(); + + try { + if (request.hasQuotaInBytes()) { + long quota = request.getQuotaInBytes(); + impl.setQuota(volume, quota); + } else { + String owner = request.getOwnerName(); + impl.setOwner(volume, owner); + } + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private CheckVolumeAccessResponse checkVolumeAccess( + CheckVolumeAccessRequest request) { + CheckVolumeAccessResponse.Builder resp = + CheckVolumeAccessResponse.newBuilder(); + resp.setStatus(Status.OK); + try { + boolean access = impl.checkVolumeAccess(request.getVolumeName(), + request.getUserAcl()); + // if no access, set the response status as access denied + if (!access) { + resp.setStatus(Status.ACCESS_DENIED); + } + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + + return resp.build(); + } + + private InfoVolumeResponse infoVolume(InfoVolumeRequest request) { + InfoVolumeResponse.Builder resp = InfoVolumeResponse.newBuilder(); + resp.setStatus(Status.OK); + String volume = request.getVolumeName(); + try { + OmVolumeArgs ret = impl.getVolumeInfo(volume); + resp.setVolumeInfo(ret.getProtobuf()); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private DeleteVolumeResponse deleteVolume(DeleteVolumeRequest request) { + DeleteVolumeResponse.Builder resp = DeleteVolumeResponse.newBuilder(); + resp.setStatus(Status.OK); + try { + impl.deleteVolume(request.getVolumeName()); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private ListVolumeResponse listVolumes(ListVolumeRequest request) { + ListVolumeResponse.Builder resp = ListVolumeResponse.newBuilder(); + List result = Lists.newArrayList(); + try { + if (request.getScope() + == ListVolumeRequest.Scope.VOLUMES_BY_USER) { + result = impl.listVolumeByUser(request.getUserName(), + request.getPrefix(), request.getPrevKey(), request.getMaxKeys()); + } else if (request.getScope() + == ListVolumeRequest.Scope.VOLUMES_BY_CLUSTER) { + result = impl.listAllVolumes(request.getPrefix(), request.getPrevKey(), + request.getMaxKeys()); + } + + result.forEach(item -> resp.addVolumeInfo(item.getProtobuf())); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private CreateBucketResponse createBucket(CreateBucketRequest request) { + CreateBucketResponse.Builder resp = + CreateBucketResponse.newBuilder(); + try { + impl.createBucket(OmBucketInfo.getFromProtobuf( + request.getBucketInfo())); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private InfoBucketResponse infoBucket(InfoBucketRequest request) { + InfoBucketResponse.Builder resp = + InfoBucketResponse.newBuilder(); + try { + OmBucketInfo omBucketInfo = impl.getBucketInfo( + request.getVolumeName(), request.getBucketName()); + resp.setStatus(Status.OK); + resp.setBucketInfo(omBucketInfo.getProtobuf()); + } catch(IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private CreateKeyResponse createKey(CreateKeyRequest request) { + CreateKeyResponse.Builder resp = + CreateKeyResponse.newBuilder(); + try { + KeyArgs keyArgs = request.getKeyArgs(); + HddsProtos.ReplicationType type = + keyArgs.hasType()? keyArgs.getType() : null; + HddsProtos.ReplicationFactor factor = + keyArgs.hasFactor()? keyArgs.getFactor() : null; + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .setDataSize(keyArgs.getDataSize()) + .setType(type) + .setFactor(factor) + .setIsMultipartKey(keyArgs.getIsMultipartKey()) + .setMultipartUploadID(keyArgs.getMultipartUploadID()) + .setMultipartUploadPartNumber(keyArgs.getMultipartNumber()) + .build(); + if (keyArgs.hasDataSize()) { + omKeyArgs.setDataSize(keyArgs.getDataSize()); + } else { + omKeyArgs.setDataSize(0); + } + OpenKeySession openKey = impl.openKey(omKeyArgs); + resp.setKeyInfo(openKey.getKeyInfo().getProtobuf()); + resp.setID(openKey.getId()); + resp.setOpenVersion(openKey.getOpenVersion()); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private LookupKeyResponse lookupKey(LookupKeyRequest request) { + LookupKeyResponse.Builder resp = + LookupKeyResponse.newBuilder(); + try { + KeyArgs keyArgs = request.getKeyArgs(); + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .build(); + OmKeyInfo keyInfo = impl.lookupKey(omKeyArgs); + resp.setKeyInfo(keyInfo.getProtobuf()); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private RenameKeyResponse renameKey(RenameKeyRequest request) { + RenameKeyResponse.Builder resp = RenameKeyResponse.newBuilder(); + try { + KeyArgs keyArgs = request.getKeyArgs(); + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .build(); + impl.renameKey(omKeyArgs, request.getToKeyName()); + resp.setStatus(Status.OK); + } catch (IOException e){ + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private SetBucketPropertyResponse setBucketProperty( + SetBucketPropertyRequest request) { + SetBucketPropertyResponse.Builder resp = + SetBucketPropertyResponse.newBuilder(); + try { + impl.setBucketProperty(OmBucketArgs.getFromProtobuf( + request.getBucketArgs())); + resp.setStatus(Status.OK); + } catch(IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private DeleteKeyResponse deleteKey(DeleteKeyRequest request) { + DeleteKeyResponse.Builder resp = + DeleteKeyResponse.newBuilder(); + try { + KeyArgs keyArgs = request.getKeyArgs(); + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .build(); + impl.deleteKey(omKeyArgs); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private DeleteBucketResponse deleteBucket(DeleteBucketRequest request) { + DeleteBucketResponse.Builder resp = DeleteBucketResponse.newBuilder(); + resp.setStatus(Status.OK); + try { + impl.deleteBucket(request.getVolumeName(), request.getBucketName()); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private ListBucketsResponse listBuckets(ListBucketsRequest request) { + ListBucketsResponse.Builder resp = + ListBucketsResponse.newBuilder(); + try { + List buckets = impl.listBuckets( + request.getVolumeName(), + request.getStartKey(), + request.getPrefix(), + request.getCount()); + for(OmBucketInfo bucket : buckets) { + resp.addBucketInfo(bucket.getProtobuf()); + } + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private ListKeysResponse listKeys(ListKeysRequest request) { + ListKeysResponse.Builder resp = + ListKeysResponse.newBuilder(); + try { + List keys = impl.listKeys( + request.getVolumeName(), + request.getBucketName(), + request.getStartKey(), + request.getPrefix(), + request.getCount()); + for(OmKeyInfo key : keys) { + resp.addKeyInfo(key.getProtobuf()); + } + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private CommitKeyResponse commitKey(CommitKeyRequest request) { + CommitKeyResponse.Builder resp = + CommitKeyResponse.newBuilder(); + try { + KeyArgs keyArgs = request.getKeyArgs(); + HddsProtos.ReplicationType type = + keyArgs.hasType()? keyArgs.getType() : null; + HddsProtos.ReplicationFactor factor = + keyArgs.hasFactor()? keyArgs.getFactor() : null; + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .setLocationInfoList(keyArgs.getKeyLocationsList().stream() + .map(OmKeyLocationInfo::getFromProtobuf) + .collect(Collectors.toList())) + .setType(type) + .setFactor(factor) + .setDataSize(keyArgs.getDataSize()) + .build(); + impl.commitKey(omKeyArgs, request.getClientID()); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private AllocateBlockResponse allocateBlock(AllocateBlockRequest request) { + AllocateBlockResponse.Builder resp = + AllocateBlockResponse.newBuilder(); + try { + KeyArgs keyArgs = request.getKeyArgs(); + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .build(); + OmKeyLocationInfo newLocation = impl.allocateBlock(omKeyArgs, + request.getClientID()); + resp.setKeyLocation(newLocation.getProtobuf()); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private ServiceListResponse getServiceList(ServiceListRequest request) { + ServiceListResponse.Builder resp = ServiceListResponse.newBuilder(); + try { + resp.addAllServiceInfo(impl.getServiceList().stream() + .map(ServiceInfo::getProtobuf) + .collect(Collectors.toList())); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private S3CreateBucketResponse createS3Bucket(S3CreateBucketRequest request) { + S3CreateBucketResponse.Builder resp = S3CreateBucketResponse.newBuilder(); + try { + impl.createS3Bucket(request.getUserName(), request.getS3Bucketname()); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private S3DeleteBucketResponse deleteS3Bucket(S3DeleteBucketRequest request) { + S3DeleteBucketResponse.Builder resp = S3DeleteBucketResponse.newBuilder(); + try { + impl.deleteS3Bucket(request.getS3BucketName()); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private S3BucketInfoResponse getS3Bucketinfo(S3BucketInfoRequest request) { + S3BucketInfoResponse.Builder resp = S3BucketInfoResponse.newBuilder(); + try { + resp.setOzoneMapping( + impl.getOzoneBucketMapping(request.getS3BucketName())); + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private S3ListBucketsResponse listS3Buckets(S3ListBucketsRequest request) { + S3ListBucketsResponse.Builder resp = S3ListBucketsResponse.newBuilder(); + try { + List buckets = impl.listS3Buckets( + request.getUserName(), + request.getStartKey(), + request.getPrefix(), + request.getCount()); + for(OmBucketInfo bucket : buckets) { + resp.addBucketInfo(bucket.getProtobuf()); + } + resp.setStatus(Status.OK); + } catch (IOException e) { + resp.setStatus(exceptionToResponseStatus(e)); + } + return resp.build(); + } + + private MultipartInfoInitiateResponse initiateMultiPartUpload( + MultipartInfoInitiateRequest request) { + MultipartInfoInitiateResponse.Builder resp = MultipartInfoInitiateResponse + .newBuilder(); + try { + KeyArgs keyArgs = request.getKeyArgs(); + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .setType(keyArgs.getType()) + .setFactor(keyArgs.getFactor()) + .build(); + OmMultipartInfo multipartInfo = impl.initiateMultipartUpload(omKeyArgs); + resp.setVolumeName(multipartInfo.getVolumeName()); + resp.setBucketName(multipartInfo.getBucketName()); + resp.setKeyName(multipartInfo.getKeyName()); + resp.setMultipartUploadID(multipartInfo.getUploadID()); + resp.setStatus(Status.OK); + } catch (IOException ex) { + resp.setStatus(exceptionToResponseStatus(ex)); + } + return resp.build(); + } + + private MultipartCommitUploadPartResponse commitMultipartUploadPart( + MultipartCommitUploadPartRequest request) { + MultipartCommitUploadPartResponse.Builder resp = + MultipartCommitUploadPartResponse.newBuilder(); + try { + KeyArgs keyArgs = request.getKeyArgs(); + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .setMultipartUploadID(keyArgs.getMultipartUploadID()) + .setIsMultipartKey(keyArgs.getIsMultipartKey()) + .setMultipartUploadPartNumber(keyArgs.getMultipartNumber()) + .setDataSize(keyArgs.getDataSize()) + .setLocationInfoList(keyArgs.getKeyLocationsList().stream() + .map(OmKeyLocationInfo::getFromProtobuf) + .collect(Collectors.toList())) + .build(); + OmMultipartCommitUploadPartInfo commitUploadPartInfo = + impl.commitMultipartUploadPart(omKeyArgs, request.getClientID()); + resp.setPartName(commitUploadPartInfo.getPartName()); + resp.setStatus(Status.OK); + } catch (IOException ex) { + resp.setStatus(exceptionToResponseStatus(ex)); + } + return resp.build(); + } + + + private MultipartUploadCompleteResponse completeMultipartUpload( + MultipartUploadCompleteRequest request) { + MultipartUploadCompleteResponse.Builder response = + MultipartUploadCompleteResponse.newBuilder(); + + try { + KeyArgs keyArgs = request.getKeyArgs(); + List partsList = request.getPartsListList(); + + TreeMap partsMap = new TreeMap<>(); + for (Part part : partsList) { + partsMap.put(part.getPartNumber(), part.getPartName()); + } + + OmMultipartUploadList omMultipartUploadList = + new OmMultipartUploadList(partsMap); + + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .setMultipartUploadID(keyArgs.getMultipartUploadID()) + .build(); + OmMultipartUploadCompleteInfo omMultipartUploadCompleteInfo = impl + .completeMultipartUpload(omKeyArgs, omMultipartUploadList); + + response.setVolume(omMultipartUploadCompleteInfo.getVolume()) + .setBucket(omMultipartUploadCompleteInfo.getBucket()) + .setKey(omMultipartUploadCompleteInfo.getKey()) + .setHash(omMultipartUploadCompleteInfo.getHash()); + response.setStatus(Status.OK); + } catch (IOException ex) { + response.setStatus(exceptionToResponseStatus(ex)); + } + return response.build(); + } + + private MultipartUploadAbortResponse abortMultipartUpload( + MultipartUploadAbortRequest multipartUploadAbortRequest) { + MultipartUploadAbortResponse.Builder response = + MultipartUploadAbortResponse.newBuilder(); + + try { + KeyArgs keyArgs = multipartUploadAbortRequest.getKeyArgs(); + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .setMultipartUploadID(keyArgs.getMultipartUploadID()) + .build(); + impl.abortMultipartUpload(omKeyArgs); + response.setStatus(Status.OK); + } catch (IOException ex) { + response.setStatus(exceptionToResponseStatus(ex)); + } + return response.build(); + } +} diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java index a12c22e73b..1f285db38b 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java @@ -64,7 +64,7 @@ public void init() throws Exception { conf.setTimeDuration( OMConfigKeys.OZONE_OM_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY, LEADER_ELECTION_TIMEOUT, TimeUnit.MILLISECONDS); - omRatisServer = OzoneManagerRatisServer.newOMRatisServer(omID, + omRatisServer = OzoneManagerRatisServer.newOMRatisServer(null, omID, InetAddress.getLocalHost(), conf); omRatisServer.start(); omRatisClient = OzoneManagerRatisClient.newOzoneManagerRatisClient(omID, @@ -101,7 +101,6 @@ public void testStartOMRatisServer() throws Exception { public void testSubmitRatisRequest() throws Exception { // Wait for leader election Thread.sleep(LEADER_ELECTION_TIMEOUT * 2); - OMRequest request = OMRequest.newBuilder() .setCmdType(OzoneManagerProtocolProtos.Type.CreateVolume) .setClientId(clientId) @@ -109,12 +108,9 @@ public void testSubmitRatisRequest() throws Exception { OMResponse response = omRatisClient.sendCommand(request); - // Since the state machine is not implemented yet, we should get the - // configured dummy message from Ratis. + Assert.assertEquals(OzoneManagerProtocolProtos.Type.CreateVolume, + response.getCmdType()); Assert.assertEquals(false, response.getSuccess()); - Assert.assertTrue(response.getMessage().contains("Dummy response from " + - "Ratis server for command type: " + - OzoneManagerProtocolProtos.Type.CreateVolume)); Assert.assertEquals(false, response.hasCreateVolumeResponse()); }