You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by el...@apache.org on 2018/09/03 11:39:25 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/trunk 3801436e4 -> 211034a6c


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


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/211034a6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/211034a6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/211034a6

Branch: refs/heads/trunk
Commit: 211034a6c22dd4ebe697481ea4d57b5eb932fa08
Parents: 3801436
Author: Márton Elek <el...@apache.org>
Authored: Mon Sep 3 13:32:55 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Mon Sep 3 13:32:55 2018 +0200

----------------------------------------------------------------------
 .../apache/hadoop/ozone/client/OzoneBucket.java |   4 +-
 .../hadoop/ozone/client/OzoneClientUtils.java   |  29 ++++-
 .../hadoop/ozone/client/OzoneKeyDetails.java    |  58 ++++++++++
 .../hadoop/ozone/client/OzoneKeyLocation.java   |  82 ++++++++++++++
 .../ozone/client/protocol/ClientProtocol.java   |  10 +-
 .../hadoop/ozone/client/rest/RestClient.java    |  27 ++---
 .../hadoop/ozone/client/rpc/RpcClient.java      |  22 ++--
 .../ozone/client/rest/headers/Header.java       |   1 +
 .../client/rest/response/KeyInfoDetails.java    | 107 +++++++++++++++++++
 .../ozone/client/rest/response/KeyLocation.java |  89 +++++++++++++++
 .../ozone/web/response/KeyInfoDetails.java      |  80 ++++++++++++++
 .../hadoop/ozone/web/response/KeyLocation.java  |  82 ++++++++++++++
 .../ozone/client/rest/TestOzoneRestClient.java  |  86 +++++++++++++--
 .../ozone/client/rpc/TestOzoneRpcClient.java    | 101 +++++++++++++++--
 .../hadoop/ozone/ozShell/TestOzoneShell.java    |   6 +-
 .../hadoop/ozone/web/handlers/KeyHandler.java   |  12 +++
 .../ozone/web/interfaces/StorageHandler.java    |  12 +++
 .../web/storage/DistributedStorageHandler.java  |  33 ++++--
 .../ozone/web/ozShell/keys/InfoKeyHandler.java  |  10 +-
 19 files changed, 779 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index 2f3cff6..97bd682 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -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);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
index 5d57753..40e4d83 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
@@ -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;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java
new file mode 100644
index 0000000..e7709dd
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java
@@ -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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyLocation.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyLocation.java
new file mode 100644
index 0000000..0ff8ba7
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyLocation.java
@@ -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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
index 94cc257..008b69d 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -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,8 +317,8 @@ public interface ClientProtocol {
    * @return {@link OzoneKey}
    * @throws IOException
    */
-  OzoneKey getKeyDetails(String volumeName, String bucketName,
-                         String keyName)
+  OzoneKeyDetails getKeyDetails(String volumeName, String bucketName,
+                                String keyName)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
index 78fbe8d..fdd049a 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
@@ -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) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index 387f41f..e9a684e 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
index ebfc0a9..3e40493 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
@@ -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";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyInfoDetails.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyInfoDetails.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyInfoDetails.java
new file mode 100644
index 0000000..98506f0
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyInfoDetails.java
@@ -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();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyLocation.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyLocation.java
new file mode 100644
index 0000000..e5f4698
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyLocation.java
@@ -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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyInfoDetails.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyInfoDetails.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyInfoDetails.java
new file mode 100644
index 0000000..7f2ba09
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyInfoDetails.java
@@ -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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyLocation.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyLocation.java
new file mode 100644
index 0000000..d03eff7
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyLocation.java
@@ -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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
index 73bb32d..ddff0c5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
@@ -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.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
index f8ad32e..bf1eba6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
@@ -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;
@@ -536,6 +535,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 {
     thrown.expectMessage("Lookup key failed, error");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
index 528828b..b4740b4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
@@ -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();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
index 8c0b103..97e2a1a 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
@@ -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
@@ -119,6 +121,16 @@ public class KeyHandler implements Keys {
   }
 
   /**
+   * 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
    * turned on.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
index 338ff63..9c115a8 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
@@ -298,6 +298,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.
    */
   void close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
index 0d62432..88f2d6e 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
@@ -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;
 
 /**
@@ -495,6 +490,30 @@ public final class DistributedStorageHandler implements StorageHandler {
   }
 
   @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();
     UserArgs userArgs = args.getArgs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
index 3fcdda9..fe87756 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
@@ -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))));
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org