Ver código fonte

HDDS-517. Implement HeadObject REST endpoint. Contributed by LiXin Ge.

(cherry picked from commit cd280514b84e8a2915f274883a07f152b772ebad)
Márton Elek 6 anos atrás
pai
commit
b64fcc71f1

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

@@ -42,6 +42,8 @@ public final class S3ErrorTable {
       "BucketNotEmpty", "The bucket you tried to delete is not empty.",
       HTTP_CONFLICT);
 
+  public static final OS3Exception NO_SUCH_OBJECT = new OS3Exception(
+      "NoSuchObject", "The specified object does not exist", HTTP_NOT_FOUND);
 
   /**
    * Create a new instance of Error.
@@ -60,7 +62,8 @@ public final class S3ErrorTable {
    * Resources, which can be defined in OS3Exception.
    */
   public enum Resource {
-    BUCKET("Bucket");
+    BUCKET("Bucket"),
+    OBJECT("Object");
 
     private final String resource;
 

+ 27 - 11
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/HeadObject.java

@@ -18,41 +18,57 @@
 package org.apache.hadoop.ozone.s3.object;
 
 import javax.ws.rs.HEAD;
-import javax.ws.rs.HeaderParam;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.io.IOException;
-import java.io.InputStream;
 
-import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKeyDetails;
 import org.apache.hadoop.ozone.s3.EndpointBase;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+
+import org.apache.http.HttpStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Get object info rest endpoint.
  */
 @Path("/{volume}/{bucket}/{path:.+}")
 public class HeadObject extends EndpointBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(HeadObject.class);
 
   @HEAD
   @Produces(MediaType.APPLICATION_XML)
   public Response head(
       @PathParam("volume") String volumeName,
       @PathParam("bucket") String bucketName,
-      @PathParam("path") String keyPath,
-      @HeaderParam("Content-Length") long length,
-      InputStream body) throws IOException {
+      @PathParam("path") String keyPath) throws Exception {
+    OzoneKeyDetails key;
 
-    OzoneBucket bucket = getBucket(volumeName, bucketName);
-    OzoneKeyDetails key = bucket.getKey(keyPath);
+    try {
+      key = getVolume(volumeName).getBucket(bucketName).getKey(keyPath);
+      // TODO: return the specified range bytes of this object.
+    } catch (IOException ex) {
+      LOG.error("Exception occurred in HeadObject", ex);
+      if (ex.getMessage().contains("KEY_NOT_FOUND")) {
+        OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
+            .NO_SUCH_OBJECT, S3ErrorTable.Resource.OBJECT);
+        throw os3Exception;
+      } else {
+        throw ex;
+      }
+    }
 
-    return Response.
-        ok()
+    return Response.ok().status(HttpStatus.SC_OK)
+        .header("Last-Modified", key.getModificationTime())
+        .header("ETag", "" + key.getModificationTime())
         .header("Content-Length", key.getDataSize())
+        .header("Content-Type", "binary/octet-stream")
         .build();
-
   }
 }

+ 5 - 1
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java

@@ -107,7 +107,11 @@ public class OzoneBucketStub extends OzoneBucket {
 
   @Override
   public OzoneKeyDetails getKey(String key) throws IOException {
-    return keyDetails.get(key);
+    if (keyDetails.containsKey(key)) {
+      return keyDetails.get(key);
+    } else {
+      throw new IOException("Lookup key failed, error:KEY_NOT_FOUND");
+    }
   }
 
   @Override

+ 95 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/TestHeadObject.java

@@ -0,0 +1,95 @@
+/*
+ * 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.object;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.ozone.client.*;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.ws.rs.core.Response;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import java.io.IOException;
+
+import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
+
+/**
+ * Test head object.
+ */
+public class TestHeadObject {
+  private String volName = "vol1";
+  private String bucketName = "b1";
+  private OzoneClientStub clientStub;
+  private ObjectStore objectStoreStub;
+  private HeadObject headObject;
+  private OzoneBucket bucket;
+
+  @Before
+  public void setup() throws IOException {
+    //Create client stub and object store stub.
+    clientStub = new OzoneClientStub();
+    objectStoreStub = clientStub.getObjectStore();
+
+    // Create volume and bucket
+    objectStoreStub.createVolume(volName);
+    OzoneVolume volumeStub = objectStoreStub.getVolume(volName);
+    volumeStub.createBucket(bucketName);
+    bucket = objectStoreStub.getVolume(volName).getBucket(bucketName);
+
+    // Create HeadBucket and setClient to OzoneClientStub
+    headObject = new HeadObject();
+    headObject.setClient(clientStub);
+  }
+
+  @Test
+  public void testHeadObject() throws Exception {
+    //GIVEN
+    String value = RandomStringUtils.randomAlphanumeric(32);
+    OzoneOutputStream out = bucket.createKey("key1",
+        value.getBytes().length, ReplicationType.STAND_ALONE,
+        ReplicationFactor.ONE);
+    out.write(value.getBytes());
+    out.close();
+
+    //WHEN
+    Response response = headObject.head(volName, bucketName, "key1");
+
+    //THEN
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(value.getBytes().length,
+        Long.parseLong(response.getHeaderString("Content-Length")));
+  }
+
+  @Test
+  public void testHeadFailByBadName() throws Exception {
+    //Head an object that doesn't exist.
+    try {
+      headObject.head(volName, bucketName, "badKeyName");
+    } catch (OS3Exception ex) {
+      Assert.assertTrue(ex.getCode().contains("NoSuchObject"));
+      Assert.assertTrue(ex.getErrorMessage().contains("object does not exist"));
+      Assert.assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
+    }
+  }
+}