Parcourir la source

HDDS-701. Support key multi-delete.
Contributed by Elek, Marton.

Anu Engineer il y a 6 ans
Parent
commit
9aebafd2da

+ 44 - 0
hadoop-ozone/dist/src/main/smoketest/s3/objectmultidelete.robot

@@ -0,0 +1,44 @@
+# 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.
+
+*** Settings ***
+Documentation       S3 gateway test with aws cli
+Library             OperatingSystem
+Library             String
+Resource            ../commonlib.robot
+Resource            commonawslib.robot
+Test Setup          Setup s3 tests
+
+*** Variables ***
+${ENDPOINT_URL}       http://s3g:9878
+${BUCKET}             generated
+
+*** Test Cases ***
+
+Delete file with multi delete
+                        Execute                    date > /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key multidelete/f1 --body /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key multidelete/f2 --body /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key multidelete/f3 --body /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${BUCKET} --prefix multidelete/
+                        Should contain             ${result}         multidelete/f1
+                        Should contain             ${result}         multidelete/f2
+                        Should contain             ${result}         multidelete/f3
+    ${result} =         Execute AWSS3APICli        delete-objects --bucket ${BUCKET} --delete 'Objects=[{Key=multidelete/f1},{Key=multidelete/f2},{Key=multidelete/f4}]'
+                        Should not contain         ${result}         Error
+    ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${BUCKET} --prefix multidelete/
+                        Should not contain         ${result}         multidelete/f1
+                        Should not contain         ${result}         multidelete/f2
+                        Should contain             ${result}         multidelete/f3

+ 47 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/HeaderPreprocessor.java

@@ -0,0 +1,47 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.s3;
+
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.PreMatching;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+
+/**
+ * Filter to adjust request headers for compatible reasons.
+ */
+
+@Provider
+@PreMatching
+public class HeaderPreprocessor implements ContainerRequestFilter {
+
+  @Override
+  public void filter(ContainerRequestContext requestContext) throws
+      IOException {
+    if (requestContext.getUriInfo().getQueryParameters()
+        .containsKey("delete")) {
+      //aws cli doesn't send proper Content-Type and by default POST requests
+      //processed as form-url-encoded. Here we can fix this.
+      requestContext.getHeaders()
+          .putSingle("Content-Type", MediaType.APPLICATION_XML);
+    }
+  }
+
+}

+ 51 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java

@@ -21,9 +21,11 @@ import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
 import javax.ws.rs.HEAD;
+import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.HttpHeaders;
@@ -34,10 +36,15 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.time.Instant;
 import java.util.Iterator;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response.ResponseBuilder;
 
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.s3.commontypes.KeyMetadata;
+import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteRequest.DeleteObject;
+import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.DeletedObject;
+import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.Error;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 
@@ -208,4 +215,48 @@ public class BucketEndpoint extends EndpointBase {
         .build();
 
   }
+
+  /**
+   * Implement multi delete.
+   * <p>
+   * see: https://docs.aws.amazon
+   * .com/AmazonS3/latest/API/multiobjectdeleteapi.html
+   */
+  @POST
+  @Produces(MediaType.APPLICATION_XML)
+  public Response multiDelete(@PathParam("bucket") String bucketName,
+      @QueryParam("delete") String delete,
+      MultiDeleteRequest request) throws OS3Exception, IOException {
+    OzoneBucket bucket = getBucket(bucketName);
+    MultiDeleteResponse result = new MultiDeleteResponse();
+    if (request.getObjects() != null) {
+      for (DeleteObject keyToDelete : request.getObjects()) {
+        try {
+          bucket.deleteKey(keyToDelete.getKey());
+
+          if (!request.isQuiet()) {
+            result.addDeleted(new DeletedObject(keyToDelete.getKey()));
+          }
+        } catch (IOException ex) {
+          if (!ex.getMessage().contains("KEY_NOT_FOUND")) {
+            result.addError(
+                new Error(keyToDelete.getKey(), "InternalError",
+                    ex.getMessage()));
+          } else if (!request.isQuiet()) {
+            result.addDeleted(new DeletedObject(keyToDelete.getKey()));
+          }
+        } catch (Exception ex) {
+          result.addError(
+              new Error(keyToDelete.getKey(), "InternalError",
+                  ex.getMessage()));
+        }
+      }
+    }
+    ResponseBuilder response = Response.ok();
+    if (!request.isQuiet() || result.getErrors().size() > 0) {
+      response = response.entity(result);
+    }
+    return response.build();
+
+  }
 }

+ 96 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/MultiDeleteRequest.java

@@ -0,0 +1,96 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.s3.endpoint;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Request for multi object delete request.
+ */
+
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlRootElement(name = "Delete", namespace = "http://s3.amazonaws"
+    + ".com/doc/2006-03-01/")
+public class MultiDeleteRequest {
+
+  @XmlElement(name = "Quiet")
+  private boolean quiet;
+
+  @XmlElement(name = "Object")
+  private List<DeleteObject> objects = new ArrayList<>();
+
+  public boolean isQuiet() {
+    return quiet;
+  }
+
+  public void setQuiet(boolean quiet) {
+    this.quiet = quiet;
+  }
+
+  public List<DeleteObject> getObjects() {
+    return objects;
+  }
+
+  public void setObjects(
+      List<DeleteObject> objects) {
+    this.objects = objects;
+  }
+
+  /**
+   * JAXB entity for child element.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Object", namespace = "http://s3.amazonaws"
+      + ".com/doc/2006-03-01/")
+  public static class DeleteObject {
+
+    @XmlElement(name = "Key")
+    private String key;
+
+    @XmlElement(name = "VersionId")
+    private String versionId;
+
+    public DeleteObject() {
+    }
+
+    public DeleteObject(String key) {
+      this.key = key;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public void setKey(String key) {
+      this.key = key;
+    }
+
+    public String getVersionId() {
+      return versionId;
+    }
+
+    public void setVersionId(String versionId) {
+      this.versionId = versionId;
+    }
+  }
+}

+ 154 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/MultiDeleteResponse.java

@@ -0,0 +1,154 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.s3.endpoint;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Response for multi object delete request.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlRootElement(name = "DeleteResult", namespace = "http://s3.amazonaws"
+    + ".com/doc/2006-03-01/")
+public class MultiDeleteResponse {
+
+  @XmlElement(name = "Deleted")
+  private List<DeletedObject> deletedObjects = new ArrayList<>();
+
+  @XmlElement(name = "Error")
+  private List<Error> errors = new ArrayList<>();
+
+  public void addDeleted(DeletedObject deletedObject) {
+    deletedObjects.add(deletedObject);
+  }
+
+  public void addError(Error error) {
+    errors.add(error);
+  }
+
+  public List<DeletedObject> getDeletedObjects() {
+    return deletedObjects;
+  }
+
+  public void setDeletedObjects(
+      List<DeletedObject> deletedObjects) {
+    this.deletedObjects = deletedObjects;
+  }
+
+  public List<Error> getErrors() {
+    return errors;
+  }
+
+  public void setErrors(
+      List<Error> errors) {
+    this.errors = errors;
+  }
+
+  /**
+   * JAXB entity for child element.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Deleted", namespace = "http://s3.amazonaws"
+      + ".com/doc/2006-03-01/")
+  public static class DeletedObject {
+
+    @XmlElement(name = "Key")
+    private String key;
+
+    private String versionId;
+
+    public DeletedObject() {
+    }
+
+    public DeletedObject(String key) {
+      this.key = key;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public void setKey(String key) {
+      this.key = key;
+    }
+
+    public String getVersionId() {
+      return versionId;
+    }
+
+    public void setVersionId(String versionId) {
+      this.versionId = versionId;
+    }
+  }
+
+  /**
+   * JAXB entity for child element.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Error", namespace = "http://s3.amazonaws"
+      + ".com/doc/2006-03-01/")
+  public static class Error {
+
+    @XmlElement(name = "Key")
+    private String key;
+
+    @XmlElement(name = "Code")
+    private String code;
+
+    @XmlElement(name = "Message")
+    private String message;
+
+    public Error() {
+    }
+
+    public Error(String key, String code, String message) {
+      this.key = key;
+      this.code = code;
+      this.message = message;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public void setKey(String key) {
+      this.key = key;
+    }
+
+    public String getCode() {
+      return code;
+    }
+
+    public void setCode(String code) {
+      this.code = code;
+    }
+
+    public String getMessage() {
+      return message;
+    }
+
+    public void setMessage(String message) {
+      this.message = message;
+    }
+  }
+}

+ 86 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectMultiDelete.java

@@ -0,0 +1,86 @@
+/*
+ * 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.s3.endpoint;
+
+import javax.ws.rs.core.Response;
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteRequest.DeleteObject;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+
+import com.google.common.collect.Sets;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test object multi delete.
+ */
+public class TestObjectMultiDelete {
+
+  @Test
+  public void delete() throws IOException, OS3Exception, JAXBException {
+    //GIVEN
+    OzoneClient client = new OzoneClientStub();
+    client.getObjectStore().createS3Bucket("bilbo", "b1");
+
+    String volumeName = client.getObjectStore().getOzoneVolumeName("b1");
+
+    OzoneBucket bucket =
+        client.getObjectStore().getVolume(volumeName).getBucket("b1");
+
+    bucket.createKey("key1", 0).close();
+    bucket.createKey("key2", 0).close();
+    bucket.createKey("key3", 0).close();
+
+    BucketEndpoint rest = new BucketEndpoint();
+    rest.setClient(client);
+
+    MultiDeleteRequest mdr = new MultiDeleteRequest();
+    mdr.getObjects().add(new DeleteObject("key1"));
+    mdr.getObjects().add(new DeleteObject("key2"));
+    mdr.getObjects().add(new DeleteObject("key4"));
+
+    //WHEN
+    Response response = rest.multiDelete("b1", "", mdr);
+
+    //THEN
+    MultiDeleteResponse mdresponse = (MultiDeleteResponse) response.getEntity();
+
+    Set<String> keysAtTheEnd = Sets.newHashSet(bucket.listKeys("")).stream()
+        .map(OzoneKey::getName)
+        .collect(Collectors.toSet());
+
+    Set<String> expectedResult = new HashSet<>();
+    expectedResult.add("key3");
+
+    //THEN
+    Assert.assertEquals(expectedResult, keysAtTheEnd);
+    Assert.assertEquals(3, mdresponse.getDeletedObjects().size());
+    Assert.assertEquals(0, mdresponse.getErrors().size());
+  }
+}