Sfoglia il codice sorgente

HDDS-606. Create delete s3Bucket. Contributed by Bharat Viswanadham.

Bharat Viswanadham 6 anni fa
parent
commit
8ae8a5004f
15 ha cambiato i file con 247 aggiunte e 0 eliminazioni
  1. 9 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
  2. 8 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
  3. 7 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
  4. 8 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
  5. 7 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
  6. 23 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
  7. 11 0
      hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
  8. 28 0
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
  9. 9 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
  10. 7 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManager.java
  11. 25 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java
  12. 18 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
  13. 18 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestS3BucketManager.java
  14. 43 0
      hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/EndpointBase.java
  15. 26 0
      hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java

+ 9 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java

@@ -94,6 +94,15 @@ public class ObjectStore {
     proxy.createS3Bucket(userName, s3BucketName);
   }
 
+  /**
+   * Deletes an s3 bucket and removes mapping of Ozone volume/bucket.
+   * @param bucketName - S3 Bucket Name.
+   * @throws  IOException in case the bucket cannot be deleted.
+   */
+  public void deleteS3Bucket(String bucketName) throws IOException {
+    proxy.deleteS3Bucket(bucketName);
+  }
+
   /**
    * Returns the Ozone Namespace for the S3Bucket. It will return the
    * OzoneVolume/OzoneBucketName.

+ 8 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java

@@ -330,6 +330,14 @@ public interface ClientProtocol {
    */
   void createS3Bucket(String userName, String s3BucketName) throws IOException;
 
+  /**
+   * Deletes an s3 bucket and removes mapping of Ozone volume/bucket.
+   * @param bucketName - S3 Bucket Name.
+   * @throws  IOException in case the bucket cannot be deleted.
+   */
+  void deleteS3Bucket(String bucketName) throws IOException;
+
+
   /**
    * Returns the Ozone Namespace for the S3Bucket. It will return the
    * OzoneVolume/OzoneBucketName.

+ 7 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java

@@ -827,6 +827,13 @@ public class RestClient implements ClientProtocol {
         "support this operation.");
   }
 
+  @Override
+  public void deleteS3Bucket(String s3BucketName)
+      throws IOException {
+    throw new UnsupportedOperationException("Ozone REST protocol does not " +
+        "support this operation.");
+  }
+
   @Override
   public String getOzoneBucketMapping(String s3BucketName) throws IOException {
     throw new UnsupportedOperationException("Ozone REST protocol does not " +

+ 8 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

@@ -579,6 +579,14 @@ public class RpcClient implements ClientProtocol {
     ozoneManagerClient.createS3Bucket(userName, s3BucketName);
   }
 
+  @Override
+  public void deleteS3Bucket(String s3BucketName)
+      throws IOException {
+    Preconditions.checkArgument(Strings.isNotBlank(s3BucketName), "bucket " +
+        "name cannot be null or empty.");
+    ozoneManagerClient.deleteS3Bucket(s3BucketName);
+  }
+
   @Override
   public String getOzoneBucketMapping(String s3BucketName) throws IOException {
     Preconditions.checkArgument(Strings.isNotBlank(s3BucketName), "bucket " +

+ 7 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java

@@ -264,6 +264,13 @@ public interface OzoneManagerProtocol {
    */
   void createS3Bucket(String userName, String s3BucketName) throws IOException;
 
+  /**
+   * Delets an S3 bucket inside Ozone manager and deletes the mapping.
+   * @param s3BucketName - S3 bucket Name.
+   * @throws IOException in case the bucket cannot be deleted.
+   */
+  void deleteS3Bucket(String s3BucketName) throws IOException;
+
   /**
    * Returns the Ozone Namespace for the S3Bucket. It will return the
    * OzoneVolume/OzoneBucketName.

+ 23 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java

@@ -118,6 +118,10 @@ import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.S3BucketRequest;
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.S3BucketResponse;
+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.S3BucketInfoRequest;
 import org.apache.hadoop.ozone.protocol.proto
@@ -795,6 +799,25 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
 
   }
 
+  @Override
+  public void deleteS3Bucket(String s3BucketName) throws IOException {
+    S3DeleteBucketRequest request  = S3DeleteBucketRequest.newBuilder()
+        .setS3BucketName(s3BucketName)
+        .build();
+    final S3DeleteBucketResponse resp;
+    try {
+      resp = rpcProxy.deleteS3Bucket(NULL_RPC_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+    if(resp.getStatus() != Status.OK) {
+      throw new IOException("Creating S3 bucket failed, error: "
+          + resp.getStatus());
+    }
+
+  }
+
   @Override
   public String getOzoneBucketMapping(String s3BucketName)
       throws IOException {

+ 11 - 0
hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto

@@ -381,6 +381,14 @@ message S3BucketInfoResponse {
     optional string ozoneMapping = 2;
 }
 
+message S3DeleteBucketRequest {
+    required string s3bucketName = 1;
+}
+
+message S3DeleteBucketResponse {
+    required Status status = 1;
+}
+
 
 /**
  The OM service that takes care of Ozone namespace.
@@ -506,6 +514,9 @@ service OzoneManagerService {
     rpc createS3Bucket(S3BucketRequest)
     returns(S3BucketResponse);
 
+    rpc deleteS3Bucket(S3DeleteBucketRequest)
+    returns(S3DeleteBucketResponse);
+
     /**
     Gets the Ozone Mapping information for the S3Bucket.
     */

+ 28 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java

@@ -45,6 +45,7 @@ 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;
@@ -215,6 +216,33 @@ public class TestOzoneRpcClient {
     Assert.assertTrue(volume.getCreationTime() >= currentTime);
   }
 
+  @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 {

+ 9 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

@@ -1140,6 +1140,15 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     s3BucketManager.createS3Bucket(userName, s3BucketName);
   }
 
+  @Override
+  /**
+   * {@inheritDoc}
+   */
+  public void deleteS3Bucket(String s3BucketName)
+      throws IOException {
+    s3BucketManager.deleteS3Bucket(s3BucketName);
+  }
+
   @Override
   /**
    * {@inheritDoc}

+ 7 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManager.java

@@ -33,6 +33,13 @@ public interface S3BucketManager {
    */
   void createS3Bucket(String userName, String bucketName) throws IOException;
 
+  /**
+   * Deletes an s3 bucket and removes mapping of Ozone volume/bucket.
+   * @param bucketName - S3 Bucket Name.
+   * @throws  IOException in case the bucket cannot be deleted.
+   */
+  void deleteS3Bucket(String bucketName) throws IOException;
+
   /**
    * Returns the Ozone volume/bucket where the S3 Bucket points to.
    * @param s3BucketName - S3 Bucket Name

+ 25 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java

@@ -127,6 +127,30 @@ public class S3BucketManagerImpl implements S3BucketManager {
     }
   }
 
+  @Override
+  public void deleteS3Bucket(String bucketName) throws IOException {
+    Preconditions.checkArgument(
+        Strings.isNotBlank(bucketName), "Bucket name cannot be null or empty");
+
+    omMetadataManager.getLock().acquireS3Lock(bucketName);
+    try {
+      byte[] bucket = bucketName.getBytes(StandardCharsets.UTF_8);
+      byte[] map = omMetadataManager.getS3Table().get(bucket);
+
+      if (map == null) {
+        throw new OMException("No such S3 bucket. " + bucketName,
+            OMException.ResultCodes.S3_BUCKET_NOT_FOUND);
+      }
+      bucketManager.deleteBucket(getOzoneVolumeName(bucketName), bucketName);
+      omMetadataManager.getS3Table().delete(bucket);
+    } catch(IOException ex) {
+      throw ex;
+    } finally {
+      omMetadataManager.getLock().releaseS3Lock(bucketName);
+    }
+
+  }
+
   private String formatOzoneVolumeName(String userName) {
     return String.format("s3%s", userName);
   }
@@ -202,4 +226,5 @@ public class S3BucketManagerImpl implements S3BucketManager {
     String mapping = getOzoneBucketMapping(s3BucketName);
     return mapping.split("/")[1];
   }
+
 }

+ 18 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java

@@ -97,6 +97,10 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .S3BucketRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .S3BucketResponse;
+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
     .ServiceListRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
@@ -596,6 +600,20 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
     return resp.build();
   }
 
+  @Override
+  public S3DeleteBucketResponse deleteS3Bucket(RpcController controller,
+                                         S3DeleteBucketRequest request) throws
+      ServiceException {
+    S3DeleteBucketResponse.Builder resp = S3DeleteBucketResponse.newBuilder();
+    try {
+      impl.deleteS3Bucket(request.getS3BucketName());
+      resp.setStatus(Status.OK);
+    } catch (IOException e) {
+      resp.setStatus(exceptionToResponseStatus(e));
+    }
+    return resp.build();
+  }
+
   @Override
   public S3BucketInfoResponse getS3Bucketinfo(RpcController controller,
       S3BucketInfoRequest request) throws ServiceException {

+ 18 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestS3BucketManager.java

@@ -75,6 +75,24 @@ public class TestS3BucketManager {
 
   }
 
+  @Test
+  public void testDeleteS3Bucket() throws IOException {
+    S3BucketManager s3BucketManager = new S3BucketManagerImpl(conf, metaMgr,
+        volumeManager, bucketManager);
+    s3BucketManager.createS3Bucket("ozone", "s3bucket");
+
+    // This call should have created a ozone volume called s3ozone and bucket
+    // called s3ozone/s3bucket.
+    Assert.assertNotNull(volumeManager.getVolumeInfo("s3ozone"));
+    Assert.assertNotNull(bucketManager.getBucketInfo("s3ozone", "s3bucket"));
+
+    s3BucketManager.deleteS3Bucket("s3bucket");
+
+    //Deleting non existing bucket should throw.
+    thrown.expect(IOException.class);
+    s3BucketManager.deleteS3Bucket("s3bucket");
+  }
+
   @Test
   public void testGetS3BucketMapping() throws IOException {
     S3BucketManager s3BucketManager = new S3BucketManagerImpl(conf, metaMgr,

+ 43 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/EndpointBase.java

@@ -107,6 +107,49 @@ public class EndpointBase {
     return "/"+location;
   }
 
+  /**
+   * Deletes an s3 bucket and removes mapping of Ozone volume/bucket.
+   * @param s3BucketName - S3 Bucket Name.
+   * @throws  IOException in case the bucket cannot be deleted.
+   */
+  public void deleteS3Bucket(String s3BucketName)
+      throws IOException {
+    client.getObjectStore().deleteS3Bucket(s3BucketName);
+  }
+
+  /**
+   * Returns the Ozone Namespace for the S3Bucket. It will return the
+   * OzoneVolume/OzoneBucketName.
+   * @param s3BucketName  - S3 Bucket Name.
+   * @return String - The Ozone canonical name for this s3 bucket. This
+   * string is useful for mounting an OzoneFS.
+   * @throws IOException - Error is throw if the s3bucket does not exist.
+   */
+  public String getOzoneBucketMapping(String s3BucketName) throws IOException {
+    return client.getObjectStore().getOzoneBucketMapping(s3BucketName);
+  }
+
+  /**
+   * Returns the corresponding Ozone volume given an S3 Bucket.
+   * @param s3BucketName - S3Bucket Name.
+   * @return String - Ozone Volume name.
+   * @throws IOException - Throws if the s3Bucket does not exist.
+   */
+  public String getOzoneVolumeName(String s3BucketName) throws IOException {
+    return client.getObjectStore().getOzoneVolumeName(s3BucketName);
+  }
+
+  /**
+   * Returns the corresponding Ozone bucket name for the given S3 bucket.
+   * @param s3BucketName - S3Bucket Name.
+   * @return String - Ozone bucket Name.
+   * @throws IOException - Throws if the s3bucket does not exist.
+   */
+  public String getOzoneBucketName(String s3BucketName) throws IOException {
+    return client.getObjectStore().getOzoneBucketName(s3BucketName);
+  }
+
+
   @VisibleForTesting
   public void setClient(OzoneClient ozoneClient) {
     this.client = ozoneClient;

+ 26 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java

@@ -36,6 +36,7 @@ public class ObjectStoreStub extends ObjectStore {
   }
 
   private Map<String, OzoneVolumeStub> volumes = new HashMap<>();
+  private Map<String, String> bucketVolumeMap = new HashMap<>();
 
   @Override
   public void createVolume(String volumeName) throws IOException {
@@ -107,4 +108,29 @@ public class ObjectStoreStub extends ObjectStore {
   public void deleteVolume(String volumeName) throws IOException {
     volumes.remove(volumeName);
   }
+
+  @Override
+  public void createS3Bucket(String userName, String s3BucketName) throws
+      IOException {
+    bucketVolumeMap.put(s3BucketName, "s3"+userName+"/"+s3BucketName);
+  }
+
+  @Override
+  public void deleteS3Bucket(String s3BucketName) throws
+      IOException {
+    bucketVolumeMap.remove(s3BucketName);
+  }
+
+  @Override
+  public String getOzoneBucketMapping(String s3BucketName) throws IOException {
+    return bucketVolumeMap.get(s3BucketName);
+  }
+  @Override
+  public String getOzoneVolumeName(String s3BucketName) throws IOException {
+    return bucketVolumeMap.get(s3BucketName).split("/")[0];
+  }
+  @Override
+  public String getOzoneBucketName(String s3BucketName) throws IOException {
+    return bucketVolumeMap.get(s3BucketName).split("/")[1];
+  }
 }