浏览代码

HDDS-519. Implement ListBucket REST endpoint. Contributed by LiXin Ge.

Márton Elek 6 年之前
父节点
当前提交
5033deb13b

+ 74 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/ListBucket.java

@@ -0,0 +1,74 @@
+/*
+ * 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.bucket;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.s3.EndpointBase;
+import org.apache.hadoop.ozone.s3.commontypes.BucketMetadata;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.*;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Iterator;
+
+/**
+ * List Object Rest endpoint.
+ */
+@Path("/{volume}")
+public class ListBucket extends EndpointBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ListBucket.class);
+
+  @GET
+  @Produces(MediaType.APPLICATION_XML)
+  public ListBucketResponse get(@PathParam("volume") String volumeName)
+      throws OS3Exception, IOException {
+    OzoneVolume volume;
+    try {
+      volume = getVolume(volumeName);
+    } catch (NotFoundException ex) {
+      LOG.error("Exception occurred in ListBucket: volume {} not found.",
+          volumeName, ex);
+      OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
+          .NO_SUCH_VOLUME, S3ErrorTable.Resource.VOLUME);
+      throw os3Exception;
+    } catch (IOException e) {
+      throw e;
+    }
+
+    Iterator<? extends OzoneBucket> volABucketIter = volume.listBuckets(null);
+    ListBucketResponse response = new ListBucketResponse();
+
+    while(volABucketIter.hasNext()) {
+      OzoneBucket next = volABucketIter.next();
+      BucketMetadata bucketMetadata = new BucketMetadata();
+      bucketMetadata.setName(next.getName());
+      bucketMetadata.setCreationDate(
+          Instant.ofEpochMilli(next.getCreationTime()));
+      response.addBucket(bucketMetadata);
+    }
+
+    return response;
+  }
+}

+ 55 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/ListBucketResponse.java

@@ -0,0 +1,55 @@
+/*
+ * 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.bucket;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.ozone.s3.commontypes.BucketMetadata;
+
+import javax.xml.bind.annotation.*;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Response from the ListBucket RPC Call.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlRootElement(name = "ListAllMyBucketsResult",
+    namespace = "http://s3.amazonaws.com/doc/2006-03-01/")
+public class ListBucketResponse {
+  @XmlElementWrapper(name = "Buckets")
+  @XmlElement(name = "Bucket")
+  private List<BucketMetadata> buckets = new ArrayList<>();
+
+  public List<BucketMetadata> getBuckets() {
+    return buckets;
+  }
+
+  @VisibleForTesting
+  public int getBucketsNum() {
+    return buckets.size();
+  }
+
+  public void setBuckets(List<BucketMetadata> buckets) {
+    this.buckets = buckets;
+  }
+
+  public void addBucket(BucketMetadata bucket) {
+    buckets.add(bucket);
+  }
+}

+ 53 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/BucketMetadata.java

@@ -0,0 +1,53 @@
+/*
+ * 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.commontypes;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import java.time.Instant;
+
+/**
+ * Metadata object represents one bucket.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+public class BucketMetadata {
+  @XmlElement(name = "Name")
+  private String name;
+
+  @XmlJavaTypeAdapter(IsoDateAdapter.class)
+  @XmlElement(name = "CreationDate")
+  private Instant creationDate;
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public Instant getCreationDate() {
+    return creationDate;
+  }
+
+  public void setCreationDate(Instant creationDate) {
+    this.creationDate = creationDate;
+  }
+}

+ 5 - 1
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java

@@ -35,6 +35,9 @@ public final class S3ErrorTable {
   public static final OS3Exception INVALID_URI = new OS3Exception("InvalidURI",
       "Couldn't parse the specified URI.", HTTP_BAD_REQUEST);
 
+  public static final OS3Exception NO_SUCH_VOLUME = new OS3Exception(
+      "NoSuchVolume", "The specified volume does not exist", HTTP_NOT_FOUND);
+
   public static final OS3Exception NO_SUCH_BUCKET = new OS3Exception(
       "NoSuchBucket", "The specified bucket does not exist", HTTP_NOT_FOUND);
 
@@ -69,7 +72,8 @@ public final class S3ErrorTable {
   public enum Resource {
     BUCKET("Bucket"),
     OBJECT("Object"),
-    HEADER("header");
+    HEADER("header"),
+    VOLUME("Volume");
 
     private final String resource;
 

+ 6 - 1
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java

@@ -77,7 +77,12 @@ public class OzoneVolumeStub extends OzoneVolume {
   public Iterator<? extends OzoneBucket> listBuckets(String bucketPrefix) {
     return buckets.values()
         .stream()
-        .filter(bucket -> bucket.getName().startsWith(bucketPrefix))
+        .filter(bucket -> {
+          if (bucketPrefix != null) {
+            return bucket.getName().startsWith(bucketPrefix);
+          } else {
+            return true;
+          }})
         .collect(Collectors.toList())
         .iterator();
   }

+ 97 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestListBucket.java

@@ -0,0 +1,97 @@
+/*
+ * 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.bucket;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.ws.rs.core.Response;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * This class test HeadBucket functionality.
+ */
+public class TestListBucket {
+
+  private String volumeName = "vol1";
+  private OzoneClientStub clientStub;
+  private ObjectStore objectStoreStub;
+  OzoneVolume volumeStub;
+  private ListBucket listBucket;
+
+  @Before
+  public void setup() throws Exception {
+
+    //Create client stub and object store stub.
+    clientStub = new OzoneClientStub();
+    objectStoreStub = clientStub.getObjectStore();
+
+    // Create volume and bucket
+    objectStoreStub.createVolume(volumeName);
+
+    volumeStub = objectStoreStub.getVolume(volumeName);
+    //volumeStub.createBucket(bucketName);
+
+    // Create HeadBucket and setClient to OzoneClientStub
+    listBucket = new ListBucket();
+    listBucket.setClient(clientStub);
+  }
+
+  @Test
+  public void testListBucket() throws Exception {
+    // List operation should success even there is no bucket.
+    ListBucketResponse response = listBucket.get(volumeName);
+    assertEquals(0, response.getBucketsNum());
+
+    String bucketBaseName = "bucket-";
+    for(int i = 0; i < 10; i++) {
+      volumeStub.createBucket(
+          bucketBaseName + RandomStringUtils.randomNumeric(3));
+    }
+    response = listBucket.get(volumeName);
+    assertEquals(10, response.getBucketsNum());
+  }
+
+  @Test
+  public void testListBucketFail() {
+    try {
+      listBucket.get("badVolumeName");
+    } catch (Exception ex) {
+      if (ex instanceof OS3Exception) {
+        assertEquals(S3ErrorTable.NO_SUCH_VOLUME.getCode(),
+            ((OS3Exception) ex).getCode());
+        assertEquals(S3ErrorTable.NO_SUCH_VOLUME.getErrorMessage(), (
+            (OS3Exception) ex).getErrorMessage());
+      } else {
+        fail("testHeadFail failed");
+      }
+      return;
+    }
+  }
+}