HDFS-12030. Ozone: CLI: support infoKey command. Contributed by Yiqun Lin.

This commit is contained in:
Weiwei Yang 2017-07-01 23:01:09 +08:00 committed by Owen O'Malley
parent 4b63e3e504
commit 5379ffa8de
11 changed files with 423 additions and 24 deletions

View File

@ -508,4 +508,66 @@ public class OzoneBucket {
}
}
}
/**
* Get info of the specified key.
*/
public OzoneKey getKeyInfo(String keyName) throws OzoneException {
if ((keyName == null) || keyName.isEmpty()) {
throw new OzoneRestClientException(
"Unable to get key info, key name is null or empty");
}
HttpGet getRequest = null;
try (CloseableHttpClient httpClient = OzoneClientUtils.newHttpClient()) {
OzoneRestClient client = getVolume().getClient();
URIBuilder builder = new URIBuilder(volume.getClient().getEndPointURI());
builder
.setPath("/" + getVolume().getVolumeName() + "/" + getBucketName()
+ "/" + keyName)
.setParameter(Header.OZONE_LIST_QUERY_TAG,
Header.OZONE_LIST_QUERY_KEY)
.build();
getRequest = client.getHttpGet(builder.toString());
return executeGetKeyInfo(getRequest, httpClient);
} catch (IOException | URISyntaxException e) {
throw new OzoneRestClientException(e.getMessage());
} finally {
OzoneClientUtils.releaseConnection(getRequest);
}
}
/**
* Execute get Key info.
*
* @param getRequest - HttpGet
* @param httpClient - HttpClient
* @return List<OzoneKey>
* @throws IOException
* @throws OzoneException
*/
private OzoneKey executeGetKeyInfo(HttpGet getRequest,
CloseableHttpClient httpClient) throws IOException, OzoneException {
HttpEntity entity = null;
try {
HttpResponse response = httpClient.execute(getRequest);
int errorCode = response.getStatusLine().getStatusCode();
entity = response.getEntity();
if (entity == null) {
throw new OzoneRestClientException("Unexpected null in http payload");
}
if (errorCode == HTTP_OK) {
OzoneKey key = new OzoneKey(
KeyInfo.parse(EntityUtils.toString(entity)));
return key;
}
throw OzoneException.parse(EntityUtils.toString(entity));
} finally {
if (entity != null) {
EntityUtils.consumeQuietly(entity);
}
}
}
}

View File

@ -20,9 +20,12 @@ package org.apache.hadoop.ozone.web.handlers;
import org.apache.commons.codec.binary.Hex;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.apache.hadoop.ozone.web.headers.Header;
import org.apache.hadoop.ozone.web.interfaces.Keys;
import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
import org.apache.hadoop.ozone.web.response.KeyInfo;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import javax.ws.rs.core.HttpHeaders;
@ -44,19 +47,21 @@ import static java.net.HttpURLConnection.HTTP_OK;
public class KeyHandler implements Keys {
/**
* Gets the Key if it exists.
* Gets the Key/key information if it exists.
*
* @param volume Storage Volume
* @param bucket Name of the bucket
* @param req Request
* @param info - UriInfo
* @param key Name of the key
* @param info Tag info
* @param req Request
* @param uriInfo Uri Info
* @param headers Http Header
* @return Response
* @throws OzoneException
*/
@Override
public Response getKey(String volume, String bucket, String key,
Request req, UriInfo info, HttpHeaders headers)
public Response getKey(String volume, String bucket, String key, String info,
Request req, UriInfo uriInfo, HttpHeaders headers)
throws OzoneException {
return new KeyProcessTemplate() {
/**
@ -77,13 +82,40 @@ public class KeyHandler implements Keys {
@Override
public Response doProcess(KeyArgs args, InputStream input,
Request request, HttpHeaders headers,
UriInfo info)
UriInfo uriInfo)
throws IOException, OzoneException, NoSuchAlgorithmException {
StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
LengthInputStream stream = fs.newKeyReader(args);
return OzoneUtils.getResponse(args, HTTP_OK, stream);
if (info == null) {
return getKey(args);
} else if (info.equals(Header.OZONE_LIST_QUERY_KEY)) {
return getKeyInfo(args);
}
OzoneException ozException = ErrorTable
.newError(ErrorTable.INVALID_QUERY_PARAM, args);
ozException.setMessage("Unrecognized query param : " + info);
throw ozException;
}
}.handleCall(volume, bucket, key, req, headers, info, null);
}.handleCall(volume, bucket, key, req, headers, uriInfo, null);
}
/**
* Gets the Key if it exists.
*/
private Response getKey(KeyArgs args)
throws IOException, OzoneException {
StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
LengthInputStream stream = fs.newKeyReader(args);
return OzoneUtils.getResponse(args, HTTP_OK, stream);
}
/**
* Gets the Key information if it exists.
*/
private Response getKeyInfo(KeyArgs args)
throws IOException, OzoneException {
StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
KeyInfo keyInfo = fs.getKeyInfo(args);
return OzoneUtils.getResponse(args, HTTP_OK, keyInfo.toJsonString());
}
/**

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.ozone.web.interfaces;
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.apache.hadoop.ozone.web.headers.Header;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
@ -26,7 +27,7 @@ import javax.ws.rs.GET;
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;
@ -76,7 +77,9 @@ public interface Keys {
* @param volume Storage Volume
* @param bucket Name of the bucket
* @param keys Object Name
* @param info Tag info
* @param req Request
* @param uriInfo Uri info
* @param headers Http Header
*
* @return Response
@ -84,12 +87,13 @@ public interface Keys {
* @throws OzoneException
*/
@GET
@Produces(MediaType.APPLICATION_OCTET_STREAM)
Response getKey(@PathParam("volume") String volume,
@PathParam("bucket") String bucket,
@PathParam("keys") String keys,
@QueryParam(Header.OZONE_LIST_QUERY_TAG)
String info,
@Context Request req,
@Context UriInfo info,
@Context UriInfo uriInfo,
@Context HttpHeaders headers)
throws OzoneException;

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.ozone.web.handlers.ListArgs;
import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.web.response.BucketInfo;
import org.apache.hadoop.ozone.web.response.KeyInfo;
import org.apache.hadoop.ozone.web.response.ListBuckets;
import org.apache.hadoop.ozone.web.response.ListKeys;
import org.apache.hadoop.ozone.web.response.ListVolumes;
@ -274,6 +275,18 @@ public interface StorageHandler {
*/
ListKeys listKeys(ListArgs args) throws IOException, OzoneException;
/**
* Get information of the specified Key.
*
* @param args Key Args
*
* @return KeyInfo
*
* @throws IOException
* @throws OzoneException
*/
KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException;
/**
* Closes all the opened resources.
*/

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.web.request.OzoneQuota;
import org.apache.hadoop.ozone.web.response.BucketInfo;
import org.apache.hadoop.ozone.web.response.KeyInfo;
import org.apache.hadoop.ozone.web.response.ListBuckets;
import org.apache.hadoop.ozone.web.response.ListKeys;
import org.apache.hadoop.ozone.web.response.ListVolumes;
@ -353,6 +354,23 @@ public class LocalStorageHandler implements StorageHandler {
}
/**
* Get information of the specified Key.
*
* @param args Key Args
*
* @return KeyInfo
*
* @throws IOException
* @throws OzoneException
*/
@Override
public KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException {
OzoneMetadataManager oz = OzoneMetadataManager
.getOzoneMetadataManager(conf);
return oz.getKeyInfo(args);
}
@Override
public void close() {
//No resource to close, do nothing.

View File

@ -1108,6 +1108,42 @@ public final class OzoneMetadataManager {
}
}
/**
* Get the Key information for a given key.
*
* @param args - Key Args
* @return KeyInfo - Key Information
* @throws OzoneException
*/
public KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException {
lock.readLock().lock();
try {
byte[] bucketInfo = metadataDB
.get(args.getParentName().getBytes(encoding));
if (bucketInfo == null) {
throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
}
byte[] bucketListBytes = userDB
.get(args.getParentName().getBytes(encoding));
if (bucketListBytes == null) {
throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
}
byte[] objectBytes = metadataDB
.get(args.getResourceName().getBytes(encoding));
if (objectBytes == null) {
throw ErrorTable.newError(ErrorTable.INVALID_KEY, args);
}
return KeyInfo.parse(new String(objectBytes, encoding));
} catch (IOException e) {
throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, e);
} finally {
lock.readLock().unlock();
}
}
/**
* This is used in updates to volume metadata.
*/

View File

@ -18,18 +18,22 @@
package org.apache.hadoop.ozone.web.ozShell.keys;
import org.apache.commons.cli.CommandLine;
import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.apache.hadoop.ozone.web.ozShell.Handler;
import org.apache.hadoop.ozone.web.ozShell.Shell;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.file.Path;
import java.nio.file.Paths;
import org.apache.commons.cli.CommandLine;
import org.apache.hadoop.ozone.web.client.OzoneBucket;
import org.apache.hadoop.ozone.web.client.OzoneKey;
import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
import org.apache.hadoop.ozone.web.client.OzoneVolume;
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.apache.hadoop.ozone.web.ozShell.Handler;
import org.apache.hadoop.ozone.web.ozShell.Shell;
import org.apache.hadoop.ozone.web.utils.JsonUtils;
/**
* Executes Info Object.
*/
@ -84,10 +88,11 @@ public class InfoKeyHandler extends Handler {
client.setEndPointURI(ozoneURI);
client.setUserAuth(userName);
OzoneVolume vol = client.getVolume(volumeName);
OzoneBucket bucket = vol.getBucket(bucketName);
OzoneKey key = bucket.getKeyInfo(keyName);
// OzoneVolume vol = client.getVolume(volumeName);
// OzoneBucket bucket = vol.createBucket(bucketName);
throw new OzoneRestClientException("Not supported yet");
System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
key.getObjectInfo().toJsonString()));
}
}

View File

@ -469,6 +469,23 @@ public final class DistributedStorageHandler implements StorageHandler {
keySpaceManagerClient.deleteKey(keyArgs);
}
@Override
public KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException {
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
.setVolumeName(args.getVolumeName())
.setBucketName(args.getBucketName())
.setKeyName(args.getKeyName())
.build();
KsmKeyInfo ksmKeyInfo = keySpaceManagerClient.lookupKey(keyArgs);
KeyInfo keyInfo = new KeyInfo();
keyInfo.setVersion(0);
keyInfo.setKeyName(ksmKeyInfo.getKeyName());
keyInfo.setSize(ksmKeyInfo.getDataSize());
return keyInfo;
}
@Override
public ListKeys listKeys(ListArgs args) throws IOException, OzoneException {
ListKeys result = new ListKeys();

View File

@ -930,4 +930,40 @@ public class TestKeySpaceManager {
volumes = storageHandler.listVolumes(listVolumeArgs);
Assert.assertEquals(0, volumes.getVolumes().size());
}
/**
* Test get key information.
*
* @throws IOException
* @throws OzoneException
*/
@Test
public void testGetKeyInfo() throws IOException, OzoneException {
String userName = "user" + RandomStringUtils.randomNumeric(5);
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
createVolumeArgs.setUserName(userName);
createVolumeArgs.setAdminName(adminName);
storageHandler.createVolume(createVolumeArgs);
BucketArgs bucketArgs = new BucketArgs(bucketName, createVolumeArgs);
bucketArgs.setAddAcls(new LinkedList<>());
bucketArgs.setRemoveAcls(new LinkedList<>());
bucketArgs.setStorageType(StorageType.DISK);
storageHandler.createBucket(bucketArgs);
String keyName = "testKey";
KeyArgs keyArgs = new KeyArgs(keyName, bucketArgs);
keyArgs.setSize(4096);
OutputStream stream = storageHandler.newKeyWriter(keyArgs);
stream.close();
KeyInfo keyInfo = storageHandler.getKeyInfo(keyArgs);
Assert.assertEquals(keyName, keyInfo.getKeyName());
Assert.assertEquals(4096, keyInfo.getSize());
}
}

View File

@ -0,0 +1,165 @@
/**
* 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.ozShell;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.PrintStream;
import java.net.URISyntaxException;
import java.util.Random;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.Status;
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.apache.hadoop.ozone.web.ozShell.Shell;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.ToolRunner;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
/**
* This test class specified for testing Ozone shell command.
*/
public class TestOzoneShell {
/**
* Set the timeout for every test.
*/
@Rule
public Timeout testTimeout = new Timeout(300000);
private static String url;
private static File baseDir;
private static OzoneConfiguration conf = null;
private static MiniOzoneCluster cluster = null;
private static Shell shell = null;
private final ByteArrayOutputStream out = new ByteArrayOutputStream();
private final ByteArrayOutputStream err = new ByteArrayOutputStream();
/**
* Create a MiniDFSCluster for testing with using distributed Ozone
* handler type.
*
* @throws IOException
*/
@BeforeClass
public static void init()
throws IOException, URISyntaxException, OzoneException {
conf = new OzoneConfiguration();
String path = GenericTestUtils.getTempPath(
TestOzoneShell.class.getSimpleName());
baseDir = new File(path);
baseDir.mkdirs();
path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
conf.setQuietMode(false);
shell = new Shell();
shell.setConf(conf);
cluster = new MiniOzoneCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = dataNode.getInfoPort();
url = String.format("http://localhost:%d", port);
}
/**
* shutdown MiniDFSCluster.
*/
@AfterClass
public static void shutdown() {
if (cluster != null) {
cluster.shutdown();
}
if (baseDir != null) {
FileUtil.fullyDelete(baseDir, true);
}
}
@Test
public void testGetKeyInfo() throws Exception {
// create a volume
String volume = "volume" + RandomStringUtils.randomNumeric(5);
String[] args = new String[] {"-createVolume", url + "/" + volume, "-user",
"bilbo", "-root"};
assertEquals(0, ToolRunner.run(shell, args));
// create a bucket
String bucket = "bucket" + RandomStringUtils.randomNumeric(5);
args = new String[] {"-createBucket", url + "/" + volume + "/" + bucket};
assertEquals(0, ToolRunner.run(shell, args));
// write a new file that used for putting key
String key = "key" + RandomStringUtils.randomNumeric(5);
File tmpFile = new File(baseDir, "/testfile");
FileOutputStream randFile = new FileOutputStream(tmpFile);
Random r = new Random();
for (int x = 0; x < 10; x++) {
char c = (char) (r.nextInt(26) + 'a');
randFile.write(c);
}
randFile.close();
// create the key in above bucket
args = new String[] {"-putKey",
url + "/" + volume + "/" + bucket + "/" + key, "-file",
tmpFile.getAbsolutePath()};
assertEquals(0, ToolRunner.run(shell, args));
System.setOut(new PrintStream(out));
System.setErr(new PrintStream(err));
args = new String[] {"-infoKey",
url + "/" + volume + "/" + bucket + "/" + key};
// verify the response output
assertEquals(0, ToolRunner.run(shell, args));
assertTrue(out.toString().contains(key));
// reset stream
out.reset();
err.reset();
// get the info of a non-exist key
args = new String[] {"-infoKey",
url + "/" + volume + "/" + bucket + "/invalid-key"};
// verify the response output
// get the non-exist key info should be failed
assertEquals(1, ToolRunner.run(shell, args));
assertTrue(err.toString().contains(Status.KEY_NOT_FOUND.toString()));
}
}

View File

@ -315,7 +315,6 @@ public class TestKeys {
}
}
@Test
public void testPutAndListKey() throws OzoneException, IOException {
PutHelper helper = new PutHelper();
@ -353,4 +352,16 @@ public class TestKeys {
ErrorTable.INVALID_RESOURCE_NAME.getMessage(), e);
}
}
@Test
public void testGetKeyInfo() throws OzoneException, IOException {
PutHelper helper = new PutHelper();
String keyName = helper.putKey();
assertNotNull(helper.getBucket());
assertNotNull(helper.getFile());
OzoneKey keyInfo = helper.getBucket().getKeyInfo(keyName);
assertNotNull(keyInfo.getObjectInfo());
assertEquals(keyName, keyInfo.getObjectInfo().getKeyName());
}
}