HDDS-517. Implement HeadObject REST endpoint. Contributed by LiXin Ge.
This commit is contained in:
parent
b39b802422
commit
cd280514b8
@ -42,6 +42,8 @@ private S3ErrorTable() {
|
|||||||
"BucketNotEmpty", "The bucket you tried to delete is not empty.",
|
"BucketNotEmpty", "The bucket you tried to delete is not empty.",
|
||||||
HTTP_CONFLICT);
|
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.
|
* Create a new instance of Error.
|
||||||
@ -60,7 +62,8 @@ public static OS3Exception newError(OS3Exception e, Resource resource) {
|
|||||||
* Resources, which can be defined in OS3Exception.
|
* Resources, which can be defined in OS3Exception.
|
||||||
*/
|
*/
|
||||||
public enum Resource {
|
public enum Resource {
|
||||||
BUCKET("Bucket");
|
BUCKET("Bucket"),
|
||||||
|
OBJECT("Object");
|
||||||
|
|
||||||
private final String resource;
|
private final String resource;
|
||||||
|
|
||||||
|
@ -18,41 +18,57 @@
|
|||||||
package org.apache.hadoop.ozone.s3.object;
|
package org.apache.hadoop.ozone.s3.object;
|
||||||
|
|
||||||
import javax.ws.rs.HEAD;
|
import javax.ws.rs.HEAD;
|
||||||
import javax.ws.rs.HeaderParam;
|
|
||||||
import javax.ws.rs.Path;
|
import javax.ws.rs.Path;
|
||||||
import javax.ws.rs.PathParam;
|
import javax.ws.rs.PathParam;
|
||||||
import javax.ws.rs.Produces;
|
import javax.ws.rs.Produces;
|
||||||
import javax.ws.rs.core.MediaType;
|
import javax.ws.rs.core.MediaType;
|
||||||
import javax.ws.rs.core.Response;
|
import javax.ws.rs.core.Response;
|
||||||
import java.io.IOException;
|
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.client.OzoneKeyDetails;
|
||||||
import org.apache.hadoop.ozone.s3.EndpointBase;
|
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.
|
* Get object info rest endpoint.
|
||||||
*/
|
*/
|
||||||
@Path("/{volume}/{bucket}/{path:.+}")
|
@Path("/{volume}/{bucket}/{path:.+}")
|
||||||
public class HeadObject extends EndpointBase {
|
public class HeadObject extends EndpointBase {
|
||||||
|
private static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(HeadObject.class);
|
||||||
|
|
||||||
@HEAD
|
@HEAD
|
||||||
@Produces(MediaType.APPLICATION_XML)
|
@Produces(MediaType.APPLICATION_XML)
|
||||||
public Response head(
|
public Response head(
|
||||||
@PathParam("volume") String volumeName,
|
@PathParam("volume") String volumeName,
|
||||||
@PathParam("bucket") String bucketName,
|
@PathParam("bucket") String bucketName,
|
||||||
@PathParam("path") String keyPath,
|
@PathParam("path") String keyPath) throws Exception {
|
||||||
@HeaderParam("Content-Length") long length,
|
OzoneKeyDetails key;
|
||||||
InputStream body) throws IOException {
|
|
||||||
|
|
||||||
OzoneBucket bucket = getBucket(volumeName, bucketName);
|
try {
|
||||||
OzoneKeyDetails key = bucket.getKey(keyPath);
|
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.
|
return Response.ok().status(HttpStatus.SC_OK)
|
||||||
ok()
|
.header("Last-Modified", key.getModificationTime())
|
||||||
|
.header("ETag", "" + key.getModificationTime())
|
||||||
.header("Content-Length", key.getDataSize())
|
.header("Content-Length", key.getDataSize())
|
||||||
|
.header("Content-Type", "binary/octet-stream")
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -107,7 +107,11 @@ public OzoneInputStream readKey(String key) throws IOException {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OzoneKeyDetails getKey(String key) throws IOException {
|
public OzoneKeyDetails getKey(String key) throws IOException {
|
||||||
|
if (keyDetails.containsKey(key)) {
|
||||||
return keyDetails.get(key);
|
return keyDetails.get(key);
|
||||||
|
} else {
|
||||||
|
throw new IOException("Lookup key failed, error:KEY_NOT_FOUND");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -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());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue
Block a user