Browse Source

HDDS-1900. Remove UpdateBucket handler which supports add/remove Acl. (#1219)

(cherry picked from commit 70b4617cfe69fcbde0dca88827b92505d0925c3d)
Bharat Viswanadham 5 năm trước cách đây
mục cha
commit
b813c14835

+ 0 - 18
hadoop-hdds/docs/content/shell/BucketCommands.md

@@ -26,7 +26,6 @@ Ozone shell supports the following bucket commands.
   * [delete](#delete)
   * [info](#info)
   * [list](#list)
-  * [update](#update)
 
 ### Create
 
@@ -98,23 +97,6 @@ This command will list all buckets on the volume _hive_.
 
 
 
-### Update
-
-The bucket update command allows changing access permissions on bucket.
-
-***Params:***
-
-| Arguments                      |  Comment                                |
-|--------------------------------|-----------------------------------------|
-| --addAcl                       | Optional, Comma separated ACLs that will added to bucket.
-|  --removeAcl                   | Optional, Comma separated list of acl to remove.
-|  Uri                           | The name of the bucket.
-
-{{< highlight bash >}}
-ozone sh bucket update --addAcl=user:bilbo:rw /hive/jan
-{{< /highlight >}}
-
-The above command gives user bilbo read/write permission to the bucket.
 
 ### path
 The bucket command to provide ozone mapping for s3 bucket (Created via aws cli)

+ 0 - 4
hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell.robot

@@ -60,10 +60,6 @@ Test ozone shell
                     Execute             ozone sh bucket create ${protocol}${server}/${volume}/bb1
     ${result} =     Execute             ozone sh bucket info ${protocol}${server}/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.bucketName=="bb1") | .storageType'
                     Should Be Equal     ${result}       DISK
-    ${result} =     Execute             ozone sh bucket update ${protocol}${server}/${volume}/bb1 --addAcl user:frodo:rw,group:samwise:r | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.bucketName=="bb1") | .acls | .[] | select(.name=="samwise") | .type'
-                    Should Be Equal     ${result}       GROUP
-    ${result} =     Execute             ozone sh bucket update ${protocol}${server}/${volume}/bb1 --removeAcl group:samwise:r | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.bucketName=="bb1") | .acls | .[] | select(.name=="frodo") | .type'
-                    Should Be Equal     ${result}       USER
     ${result} =     Execute             ozone sh bucket list ${protocol}${server}/${volume}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.bucketName=="bb1") | .volumeName'
                     Should Be Equal     ${result}       ${volume}
                     Run Keyword         Test key handling       ${protocol}       ${server}       ${volume}

+ 0 - 48
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java

@@ -43,7 +43,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.tracing.StringCodec;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKey;
@@ -57,8 +56,6 @@ import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
-import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
-import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
 import org.apache.hadoop.ozone.web.ozShell.OzoneShell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
@@ -723,51 +720,6 @@ public class TestOzoneShell {
         ResultCodes.BUCKET_NOT_FOUND);
   }
 
-  @Test
-  public void testUpdateBucket() throws Exception {
-    LOG.info("Running testUpdateBucket");
-    OzoneVolume vol = creatVolume();
-    String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
-    vol.createBucket(bucketName);
-    OzoneBucket bucket = vol.getBucket(bucketName);
-    int aclSize = bucket.getAcls().size();
-
-    String[] args = new String[] {"bucket", "update",
-        url + "/" + vol.getName() + "/" + bucketName, "--addAcl",
-        "user:frodo:rw,group:samwise:r"};
-    execute(shell, args);
-    String output = out.toString();
-    assertTrue(output.contains("createdOn")
-        && output.contains(OzoneConsts.OZONE_TIME_ZONE));
-
-    bucket = vol.getBucket(bucketName);
-    assertEquals(2 + aclSize, bucket.getAcls().size());
-
-    OzoneAcl acl = bucket.getAcls().get(aclSize);
-    assertTrue(acl.getName().equals("frodo")
-        && acl.getType() == ACLIdentityType.USER
-        && acl.getAclBitSet().get(ACLType.READ.ordinal())
-        && acl.getAclBitSet().get(ACLType.WRITE.ordinal()));
-
-    args = new String[] {"bucket", "update",
-        url + "/" + vol.getName() + "/" + bucketName, "--removeAcl",
-        "user:frodo:rw"};
-    execute(shell, args);
-
-    bucket = vol.getBucket(bucketName);
-    acl = bucket.getAcls().get(aclSize);
-    assertEquals(1 + aclSize, bucket.getAcls().size());
-    assertTrue(acl.getName().equals("samwise")
-        && acl.getType() == ACLIdentityType.GROUP
-        && acl.getAclBitSet().get(ACLType.READ.ordinal()));
-
-    // test update bucket for a non-exist bucket
-    args = new String[] {"bucket", "update",
-        url + "/" + vol.getName() + "/invalid-bucket", "--addAcl",
-        "user:frodo:rw"};
-    executeWithError(shell, args, BUCKET_NOT_FOUND);
-  }
-
   @Test
   public void testListBucket() throws Exception {
     LOG.info("Running testListBucket");

+ 0 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/BucketCommands.java

@@ -38,7 +38,6 @@ import picocli.CommandLine.ParentCommand;
         InfoBucketHandler.class,
         ListBucketHandler.class,
         CreateBucketHandler.class,
-        UpdateBucketHandler.class,
         DeleteBucketHandler.class,
         AddAclBucketHandler.class,
         RemoveAclBucketHandler.class,

+ 0 - 96
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java

@@ -1,96 +0,0 @@
-/*
- * 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.web.ozShell.bucket;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.client.OzoneClientUtils;
-import org.apache.hadoop.ozone.client.OzoneVolume;
-import org.apache.hadoop.ozone.web.ozShell.Handler;
-import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
-import org.apache.hadoop.ozone.web.ozShell.Shell;
-import org.apache.hadoop.ozone.web.utils.JsonUtils;
-
-import picocli.CommandLine.Command;
-import picocli.CommandLine.Option;
-import picocli.CommandLine.Parameters;
-
-/**
- * Allows users to add and remove acls and from a bucket.
- */
-@Command(name = "update",
-    description = "allows changing bucket attributes")
-public class UpdateBucketHandler extends Handler {
-
-  @Parameters(arity = "1..1", description = Shell.OZONE_BUCKET_URI_DESCRIPTION)
-  private String uri;
-
-  @Option(names = {"--addAcl"},
-      description = "Comma separated list of acl rules to add (eg. " +
-          "user:bilbo:rw)")
-  private String addAcl;
-
-  @Option(names = {"--removeAcl"},
-      description = "Comma separated list of acl rules to remove (eg. "
-          + "user:bilbo:rw)")
-  private String removeAcl;
-
-  @Override
-  public Void call() throws Exception {
-
-    OzoneAddress address = new OzoneAddress(uri);
-    address.ensureBucketAddress();
-    OzoneClient client = address.createClient(createOzoneConfiguration());
-
-    String volumeName = address.getVolumeName();
-    String bucketName = address.getBucketName();
-
-    if (isVerbose()) {
-      System.out.printf("Volume Name : %s%n", volumeName);
-      System.out.printf("Bucket Name : %s%n", bucketName);
-    }
-
-    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
-    OzoneBucket bucket = vol.getBucket(bucketName);
-    if (addAcl != null) {
-      String[] aclArray = addAcl.split(",");
-      List<OzoneAcl> aclList =
-          Arrays.stream(aclArray).map(acl -> OzoneAcl.parseAcl(acl
-          ))
-              .collect(Collectors.toList());
-      bucket.addAcls(aclList);
-    }
-
-    if (removeAcl != null) {
-      String[] aclArray = removeAcl.split(",");
-      List<OzoneAcl> aclList =
-          Arrays.stream(aclArray).map(acl -> OzoneAcl.parseAcl(acl))
-              .collect(Collectors.toList());
-      bucket.removeAcls(aclList);
-    }
-
-    System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        JsonUtils.toJsonString(OzoneClientUtils.asBucketInfo(bucket))));
-    return null;
-  }
-}