Browse Source

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

Weiwei Yang 8 years ago
parent
commit
5379ffa8de

+ 62 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java

@@ -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);
+      }
+    }
+  }
 }

+ 42 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java

@@ -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());
   }
 
   /**

+ 7 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java

@@ -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;
 

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java

@@ -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.
    */

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java

@@ -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.

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java

@@ -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.
    */

+ 15 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java

@@ -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()));
   }
 }

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java

@@ -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();

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java

@@ -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());
+  }
 }

+ 165 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java

@@ -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()));
+  }
+}

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java

@@ -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());
+  }
 }