HDDS-336. Print out container location information for a specific ozone key . Contributed by LiXin Ge.

This commit is contained in:
Márton Elek 2018-09-03 13:32:55 +02:00
parent 3801436e49
commit 211034a6c2
19 changed files with 779 additions and 72 deletions

View File

@ -258,10 +258,10 @@ public class OzoneBucket {
/**
* Returns information about the key.
* @param key Name of the key.
* @return OzoneKey Information about the key.
* @return OzoneKeyDetails Information about the key.
* @throws IOException
*/
public OzoneKey getKey(String key) throws IOException {
public OzoneKeyDetails getKey(String key) throws IOException {
return proxy.getKeyDetails(volumeName, name, key);
}

View File

@ -25,10 +25,10 @@ import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
import org.apache.hadoop.ozone.client.rest.response.VolumeOwner;
import org.apache.hadoop.ozone.client.rest.response.*;
import java.util.ArrayList;
import java.util.List;
import java.util.HashMap;
import java.util.Map;
@ -112,4 +112,25 @@ public final class OzoneClientUtils {
exceptionToPolicyMap);
return retryPolicy;
}
/**
* Returns a KeyInfoDetails object constructed using fields of the input
* OzoneKeyDetails object.
*
* @param key OzoneKeyDetails instance from which KeyInfo object needs to
* be created.
* @return KeyInfoDetails instance
*/
public static KeyInfoDetails asKeyInfoDetails(OzoneKeyDetails key) {
KeyInfoDetails keyInfo = new KeyInfoDetails();
keyInfo.setKeyName(key.getName());
keyInfo.setCreatedOn(HddsClientUtils.formatDateTime(key.getCreationTime()));
keyInfo.setModifiedOn(
HddsClientUtils.formatDateTime(key.getModificationTime()));
keyInfo.setSize(key.getDataSize());
List<KeyLocation> keyLocations = new ArrayList<>();
key.getOzoneKeyLocations().forEach((a) -> keyLocations.add(new KeyLocation(
a.getContainerID(), a.getLocalID(), a.getLength(), a.getOffset())));
keyInfo.setKeyLocation(keyLocations);
return keyInfo;
}
}

View File

@ -0,0 +1,58 @@
/**
* 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.client;
import java.util.List;
/**
* A class that encapsulates OzoneKeyLocation.
*/
public class OzoneKeyDetails extends OzoneKey {
/**
* A list of block location information to specify replica locations.
*/
private List<OzoneKeyLocation> ozoneKeyLocations;
/**
* Constructs OzoneKeyDetails from OmKeyInfo.
*/
public OzoneKeyDetails(String volumeName, String bucketName, String keyName,
long size, long creationTime, long modificationTime,
List<OzoneKeyLocation> ozoneKeyLocations) {
super(volumeName, bucketName, keyName, size, creationTime,
modificationTime);
this.ozoneKeyLocations = ozoneKeyLocations;
}
/**
* Returns the location detail information of the specific Key.
*/
public List<OzoneKeyLocation> getOzoneKeyLocations() {
return ozoneKeyLocations;
}
/**
* Set details of key location.
* @param ozoneKeyLocations - details of key location
*/
public void setOzoneKeyLocations(List<OzoneKeyLocation> ozoneKeyLocations) {
this.ozoneKeyLocations = ozoneKeyLocations;
}
}

View File

@ -0,0 +1,82 @@
/**
* 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.client;
/**
* One key can be stored in one or more containers as one or more blocks.
* This class represents one such block instance.
*/
public class OzoneKeyLocation {
/**
* Which container this key stored.
*/
private final long containerID;
/**
* Which block this key stored inside a container.
*/
private final long localID;
/**
* Data length of this key replica.
*/
private final long length;
/**
* Offset of this key.
*/
private final long offset;
/**
* Constructs OzoneKeyLocation.
*/
public OzoneKeyLocation(long containerID, long localID,
long length, long offset) {
this.containerID = containerID;
this.localID = localID;
this.length = length;
this.offset = offset;
}
/**
* Returns the containerID of this Key.
*/
public long getContainerID() {
return containerID;
}
/**
* Returns the localID of this Key.
*/
public long getLocalID() {
return localID;
}
/**
* Returns the length of this Key.
*/
public long getLength() {
return length;
}
/**
* Returns the offset of this Key.
*/
public long getOffset() {
return offset;
}
}

View File

@ -20,14 +20,10 @@ package org.apache.hadoop.ozone.client.protocol;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.client.BucketArgs;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.*;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
@ -321,7 +317,7 @@ public interface ClientProtocol {
* @return {@link OzoneKey}
* @throws IOException
*/
OzoneKey getKeyDetails(String volumeName, String bucketName,
OzoneKeyDetails getKeyDetails(String volumeName, String bucketName,
String keyName)
throws IOException;

View File

@ -30,12 +30,8 @@ import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.BucketArgs;
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.*;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
@ -43,7 +39,7 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
import org.apache.hadoop.ozone.client.rest.headers.Header;
import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
import org.apache.hadoop.ozone.client.rest.response.KeyInfoDetails;
import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
import org.apache.hadoop.ozone.om.OMConfigKeys;
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
@ -80,6 +76,7 @@ import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.text.ParseException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.FutureTask;
@ -788,7 +785,7 @@ public class RestClient implements ClientProtocol {
}
@Override
public OzoneKey getKeyDetails(
public OzoneKeyDetails getKeyDetails(
String volumeName, String bucketName, String keyName)
throws IOException {
try {
@ -798,18 +795,24 @@ public class RestClient implements ClientProtocol {
builder.setPath(PATH_SEPARATOR + volumeName +
PATH_SEPARATOR + bucketName + PATH_SEPARATOR + keyName);
builder.setParameter(Header.OZONE_INFO_QUERY_TAG,
Header.OZONE_INFO_QUERY_KEY);
Header.OZONE_INFO_QUERY_KEY_DETAIL);
HttpGet httpGet = new HttpGet(builder.build());
addOzoneHeaders(httpGet);
HttpEntity response = executeHttpRequest(httpGet);
KeyInfo keyInfo =
KeyInfo.parse(EntityUtils.toString(response));
OzoneKey key = new OzoneKey(volumeName,
KeyInfoDetails keyInfo =
KeyInfoDetails.parse(EntityUtils.toString(response));
List<OzoneKeyLocation> ozoneKeyLocations = new ArrayList<>();
keyInfo.getKeyLocations().forEach((a) -> ozoneKeyLocations.add(
new OzoneKeyLocation(a.getContainerID(), a.getLocalID(),
a.getLength(), a.getOffset())));
OzoneKeyDetails key = new OzoneKeyDetails(volumeName,
bucketName,
keyInfo.getKeyName(),
keyInfo.getSize(),
HddsClientUtils.formatDateTime(keyInfo.getCreatedOn()),
HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()));
HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()),
ozoneKeyLocations);
EntityUtils.consume(response);
return key;
} catch (URISyntaxException | ParseException e) {

View File

@ -30,11 +30,8 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.BucketArgs;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.*;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.ozone.client.VolumeArgs;
@ -73,10 +70,7 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.UUID;
import java.util.*;
import java.util.stream.Collectors;
/**
@ -548,7 +542,7 @@ public class RpcClient implements ClientProtocol {
}
@Override
public OzoneKey getKeyDetails(
public OzoneKeyDetails getKeyDetails(
String volumeName, String bucketName, String keyName)
throws IOException {
Preconditions.checkNotNull(volumeName);
@ -560,12 +554,18 @@ public class RpcClient implements ClientProtocol {
.setKeyName(keyName)
.build();
OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
return new OzoneKey(keyInfo.getVolumeName(),
List<OzoneKeyLocation> ozoneKeyLocations = new ArrayList<>();
keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly().forEach(
(a) -> ozoneKeyLocations.add(new OzoneKeyLocation(a.getContainerID(),
a.getLocalID(), a.getLength(), a.getOffset())));
return new OzoneKeyDetails(keyInfo.getVolumeName(),
keyInfo.getBucketName(),
keyInfo.getKeyName(),
keyInfo.getDataSize(),
keyInfo.getCreationTime(),
keyInfo.getModificationTime());
keyInfo.getModificationTime(),
ozoneKeyLocations);
}
@Override

View File

@ -44,6 +44,7 @@ public final class Header {
public static final String OZONE_INFO_QUERY_VOLUME = "volume";
public static final String OZONE_INFO_QUERY_BUCKET = "bucket";
public static final String OZONE_INFO_QUERY_KEY = "key";
public static final String OZONE_INFO_QUERY_KEY_DETAIL = "key-detail";
public static final String OZONE_REQUEST_ID = "x-ozone-request-id";
public static final String OZONE_SERVER_NAME = "x-ozone-server-name";

View File

@ -0,0 +1,107 @@
/**
* 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.client.rest.response;
import java.io.IOException;
import java.util.List;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectReader;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
/**
* KeyInfoDetails class is used for parsing json response
* when KeyInfoDetails Call is made.
*/
public class KeyInfoDetails extends KeyInfo {
private static final ObjectReader READER =
new ObjectMapper().readerFor(KeyInfoDetails.class);
/**
* a list of Map which maps localID to ContainerID
* to specify replica locations.
*/
private List<KeyLocation> keyLocations;
/**
* Constructor needed for json serialization.
*/
public KeyInfoDetails() {
}
/**
* Set details of key location.
*
* @param locations - details of key location
*/
public void setKeyLocation(List<KeyLocation> locations) {
this.keyLocations = locations;
}
/**
* Returns details of key location.
*
* @return volumeName
*/
public List<KeyLocation> getKeyLocations() {
return keyLocations;
}
/**
* Parse a string to return KeyInfoDetails Object.
*
* @param jsonString Json String
* @return KeyInfoDetails
* @throws IOException
*/
public static KeyInfoDetails parse(String jsonString) throws IOException {
return READER.readValue(jsonString);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
KeyInfoDetails that = (KeyInfoDetails) o;
return new EqualsBuilder()
.append(getVersion(), that.getVersion())
.append(getKeyName(), that.getKeyName())
.append(keyLocations, that.keyLocations)
.isEquals();
}
@Override
public int hashCode() {
return new HashCodeBuilder(21, 33)
.append(getVersion())
.append(getKeyName())
.append(keyLocations)
.toHashCode();
}
}

View File

@ -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.client.rest.response;
/**
* KeyLocation class is used used for parsing json response
* when KeyInfoDetails Call is made.
*/
public class KeyLocation {
/**
* Which container this key stored.
*/
private long containerID;
/**
* Which block this key stored inside a container.
*/
private long localID;
/**
* Data length of this key replica.
*/
private long length;
/**
* Offset of this key.
*/
private long offset;
/**
* Empty constructor for Json serialization.
*/
public KeyLocation() {
}
/**
* Constructs KeyLocation.
*/
public KeyLocation(long containerID, long localID,
long length, long offset) {
this.containerID = containerID;
this.localID = localID;
this.length = length;
this.offset = offset;
}
/**
* Returns the containerID of this Key.
*/
public long getContainerID() {
return containerID;
}
/**
* Returns the localID of this Key.
*/
public long getLocalID() {
return localID;
}
/**
* Returns the length of this Key.
*/
public long getLength() {
return length;
}
/**
* Returns the offset of this Key.
*/
public long getOffset() {
return offset;
}
}

View File

@ -0,0 +1,80 @@
/*
* 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.web.response;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import java.util.List;
/**
* Represents an Ozone key Object with detail information of location.
*/
public class KeyInfoDetails extends KeyInfo {
/**
* a list of Map which maps localID to ContainerID
* to specify replica locations.
*/
private List<KeyLocation> keyLocations;
/**
* Set details of key location.
*
* @param keyLocations - details of key location
*/
public void setKeyLocations(List<KeyLocation> keyLocations) {
this.keyLocations = keyLocations;
}
/**
* Returns details of key location.
*
* @return volumeName
*/
public List<KeyLocation> getKeyLocations() {
return keyLocations;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
KeyInfoDetails that = (KeyInfoDetails) o;
return new EqualsBuilder()
.append(getVersion(), that.getVersion())
.append(getKeyName(), that.getKeyName())
.append(keyLocations, that.getKeyLocations())
.isEquals();
}
@Override
public int hashCode() {
return new HashCodeBuilder(17, 37)
.append(getVersion())
.append(getKeyName())
.append(keyLocations)
.toHashCode();
}
}

View File

@ -0,0 +1,82 @@
/**
* 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.web.response;
/**
* KeyLocation class is used used for parsing json response
* when KeyInfoDetails Call is made.
*/
public class KeyLocation {
/**
* Which container this key stored.
*/
private final long containerID;
/**
* Which block this key stored inside a container.
*/
private final long localID;
/**
* Data length of this key replica.
*/
private final long length;
/**
* Offset of this key.
*/
private final long offset;
/**
* Constructs KeyLocation.
*/
public KeyLocation(long containerID, long localID,
long length, long offset) {
this.containerID = containerID;
this.localID = localID;
this.length = length;
this.offset = offset;
}
/**
* Returns the containerID of this Key.
*/
public long getContainerID() {
return containerID;
}
/**
* Returns the localID of this Key.
*/
public long getLocalID() {
return localID;
}
/**
* Returns the length of this Key.
*/
public long getLength() {
return length;
}
/**
* Returns the offset of this Key.
*/
public long getOffset() {
return offset;
}
}

View File

@ -18,23 +18,22 @@
package org.apache.hadoop.ozone.client.rest;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.client.BucketArgs;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.ozone.*;
import org.apache.hadoop.ozone.client.*;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@ -42,6 +41,7 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
@ -414,6 +414,72 @@ public class TestOzoneRestClient {
bucket.getKey(fromKeyName);
}
@Test
public void testGetKeyDetails() throws IOException, OzoneException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
store.createVolume(volumeName);
OzoneVolume volume = store.getVolume(volumeName);
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
String keyName = UUID.randomUUID().toString();
String keyValue = RandomStringUtils.random(128);
OzoneOutputStream out = bucket.createKey(keyName,
keyValue.getBytes().length, ReplicationType.STAND_ALONE,
ReplicationFactor.ONE);
out.write(keyValue.getBytes());
out.close();
// Get the containerID and localID.
OzoneKeyDetails keyDetails = (OzoneKeyDetails)bucket.getKey(keyName);
Assert.assertEquals(keyName, keyDetails.getName());
List<OzoneKeyLocation> keyLocations = keyDetails.getOzoneKeyLocations();
Assert.assertEquals(1, keyLocations.size());
Long containerID = keyLocations.get(0).getContainerID();
Long localID = keyLocations.get(0).getLocalID();
// Make sure that the data size matched.
Assert.assertEquals(keyValue.getBytes().length,
keyLocations.get(0).getLength());
// Sum the data size from chunks in Container via containerID
// and localID, make sure the size equals to the actually value size.
Pipeline pipeline = cluster.getStorageContainerManager()
.getScmContainerManager().getContainerWithPipeline(containerID)
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
Assert.assertEquals(datanodes.size(), 1);
DatanodeDetails datanodeDetails = datanodes.get(0);
Assert.assertNotNull(datanodeDetails);
HddsDatanodeService datanodeService = null;
for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) {
if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) {
datanodeService = datanodeServiceItr;
break;
}
}
KeyValueContainerData containerData =
(KeyValueContainerData)(datanodeService.getDatanodeStateMachine()
.getContainer().getContainerSet().getContainer(containerID)
.getContainerData());
String containerPath = new File(containerData.getMetadataPath())
.getParent();
KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
containerID, new File(containerPath));
long valueLength = 0;
while (keyValueBlockIterator.hasNext()) {
KeyData keyData = keyValueBlockIterator.nextBlock();
if (keyData.getBlockID().getLocalID() == localID) {
List<ContainerProtos.ChunkInfo> chunks = keyData.getChunks();
for (ContainerProtos.ChunkInfo chunk : chunks) {
valueLength += chunk.getLen();
}
}
}
Assert.assertEquals(keyValue.getBytes().length, valueLength);
}
/**
* Close OzoneClient and shutdown MiniDFSCluster.
*/

View File

@ -21,23 +21,21 @@ package org.apache.hadoop.ozone.client.rpc;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.ozone.*;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.client.BucketArgs;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.*;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.om.OzoneManager;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
@ -56,6 +54,7 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
@ -535,6 +534,88 @@ public class TestOzoneRpcClient {
}
}
@Test
public void testGetKeyDetails() throws IOException, OzoneException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
store.createVolume(volumeName);
OzoneVolume volume = store.getVolume(volumeName);
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
String keyName = UUID.randomUUID().toString();
String keyValue = RandomStringUtils.random(128);
//String keyValue = "this is a test value.glx";
// create the initial key with size 0, write will allocate the first block.
OzoneOutputStream out = bucket.createKey(keyName,
keyValue.getBytes().length, ReplicationType.STAND_ALONE,
ReplicationFactor.ONE);
out.write(keyValue.getBytes());
out.close();
OzoneInputStream is = bucket.readKey(keyName);
byte[] fileContent = new byte[32];
is.read(fileContent);
// First, confirm the key info from the client matches the info in OM.
OmKeyArgs.Builder builder = new OmKeyArgs.Builder();
builder.setVolumeName(volumeName).setBucketName(bucketName)
.setKeyName(keyName);
OmKeyLocationInfo keyInfo = ozoneManager.lookupKey(builder.build()).
getKeyLocationVersions().get(0).getBlocksLatestVersionOnly().get(0);
long containerID = keyInfo.getContainerID();
long localID = keyInfo.getLocalID();
OzoneKeyDetails keyDetails = (OzoneKeyDetails)bucket.getKey(keyName);
Assert.assertEquals(keyName, keyDetails.getName());
List<OzoneKeyLocation> keyLocations = keyDetails.getOzoneKeyLocations();
Assert.assertEquals(1, keyLocations.size());
Assert.assertEquals(containerID, keyLocations.get(0).getContainerID());
Assert.assertEquals(localID, keyLocations.get(0).getLocalID());
// Make sure that the data size matched.
Assert.assertEquals(keyValue.getBytes().length,
keyLocations.get(0).getLength());
// Second, sum the data size from chunks in Container via containerID
// and localID, make sure the size equals to the size from keyDetails.
Pipeline pipeline = cluster.getStorageContainerManager()
.getScmContainerManager().getContainerWithPipeline(containerID)
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
Assert.assertEquals(datanodes.size(), 1);
DatanodeDetails datanodeDetails = datanodes.get(0);
Assert.assertNotNull(datanodeDetails);
HddsDatanodeService datanodeService = null;
for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) {
if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) {
datanodeService = datanodeServiceItr;
break;
}
}
KeyValueContainerData containerData =
(KeyValueContainerData)(datanodeService.getDatanodeStateMachine()
.getContainer().getContainerSet().getContainer(containerID)
.getContainerData());
String containerPath = new File(containerData.getMetadataPath())
.getParent();
KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
containerID, new File(containerPath));
while (keyValueBlockIterator.hasNext()) {
KeyData keyData = keyValueBlockIterator.nextBlock();
if (keyData.getBlockID().getLocalID() == localID) {
long length = 0;
List<ContainerProtos.ChunkInfo> chunks = keyData.getChunks();
for (ContainerProtos.ChunkInfo chunk : chunks) {
length += chunk.getLen();
}
Assert.assertEquals(length, keyValue.getBytes().length);
break;
}
}
}
@Test
public void testDeleteKey()
throws IOException, OzoneException {

View File

@ -773,7 +773,7 @@ public class TestOzoneShell {
}
@Test
public void testInfoKey() throws Exception {
public void testInfoKeyDetails() throws Exception {
LOG.info("Running testInfoKey");
String keyName = "key" + RandomStringUtils.randomNumeric(5);
OzoneBucket bucket = creatBucket();
@ -797,7 +797,9 @@ public class TestOzoneShell {
assertTrue(
output.contains("createdOn") && output.contains("modifiedOn") && output
.contains(OzoneConsts.OZONE_TIME_ZONE));
assertTrue(
output.contains("containerID") && output.contains("localID") && output
.contains("length") && output.contains("offset"));
// reset stream
out.reset();
err.reset();

View File

@ -88,6 +88,8 @@ public class KeyHandler implements Keys {
return getKey(args);
} else if (info.equals(Header.OZONE_INFO_QUERY_KEY)) {
return getKeyInfo(args);
} else if (info.equals(Header.OZONE_INFO_QUERY_KEY_DETAIL)) {
return getKeyInfoDetail(args);
}
OzoneException ozException = ErrorTable
@ -118,6 +120,16 @@ public class KeyHandler implements Keys {
return OzoneRestUtils.getResponse(args, HTTP_OK, keyInfo.toJsonString());
}
/**
* Gets the Key detail information if it exists.
*/
private Response getKeyInfoDetail(KeyArgs args)
throws IOException, OzoneException {
StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
KeyInfo keyInfo = fs.getKeyInfoDetails(args);
return OzoneRestUtils.getResponse(args, HTTP_OK, keyInfo.toJsonString());
}
/**
* Adds a key to an existing bucket. If the object already exists this call
* will overwrite or add with new version number if the bucket versioning is

View File

@ -297,6 +297,18 @@ public interface StorageHandler extends Closeable{
*/
KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException;
/**
* Get detail information of the specified Key.
*
* @param args Key Args
*
* @return KeyInfo
*
* @throws IOException
* @throws OzoneException
*/
KeyInfo getKeyInfoDetails(KeyArgs args) throws IOException, OzoneException;
/**
* Closes all the opened resources.
*/

View File

@ -54,18 +54,13 @@ import org.apache.hadoop.ozone.web.handlers.ListArgs;
import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
import org.apache.hadoop.ozone.web.handlers.UserArgs;
import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
import org.apache.hadoop.ozone.web.response.ListVolumes;
import org.apache.hadoop.ozone.web.response.VolumeInfo;
import org.apache.hadoop.ozone.web.response.VolumeOwner;
import org.apache.hadoop.ozone.web.response.ListBuckets;
import org.apache.hadoop.ozone.web.response.BucketInfo;
import org.apache.hadoop.ozone.web.response.KeyInfo;
import org.apache.hadoop.ozone.web.response.ListKeys;
import org.apache.hadoop.ozone.web.response.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.List;
/**
@ -494,6 +489,30 @@ public final class DistributedStorageHandler implements StorageHandler {
return keyInfo;
}
@Override
public KeyInfo getKeyInfoDetails(KeyArgs args) throws IOException{
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(args.getVolumeName())
.setBucketName(args.getBucketName())
.setKeyName(args.getKeyName())
.build();
OmKeyInfo omKeyInfo = ozoneManagerClient.lookupKey(keyArgs);
List<KeyLocation> keyLocations = new ArrayList<>();
omKeyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly()
.forEach((a) -> keyLocations.add(new KeyLocation(a.getContainerID(),
a.getLocalID(), a.getLength(), a.getOffset())));
KeyInfoDetails keyInfoDetails = new KeyInfoDetails();
keyInfoDetails.setVersion(0);
keyInfoDetails.setKeyName(omKeyInfo.getKeyName());
keyInfoDetails.setSize(omKeyInfo.getDataSize());
keyInfoDetails.setCreatedOn(
HddsClientUtils.formatDateTime(omKeyInfo.getCreationTime()));
keyInfoDetails.setModifiedOn(
HddsClientUtils.formatDateTime(omKeyInfo.getModificationTime()));
keyInfoDetails.setKeyLocations(keyLocations);
return keyInfoDetails;
}
@Override
public ListKeys listKeys(ListArgs args) throws IOException, OzoneException {
ListKeys result = new ListKeys();

View File

@ -25,11 +25,7 @@ import java.nio.file.Path;
import java.nio.file.Paths;
import org.apache.commons.cli.CommandLine;
import org.apache.hadoop.ozone.client.OzoneClientException;
import org.apache.hadoop.ozone.client.OzoneClientUtils;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.*;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.web.ozShell.Handler;
import org.apache.hadoop.ozone.web.ozShell.Shell;
@ -79,9 +75,9 @@ public class InfoKeyHandler extends Handler {
OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
OzoneBucket bucket = vol.getBucket(bucketName);
OzoneKey key = bucket.getKey(keyName);
OzoneKeyDetails key = bucket.getKey(keyName);
System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
JsonUtils.toJsonString(OzoneClientUtils.asKeyInfo(key))));
JsonUtils.toJsonString(OzoneClientUtils.asKeyInfoDetails(key))));
}
}