HDDS-518. Implement PutObject Rest endpoint. Contributed by chencan.
This commit is contained in:
parent
cdad91c039
commit
ae268b4894
|
@ -17,45 +17,76 @@
|
|||
*/
|
||||
package org.apache.hadoop.ozone.s3.object;
|
||||
|
||||
import javax.ws.rs.DefaultValue;
|
||||
import javax.ws.rs.HeaderParam;
|
||||
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;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
||||
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.ozone.client.OzoneBucket;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||
import org.apache.hadoop.ozone.s3.EndpointBase;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.ozone.s3.EndpointBase;
|
||||
import org.apache.http.HttpStatus;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* File upload.
|
||||
*/
|
||||
@Path("/{volume}/{bucket}/{path:.+}")
|
||||
@Path("/{bucket}/{path:.+}")
|
||||
public class PutObject extends EndpointBase {
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(PutObject.class);
|
||||
|
||||
@PUT
|
||||
@Produces(MediaType.APPLICATION_XML)
|
||||
public void put(
|
||||
@PathParam("volume") String volumeName,
|
||||
public Response put(
|
||||
@Context HttpHeaders headers,
|
||||
@PathParam("bucket") String bucketName,
|
||||
@PathParam("path") String keyPath,
|
||||
@DefaultValue("STAND_ALONE" ) @QueryParam("replicationType")
|
||||
ReplicationType replicationType,
|
||||
@DefaultValue("ONE") @QueryParam("replicationFactor")
|
||||
ReplicationFactor replicationFactor,
|
||||
@DefaultValue("32 * 1024 * 1024") @QueryParam("chunkSize")
|
||||
String chunkSize,
|
||||
@HeaderParam("Content-Length") long length,
|
||||
InputStream body) throws IOException {
|
||||
|
||||
OzoneBucket bucket = getBucket(volumeName, bucketName);
|
||||
try {
|
||||
Configuration config = new OzoneConfiguration();
|
||||
config.set(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, chunkSize);
|
||||
|
||||
OzoneOutputStream output = bucket
|
||||
.createKey(keyPath, length, ReplicationType.STAND_ALONE,
|
||||
ReplicationFactor.ONE);
|
||||
OzoneBucket bucket = getVolume(getOzoneVolumeName(bucketName))
|
||||
.getBucket(bucketName);
|
||||
OzoneOutputStream output = bucket
|
||||
.createKey(keyPath, length, replicationType, replicationFactor);
|
||||
|
||||
IOUtils.copy(body, output);
|
||||
output.close();
|
||||
IOUtils.copy(body, output);
|
||||
output.close();
|
||||
|
||||
return Response.ok().status(HttpStatus.SC_OK)
|
||||
.header("Content-Length", length)
|
||||
.build();
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Exception occurred in PutObject", ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* 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 javax.ws.rs.core.HttpHeaders;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
|
||||
import org.apache.hadoop.ozone.client.ObjectStore;
|
||||
import org.apache.hadoop.ozone.client.OzoneClientStub;
|
||||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
||||
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
/**
|
||||
* Test put object.
|
||||
*/
|
||||
public class TestPutObject {
|
||||
public static final String CONTENT = "0123456789";
|
||||
private String userName = "ozone";
|
||||
private String bucketName = "b1";
|
||||
private String keyName = "key1";
|
||||
private OzoneClientStub clientStub;
|
||||
private ObjectStore objectStoreStub;
|
||||
private PutObject putObject;
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException {
|
||||
//Create client stub and object store stub.
|
||||
clientStub = new OzoneClientStub();
|
||||
objectStoreStub = clientStub.getObjectStore();
|
||||
|
||||
// Create bucket
|
||||
objectStoreStub.createS3Bucket(userName, bucketName);
|
||||
|
||||
// Create PutObject and setClient to OzoneClientStub
|
||||
putObject = new PutObject();
|
||||
putObject.setClient(clientStub);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPutObject() throws IOException {
|
||||
//GIVEN
|
||||
HttpHeaders headers = Mockito.mock(HttpHeaders.class);
|
||||
ByteArrayInputStream body = new ByteArrayInputStream(CONTENT.getBytes());
|
||||
|
||||
//WHEN
|
||||
Response response = putObject.put(headers, bucketName, keyName,
|
||||
ReplicationType.STAND_ALONE, ReplicationFactor.ONE, "32 * 1024 * 1024",
|
||||
CONTENT.length(), body);
|
||||
|
||||
//THEN
|
||||
String volumeName = clientStub.getObjectStore().getOzoneVolumeName(bucketName);
|
||||
OzoneInputStream ozoneInputStream =
|
||||
clientStub.getObjectStore().getVolume(volumeName).getBucket(bucketName)
|
||||
.readKey(keyName);
|
||||
String keyContent =
|
||||
IOUtils.toString(ozoneInputStream, Charset.forName("UTF-8"));
|
||||
|
||||
Assert.assertEquals(200, response.getStatus());
|
||||
Assert.assertEquals(CONTENT, keyContent);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue