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 ha...@apache.org on 2018/09/24 19:07:38 UTC

[16/50] [abbrv] hadoop git commit: HDDS-394. Rename *Key Apis in DatanodeContainerProtocol to *Block apis. Contributed Dinesh Chitlangia.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestBlockData.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestBlockData.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestBlockData.java
new file mode 100644
index 0000000..9df4249
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestBlockData.java
@@ -0,0 +1,127 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.container.common.helpers;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Tests to test block deleting service.
+ */
+public class TestBlockData {
+  static final Logger LOG = LoggerFactory.getLogger(TestBlockData.class);
+  @Rule
+  public TestRule timeout = new Timeout(10000);
+
+  static ContainerProtos.ChunkInfo buildChunkInfo(String name, long offset,
+      long len) {
+    return ContainerProtos.ChunkInfo.newBuilder()
+        .setChunkName(name).setOffset(offset).setLen(len).build();
+  }
+
+  @Test
+  public void testAddAndRemove() {
+    final BlockData computed = new BlockData(null);
+    final List<ContainerProtos.ChunkInfo> expected = new ArrayList<>();
+
+    assertChunks(expected, computed);
+    long offset = 0;
+    int n = 5;
+    for(int i = 0; i < n; i++) {
+      offset += assertAddChunk(expected, computed, offset);
+    }
+
+    for(; !expected.isEmpty();) {
+      removeChunk(expected, computed);
+    }
+  }
+
+  private static int chunkCount = 0;
+  static ContainerProtos.ChunkInfo addChunk(
+      List<ContainerProtos.ChunkInfo> expected, long offset) {
+    final long length = ThreadLocalRandom.current().nextLong(1000);
+    final ContainerProtos.ChunkInfo info =
+        buildChunkInfo("c" + ++chunkCount, offset, length);
+    expected.add(info);
+    return info;
+  }
+
+  static long assertAddChunk(List<ContainerProtos.ChunkInfo> expected,
+      BlockData computed, long offset) {
+    final ContainerProtos.ChunkInfo info = addChunk(expected, offset);
+    LOG.info("addChunk: " + toString(info));
+    computed.addChunk(info);
+    assertChunks(expected, computed);
+    return info.getLen();
+  }
+
+
+  static void removeChunk(List<ContainerProtos.ChunkInfo> expected,
+      BlockData computed) {
+    final int i = ThreadLocalRandom.current().nextInt(expected.size());
+    final ContainerProtos.ChunkInfo info = expected.remove(i);
+    LOG.info("removeChunk: " + toString(info));
+    computed.removeChunk(info);
+    assertChunks(expected, computed);
+  }
+
+  static void assertChunks(List<ContainerProtos.ChunkInfo> expected,
+      BlockData computed) {
+    final List<ContainerProtos.ChunkInfo> computedChunks = computed.getChunks();
+    Assert.assertEquals("expected=" + expected + "\ncomputed=" +
+        computedChunks, expected, computedChunks);
+    Assert.assertEquals(expected.stream().mapToLong(i -> i.getLen()).sum(),
+        computed.getSize());
+  }
+
+  static String toString(ContainerProtos.ChunkInfo info) {
+    return info.getChunkName() + ":" + info.getOffset() + "," + info.getLen();
+  }
+
+  static String toString(List<ContainerProtos.ChunkInfo> infos) {
+    return infos.stream().map(TestBlockData::toString)
+        .reduce((left, right) -> left + ", " + right)
+        .orElse("");
+  }
+
+  @Test
+  public void testSetChunks() {
+    final BlockData computed = new BlockData(null);
+    final List<ContainerProtos.ChunkInfo> expected = new ArrayList<>();
+
+    assertChunks(expected, computed);
+    long offset = 0;
+    int n = 5;
+    for(int i = 0; i < n; i++) {
+      offset += addChunk(expected, offset).getLen();
+      LOG.info("setChunk: " + toString(expected));
+      computed.setChunks(expected);
+      assertChunks(expected, computed);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestKeyData.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestKeyData.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestKeyData.java
deleted file mode 100644
index f57fe99..0000000
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestKeyData.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.container.common.helpers;
-
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * Tests to test block deleting service.
- */
-public class TestKeyData {
-  static final Logger LOG = LoggerFactory.getLogger(TestKeyData.class);
-  @Rule
-  public TestRule timeout = new Timeout(10000);
-
-  static ContainerProtos.ChunkInfo buildChunkInfo(String name, long offset, long len) {
-    return ContainerProtos.ChunkInfo.newBuilder()
-        .setChunkName(name).setOffset(offset).setLen(len).build();
-  }
-
-  @Test
-  public void testAddAndRemove() {
-    final KeyData computed = new KeyData(null);
-    final List<ContainerProtos.ChunkInfo> expected = new ArrayList<>();
-
-    assertChunks(expected, computed);
-    long offset = 0;
-    int n = 5;
-    for(int i = 0; i < n; i++) {
-      offset += assertAddChunk(expected, computed, offset);
-    }
-
-    for(; !expected.isEmpty(); ) {
-      removeChunk(expected, computed);
-    }
-  }
-
-  private static int chunkCount = 0;
-  static ContainerProtos.ChunkInfo addChunk(List<ContainerProtos.ChunkInfo> expected, long offset) {
-    final long length = ThreadLocalRandom.current().nextLong(1000);
-    final ContainerProtos.ChunkInfo info = buildChunkInfo("c" + ++chunkCount, offset, length);
-    expected.add(info);
-    return info;
-  }
-
-  static long assertAddChunk(List<ContainerProtos.ChunkInfo> expected, KeyData computed, long offset) {
-    final ContainerProtos.ChunkInfo info = addChunk(expected, offset);
-    LOG.info("addChunk: " + toString(info));
-    computed.addChunk(info);
-    assertChunks(expected, computed);
-    return info.getLen();
-  }
-
-
-  static void removeChunk(List<ContainerProtos.ChunkInfo> expected, KeyData computed) {
-    final int i = ThreadLocalRandom.current().nextInt(expected.size());
-    final ContainerProtos.ChunkInfo info = expected.remove(i);
-    LOG.info("removeChunk: " + toString(info));
-    computed.removeChunk(info);
-    assertChunks(expected, computed);
-  }
-
-  static void assertChunks(List<ContainerProtos.ChunkInfo> expected, KeyData computed) {
-    final List<ContainerProtos.ChunkInfo> computedChunks = computed.getChunks();
-    Assert.assertEquals("expected=" + expected + "\ncomputed=" + computedChunks, expected, computedChunks);
-    Assert.assertEquals(expected.stream().mapToLong(i -> i.getLen()).sum(), computed.getSize());
-  }
-
-  static String toString(ContainerProtos.ChunkInfo info) {
-    return info.getChunkName() + ":" + info.getOffset() + "," + info.getLen();
-  }
-
-  static String toString(List<ContainerProtos.ChunkInfo> infos) {
-    return infos.stream().map(TestKeyData::toString)
-        .reduce((left, right) -> left + ", " + right)
-        .orElse("");
-  }
-
-  @Test
-  public void testSetChunks() {
-    final KeyData computed = new KeyData(null);
-    final List<ContainerProtos.ChunkInfo> expected = new ArrayList<>();
-
-    assertChunks(expected, computed);
-    long offset = 0;
-    int n = 5;
-    for(int i = 0; i < n; i++) {
-      offset += addChunk(expected, offset).getLen();
-      LOG.info("setChunk: " + toString(expected));
-      computed.setChunks(expected);
-      assertChunks(expected, computed);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
index 73fa70d..78bf008 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
@@ -25,12 +25,12 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.junit.AfterClass;
@@ -71,14 +71,14 @@ public class TestCloseContainerHandler {
 
   private final static String DATANODE_UUID = UUID.randomUUID().toString();
 
-  private static final String baseDir = MiniDFSCluster.getBaseDirectory();
-  private static final String volume1 = baseDir + "disk1";
-  private static final String volume2 = baseDir + "disk2";
+  private static final String BASE_DIR = MiniDFSCluster.getBaseDirectory();
+  private static final String VOLUME_1 = BASE_DIR + "disk1";
+  private static final String VOLUME_2 = BASE_DIR + "disk2";
 
   @BeforeClass
   public static void setup() throws Exception {
     conf = new Configuration();
-    String dataDirKey = volume1 + "," + volume2;
+    String dataDirKey = VOLUME_1 + "," + VOLUME_2;
     conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirKey);
     containerSet = new ContainerSet();
     DatanodeDetails datanodeDetails =
@@ -160,31 +160,31 @@ public class TestCloseContainerHandler {
         getTestBlockID(testContainerID);
     Pipeline pipeline = createSingleNodePipeline();
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
-    // the key should exist in the map
+    // the block should exist in the map
     Assert.assertNotNull(
-        openContainerBlockMap.getKeyDataMap(testContainerID)
+        openContainerBlockMap.getBlockDataMap(testContainerID)
             .get(blockID.getLocalID()));
-    KeyData keyData = new KeyData(blockID);
+    BlockData blockData = new BlockData(blockID);
     List<ContainerProtos.ChunkInfo> chunkProtoList = new LinkedList<>();
     for (ChunkInfo i : chunkList) {
       chunkProtoList.add(i.getProtoBufMessage());
     }
-    keyData.setChunks(chunkProtoList);
-    ContainerProtos.PutKeyRequestProto.Builder putKeyRequestProto =
-        ContainerProtos.PutKeyRequestProto.newBuilder();
-    putKeyRequestProto.setKeyData(keyData.getProtoBufMessage());
+    blockData.setChunks(chunkProtoList);
+    ContainerProtos.PutBlockRequestProto.Builder putBlockRequestProto =
+        ContainerProtos.PutBlockRequestProto.newBuilder();
+    putBlockRequestProto.setBlockData(blockData.getProtoBufMessage());
     ContainerProtos.ContainerCommandRequestProto.Builder request =
         ContainerProtos.ContainerCommandRequestProto.newBuilder();
-    request.setCmdType(ContainerProtos.Type.PutKey);
+    request.setCmdType(ContainerProtos.Type.PutBlock);
     request.setContainerID(blockID.getContainerID());
-    request.setPutKey(putKeyRequestProto);
+    request.setPutBlock(putBlockRequestProto);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
     dispatcher.dispatch(request.build());
 
-    //the open key should be removed from Map
+    //the open block should be removed from Map
     Assert.assertNull(
-        openContainerBlockMap.getKeyDataMap(testContainerID));
+        openContainerBlockMap.getBlockDataMap(testContainerID));
   }
 
   @Test
@@ -197,10 +197,10 @@ public class TestCloseContainerHandler {
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
     // the key should exist in the map
     Assert.assertNotNull(
-        openContainerBlockMap.getKeyDataMap(testContainerID)
+        openContainerBlockMap.getBlockDataMap(testContainerID)
             .get(blockID.getLocalID()));
     Assert.assertTrue(
-        openContainerBlockMap.getKeyDataMap(testContainerID)
+        openContainerBlockMap.getBlockDataMap(testContainerID)
             .get(blockID.getLocalID()).getChunks().size() == 3);
     ContainerProtos.DeleteChunkRequestProto.Builder deleteChunkProto =
         ContainerProtos.DeleteChunkRequestProto.newBuilder();
@@ -220,7 +220,7 @@ public class TestCloseContainerHandler {
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
     dispatcher.dispatch(request.build());
     Assert.assertTrue(
-        openContainerBlockMap.getKeyDataMap(testContainerID)
+        openContainerBlockMap.getBlockDataMap(testContainerID)
             .get(blockID.getLocalID()).getChunks().size() == 2);
 
   }
@@ -235,14 +235,14 @@ public class TestCloseContainerHandler {
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
 
     Container container = containerSet.getContainer(testContainerID);
-    KeyData keyData = openContainerBlockMap.
-        getKeyDataMap(testContainerID).get(blockID.getLocalID());
+    BlockData blockData = openContainerBlockMap.
+        getBlockDataMap(testContainerID).get(blockID.getLocalID());
     // the key should exist in the map
     Assert.assertNotNull(
-        openContainerBlockMap.getKeyDataMap(testContainerID)
+        openContainerBlockMap.getBlockDataMap(testContainerID)
             .get(blockID.getLocalID()));
     Assert.assertTrue(
-        keyData.getChunks().size() == chunkList.size());
+        blockData.getChunks().size() == chunkList.size());
     ContainerProtos.ContainerCommandRequestProto.Builder request =
         ContainerProtos.ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.CloseContainer);
@@ -253,8 +253,9 @@ public class TestCloseContainerHandler {
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
     dispatcher.dispatch(request.build());
     Assert.assertNull(
-        openContainerBlockMap.getKeyDataMap(testContainerID));
+        openContainerBlockMap.getBlockDataMap(testContainerID));
     // Make sure the key got committed
-    Assert.assertNotNull(handler.getKeyManager().getKey(container, blockID));
+    Assert.assertNotNull(handler.getBlockManager()
+        .getBlock(container, blockID));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index 3a36331..b6cdc9d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -29,20 +29,20 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerExcep
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
-import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
+import org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
-import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.utils.MetadataStore;
 import org.junit.After;
@@ -97,7 +97,7 @@ public class TestContainerPersistence {
   private static ContainerSet containerSet;
   private static VolumeSet volumeSet;
   private static VolumeChoosingPolicy volumeChoosingPolicy;
-  private static KeyManager keyManager;
+  private static BlockManager blockManager;
   private static ChunkManager chunkManager;
   @Rule
   public ExpectedException exception = ExpectedException.none();
@@ -126,7 +126,7 @@ public class TestContainerPersistence {
   public void setupPaths() throws IOException {
     containerSet = new ContainerSet();
     volumeSet = new VolumeSet(DATANODE_UUID, conf);
-    keyManager = new KeyManagerImpl(conf);
+    blockManager = new BlockManagerImpl(conf);
     chunkManager = new ChunkManagerImpl();
 
     for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
@@ -152,15 +152,15 @@ public class TestContainerPersistence {
     return ContainerTestHelper.getTestContainerID();
   }
 
-  private Container addContainer(ContainerSet containerSet, long containerID)
+  private Container addContainer(ContainerSet cSet, long cID)
       throws IOException {
-    KeyValueContainerData data = new KeyValueContainerData(containerID,
+    KeyValueContainerData data = new KeyValueContainerData(cID,
         ContainerTestHelper.CONTAINER_MAX_SIZE);
     data.addMetadata("VOLUME", "shire");
     data.addMetadata("owner)", "bilbo");
     KeyValueContainer container = new KeyValueContainer(data, conf);
     container.create(volumeSet, volumeChoosingPolicy, SCM_ID);
-    containerSet.addContainer(container);
+    cSet.addContainer(container);
     return container;
   }
 
@@ -184,7 +184,7 @@ public class TestContainerPersistence {
 
     MetadataStore store = null;
     try {
-      store = KeyUtils.getDB(kvData, conf);
+      store = BlockUtils.getDB(kvData, conf);
       Assert.assertNotNull(store);
     } finally {
       if (store != null) {
@@ -227,19 +227,19 @@ public class TestContainerPersistence {
     Assert.assertFalse(containerSet.getContainerMap()
         .containsKey(testContainerID1));
 
-    // Adding key to a deleted container should fail.
+    // Adding block to a deleted container should fail.
     exception.expect(StorageContainerException.class);
     exception.expectMessage("Error opening DB.");
     BlockID blockID1 = ContainerTestHelper.getTestBlockID(testContainerID1);
-    KeyData someKey1 = new KeyData(blockID1);
+    BlockData someKey1 = new BlockData(blockID1);
     someKey1.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
-    keyManager.putKey(container1, someKey1);
+    blockManager.putBlock(container1, someKey1);
 
     // Deleting a non-empty container should fail.
     BlockID blockID2 = ContainerTestHelper.getTestBlockID(testContainerID2);
-    KeyData someKey2 = new KeyData(blockID2);
+    BlockData someKey2 = new BlockData(blockID2);
     someKey2.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
-    keyManager.putKey(container2, someKey2);
+    blockManager.putBlock(container2, someKey2);
 
     exception.expect(StorageContainerException.class);
     exception.expectMessage(
@@ -325,7 +325,8 @@ public class TestContainerPersistence {
     if (container == null) {
       container = addContainer(containerSet, testContainerID);
     }
-    ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
+    ChunkInfo info = getChunk(
+        blockID.getLocalID(), 0, 0, datalen);
     byte[] data = getData(datalen);
     setDataChecksum(info, data);
     chunkManager.writeChunk(container, blockID, info, data, COMBINED);
@@ -348,8 +349,8 @@ public class TestContainerPersistence {
   }
 
   /**
-   * Writes many chunks of the same key into different chunk files and verifies
-   * that we have that data in many files.
+   * Writes many chunks of the same block into different chunk files and
+   * verifies that we have that data in many files.
    *
    * @throws IOException
    * @throws NoSuchAlgorithmException
@@ -425,7 +426,8 @@ public class TestContainerPersistence {
     Container container = addContainer(containerSet, testContainerID);
 
     BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
-    ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
+    ChunkInfo info = getChunk(
+        blockID.getLocalID(), 0, 0, datalen);
     byte[] data = getData(datalen);
     setDataChecksum(info, data);
     chunkManager.writeChunk(container, blockID, info, data, COMBINED);
@@ -456,7 +458,8 @@ public class TestContainerPersistence {
     Container container = addContainer(containerSet, testContainerID);
 
     BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
-    ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
+    ChunkInfo info = getChunk(
+        blockID.getLocalID(), 0, 0, datalen);
     byte[] data = getData(datalen);
     setDataChecksum(info, data);
     chunkManager.writeChunk(container, blockID, info, data, COMBINED);
@@ -500,7 +503,8 @@ public class TestContainerPersistence {
     for (int x = 0; x < chunkCount; x++) {
       // we are writing to the same chunk file but at different offsets.
       long offset = x * datalen;
-      ChunkInfo info = getChunk(blockID.getLocalID(), 0, offset, datalen);
+      ChunkInfo info = getChunk(
+          blockID.getLocalID(), 0, offset, datalen);
       byte[] data = getData(datalen);
       oldSha.update(data);
       setDataChecksum(info, data);
@@ -531,7 +535,8 @@ public class TestContainerPersistence {
     Container container = addContainer(containerSet, testContainerID);
 
     BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
-    ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
+    ChunkInfo info = getChunk(
+        blockID.getLocalID(), 0, 0, datalen);
     byte[] data = getData(datalen);
     setDataChecksum(info, data);
     chunkManager.writeChunk(container, blockID, info, data, COMBINED);
@@ -542,37 +547,38 @@ public class TestContainerPersistence {
   }
 
   /**
-   * Tests a put key and read key.
+   * Tests a put block and read block.
    *
    * @throws IOException
    * @throws NoSuchAlgorithmException
    */
   @Test
-  public void testPutKey() throws IOException, NoSuchAlgorithmException {
+  public void testPutBlock() throws IOException, NoSuchAlgorithmException {
     long testContainerID = getTestContainerID();
     Container container = addContainer(containerSet, testContainerID);
 
     BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
     ChunkInfo info = writeChunkHelper(blockID);
-    KeyData keyData = new KeyData(blockID);
+    BlockData blockData = new BlockData(blockID);
     List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkList.add(info.getProtoBufMessage());
-    keyData.setChunks(chunkList);
-    keyManager.putKey(container, keyData);
-    KeyData readKeyData = keyManager.getKey(container, keyData.getBlockID());
+    blockData.setChunks(chunkList);
+    blockManager.putBlock(container, blockData);
+    BlockData readBlockData = blockManager.
+        getBlock(container, blockData.getBlockID());
     ChunkInfo readChunk =
-        ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(0));
+        ChunkInfo.getFromProtoBuf(readBlockData.getChunks().get(0));
     Assert.assertEquals(info.getChecksum(), readChunk.getChecksum());
   }
 
   /**
-   * Tests a put key and read key.
+   * Tests a put block and read block.
    *
    * @throws IOException
    * @throws NoSuchAlgorithmException
    */
   @Test
-  public void testPutKeyWithLotsOfChunks() throws IOException,
+  public void testPutBlockWithLotsOfChunks() throws IOException,
       NoSuchAlgorithmException {
     final int chunkCount = 2;
     final int datalen = 1024;
@@ -603,66 +609,67 @@ public class TestContainerPersistence {
     long writeCount = container.getContainerData().getWriteCount();
     Assert.assertEquals(chunkCount, writeCount);
 
-    KeyData keyData = new KeyData(blockID);
+    BlockData blockData = new BlockData(blockID);
     List<ContainerProtos.ChunkInfo> chunkProtoList = new LinkedList<>();
     for (ChunkInfo i : chunkList) {
       chunkProtoList.add(i.getProtoBufMessage());
     }
-    keyData.setChunks(chunkProtoList);
-    keyManager.putKey(container, keyData);
-    KeyData readKeyData = keyManager.getKey(container, keyData.getBlockID());
+    blockData.setChunks(chunkProtoList);
+    blockManager.putBlock(container, blockData);
+    BlockData readBlockData = blockManager.
+        getBlock(container, blockData.getBlockID());
     ChunkInfo lastChunk = chunkList.get(chunkList.size() - 1);
     ChunkInfo readChunk =
-        ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(readKeyData
+        ChunkInfo.getFromProtoBuf(readBlockData.getChunks().get(readBlockData
             .getChunks().size() - 1));
     Assert.assertEquals(lastChunk.getChecksum(), readChunk.getChecksum());
   }
 
   /**
-   * Deletes a key and tries to read it back.
+   * Deletes a block and tries to read it back.
    *
    * @throws IOException
    * @throws NoSuchAlgorithmException
    */
   @Test
-  public void testDeleteKey() throws IOException, NoSuchAlgorithmException {
+  public void testDeleteBlock() throws IOException, NoSuchAlgorithmException {
     long testContainerID = getTestContainerID();
     Container container = addContainer(containerSet, testContainerID);
     BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
     ChunkInfo info = writeChunkHelper(blockID);
-    KeyData keyData = new KeyData(blockID);
+    BlockData blockData = new BlockData(blockID);
     List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkList.add(info.getProtoBufMessage());
-    keyData.setChunks(chunkList);
-    keyManager.putKey(container, keyData);
-    keyManager.deleteKey(container, blockID);
+    blockData.setChunks(chunkList);
+    blockManager.putBlock(container, blockData);
+    blockManager.deleteBlock(container, blockID);
     exception.expect(StorageContainerException.class);
-    exception.expectMessage("Unable to find the key.");
-    keyManager.getKey(container, keyData.getBlockID());
+    exception.expectMessage("Unable to find the block.");
+    blockManager.getBlock(container, blockData.getBlockID());
   }
 
   /**
-   * Tries to Deletes a key twice.
+   * Tries to Deletes a block twice.
    *
    * @throws IOException
    * @throws NoSuchAlgorithmException
    */
   @Test
-  public void testDeleteKeyTwice() throws IOException,
+  public void testDeleteBlockTwice() throws IOException,
       NoSuchAlgorithmException {
     long testContainerID = getTestContainerID();
     Container container = addContainer(containerSet, testContainerID);
     BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
     ChunkInfo info = writeChunkHelper(blockID);
-    KeyData keyData = new KeyData(blockID);
+    BlockData blockData = new BlockData(blockID);
     List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkList.add(info.getProtoBufMessage());
-    keyData.setChunks(chunkList);
-    keyManager.putKey(container, keyData);
-    keyManager.deleteKey(container, blockID);
+    blockData.setChunks(chunkList);
+    blockManager.putBlock(container, blockData);
+    blockManager.deleteBlock(container, blockID);
     exception.expect(StorageContainerException.class);
-    exception.expectMessage("Unable to find the key.");
-    keyManager.deleteKey(container, blockID);
+    exception.expectMessage("Unable to find the block.");
+    blockManager.deleteBlock(container, blockID);
   }
 
   /**
@@ -722,8 +729,9 @@ public class TestContainerPersistence {
     try {
       container.update(newMetadata, false);
     } catch (StorageContainerException ex) {
-      Assert.assertEquals("Updating a closed container without force option " +
-          "is not allowed. ContainerID: " + testContainerID, ex.getMessage());
+      Assert.assertEquals("Updating a closed container without " +
+          "force option is not allowed. ContainerID: " +
+          testContainerID, ex.getMessage());
     }
 
     // Update with force flag, it should be success.
@@ -741,53 +749,55 @@ public class TestContainerPersistence {
 
   }
 
-  private KeyData writeKeyHelper(BlockID blockID)
+  private BlockData writeBlockHelper(BlockID blockID)
       throws IOException, NoSuchAlgorithmException {
     ChunkInfo info = writeChunkHelper(blockID);
-    KeyData keyData = new KeyData(blockID);
+    BlockData blockData = new BlockData(blockID);
     List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkList.add(info.getProtoBufMessage());
-    keyData.setChunks(chunkList);
-    return keyData;
+    blockData.setChunks(chunkList);
+    return blockData;
   }
 
   @Test
-  public void testListKey() throws Exception {
+  public void testListBlock() throws Exception {
     long testContainerID = getTestContainerID();
     Container container = addContainer(containerSet, testContainerID);
-    List<BlockID> expectedKeys = new ArrayList<>();
+    List<BlockID> expectedBlocks = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
       BlockID blockID = new BlockID(testContainerID, i);
-      expectedKeys.add(blockID);
-      KeyData kd = writeKeyHelper(blockID);
-      keyManager.putKey(container, kd);
+      expectedBlocks.add(blockID);
+      BlockData kd = writeBlockHelper(blockID);
+      blockManager.putBlock(container, kd);
     }
 
-    // List all keys
-    List<KeyData> result = keyManager.listKey(container, 0, 100);
+    // List all blocks
+    List<BlockData> result = blockManager.listBlock(
+        container, 0, 100);
     Assert.assertEquals(10, result.size());
 
     int index = 0;
     for (int i = index; i < result.size(); i++) {
-      KeyData data = result.get(i);
+      BlockData data = result.get(i);
       Assert.assertEquals(testContainerID, data.getContainerID());
-      Assert.assertEquals(expectedKeys.get(i).getLocalID(), data.getLocalID());
+      Assert.assertEquals(expectedBlocks.get(i).getLocalID(),
+          data.getLocalID());
       index++;
     }
 
-    // List key with startKey filter
-    long k6 = expectedKeys.get(6).getLocalID();
-    result = keyManager.listKey(container, k6, 100);
+    // List block with startBlock filter
+    long k6 = expectedBlocks.get(6).getLocalID();
+    result = blockManager.listBlock(container, k6, 100);
 
     Assert.assertEquals(4, result.size());
     for (int i = 6; i < 10; i++) {
-      Assert.assertEquals(expectedKeys.get(i).getLocalID(),
+      Assert.assertEquals(expectedBlocks.get(i).getLocalID(),
           result.get(i - 6).getLocalID());
     }
 
     // Count must be >0
     exception.expect(IllegalArgumentException.class);
     exception.expectMessage("Count must be a positive number.");
-    keyManager.listKey(container, 0, -1);
+    blockManager.listBlock(container, 0, -1);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 3c77687..a129ed0 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
@@ -69,6 +69,9 @@ import static org.apache.hadoop.hdds
 import static org.apache.hadoop.ozone
     .OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
 
+/**
+ * Tests for Block deletion.
+ */
 public class TestBlockDeletion {
   private static OzoneConfiguration conf = null;
   private static ObjectStore store;
@@ -229,7 +232,7 @@ public class TestBlockDeletion {
       throws IOException {
     return OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
       try {
-        MetadataStore db = KeyUtils.getDB((KeyValueContainerData)
+        MetadataStore db = BlockUtils.getDB((KeyValueContainerData)
                 dnContainerSet.getContainer(blockID.getContainerID())
                     .getContainerData(), conf);
         Assert.assertNotNull(db.get(Longs.toByteArray(blockID.getLocalID())));
@@ -244,7 +247,7 @@ public class TestBlockDeletion {
       throws IOException {
     return OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
       try {
-        MetadataStore db = KeyUtils.getDB((KeyValueContainerData)
+        MetadataStore db = BlockUtils.getDB((KeyValueContainerData)
             dnContainerSet.getContainer(blockID.getContainerID())
                 .getContainerData(), conf);
         Assert.assertNull(db.get(Longs.toByteArray(blockID.getLocalID())));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 5dd88fb..a3c92fb 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -68,7 +68,8 @@ public class TestOzoneContainer {
       conf.set(HDDS_DATANODE_DIR_KEY, tempFolder.getRoot().getPath());
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader()
               .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
-      conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
+      conf.setBoolean(
+          OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
 
       container = new OzoneContainer(TestUtils.randomDatanodeDetails(),
           conf, null);
@@ -129,7 +130,7 @@ public class TestOzoneContainer {
   static void runTestOzoneContainerViaDataNode(
       long testContainerID, XceiverClientSpi client) throws Exception {
     ContainerProtos.ContainerCommandRequestProto
-        request, writeChunkRequest, putKeyRequest,
+        request, writeChunkRequest, putBlockRequest,
         updateRequest1, updateRequest2;
     ContainerProtos.ContainerCommandResponseProto response,
         updateResponse1, updateResponse2;
@@ -138,46 +139,50 @@ public class TestOzoneContainer {
 
       Pipeline pipeline = client.getPipeline();
       createContainerForTesting(client, testContainerID);
-      writeChunkRequest = writeChunkForContainer(client, testContainerID, 1024);
+      writeChunkRequest = writeChunkForContainer(client, testContainerID,
+          1024);
 
       // Read Chunk
-      request = ContainerTestHelper.getReadChunkRequest(pipeline, writeChunkRequest
-          .getWriteChunk());
+      request = ContainerTestHelper.getReadChunkRequest(
+          pipeline, writeChunkRequest.getWriteChunk());
 
       response = client.sendCommand(request);
       Assert.assertNotNull(response);
       Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
       Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
 
-      // Put Key
-      putKeyRequest = ContainerTestHelper.getPutKeyRequest(pipeline, writeChunkRequest
-              .getWriteChunk());
+      // Put Block
+      putBlockRequest = ContainerTestHelper.getPutBlockRequest(
+          pipeline, writeChunkRequest.getWriteChunk());
 
 
-      response = client.sendCommand(putKeyRequest);
+      response = client.sendCommand(putBlockRequest);
       Assert.assertNotNull(response);
       Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
-      Assert
-          .assertTrue(putKeyRequest.getTraceID().equals(response.getTraceID()));
+      Assert.assertTrue(putBlockRequest.getTraceID()
+          .equals(response.getTraceID()));
 
-      // Get Key
-      request = ContainerTestHelper.getKeyRequest(pipeline, putKeyRequest.getPutKey());
+      // Get Block
+      request = ContainerTestHelper.
+          getBlockRequest(pipeline, putBlockRequest.getPutBlock());
       response = client.sendCommand(request);
-      int chunksCount = putKeyRequest.getPutKey().getKeyData().getChunksCount();
-      ContainerTestHelper.verifyGetKey(request, response, chunksCount);
+      int chunksCount = putBlockRequest.getPutBlock().getBlockData().
+          getChunksCount();
+      ContainerTestHelper.verifyGetBlock(request, response, chunksCount);
 
 
-      // Delete Key
+      // Delete Block
       request =
-          ContainerTestHelper.getDeleteKeyRequest(pipeline, putKeyRequest.getPutKey());
+          ContainerTestHelper.getDeleteBlockRequest(
+              pipeline, putBlockRequest.getPutBlock());
       response = client.sendCommand(request);
       Assert.assertNotNull(response);
       Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
       Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
 
       //Delete Chunk
-      request = ContainerTestHelper.getDeleteChunkRequest(pipeline, writeChunkRequest
-          .getWriteChunk());
+      request = ContainerTestHelper.getDeleteChunkRequest(
+          pipeline, writeChunkRequest.getWriteChunk());
 
       response = client.sendCommand(request);
       Assert.assertNotNull(response);
@@ -249,7 +254,7 @@ public class TestOzoneContainer {
 
       final ContainerProtos.ContainerCommandRequestProto getSmallFileRequest
           = ContainerTestHelper.getReadSmallFileRequest(client.getPipeline(),
-          smallFileRequest.getPutSmallFile().getKey());
+          smallFileRequest.getPutSmallFile().getBlock());
       response = client.sendCommand(getSmallFileRequest);
       Assert.assertArrayEquals(
           smallFileRequest.getPutSmallFile().getData().toByteArray(),
@@ -269,7 +274,7 @@ public class TestOzoneContainer {
     XceiverClientGrpc client = null;
     ContainerProtos.ContainerCommandResponseProto response;
     ContainerProtos.ContainerCommandRequestProto
-        writeChunkRequest, putKeyRequest, request;
+        writeChunkRequest, putBlockRequest, request;
     try {
 
       OzoneConfiguration conf = newOzoneConfiguration();
@@ -283,18 +288,19 @@ public class TestOzoneContainer {
 
       long containerID = ContainerTestHelper.getTestContainerID();
       createContainerForTesting(client, containerID);
-      writeChunkRequest = writeChunkForContainer(client, containerID, 1024);
+      writeChunkRequest = writeChunkForContainer(client, containerID,
+          1024);
 
 
-      putKeyRequest = ContainerTestHelper.getPutKeyRequest(client.getPipeline(),
-          writeChunkRequest.getWriteChunk());
-      // Put key before closing.
-      response = client.sendCommand(putKeyRequest);
+      putBlockRequest = ContainerTestHelper.getPutBlockRequest(
+          client.getPipeline(), writeChunkRequest.getWriteChunk());
+      // Put block before closing.
+      response = client.sendCommand(putBlockRequest);
       Assert.assertNotNull(response);
       Assert.assertEquals(ContainerProtos.Result.SUCCESS,
           response.getResult());
       Assert.assertTrue(
-          putKeyRequest.getTraceID().equals(response.getTraceID()));
+          putBlockRequest.getTraceID().equals(response.getTraceID()));
 
       // Close the contianer.
       request = ContainerTestHelper.getCloseContainer(
@@ -325,25 +331,26 @@ public class TestOzoneContainer {
       Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
 
 
-      // Put key will fail on a closed container.
-      response = client.sendCommand(putKeyRequest);
+      // Put block will fail on a closed container.
+      response = client.sendCommand(putBlockRequest);
       Assert.assertNotNull(response);
       Assert.assertEquals(ContainerProtos.Result.CLOSED_CONTAINER_IO,
           response.getResult());
-      Assert
-          .assertTrue(putKeyRequest.getTraceID().equals(response.getTraceID()));
+      Assert.assertTrue(putBlockRequest.getTraceID()
+          .equals(response.getTraceID()));
 
-      // Get key must work on the closed container.
-      request = ContainerTestHelper.getKeyRequest(client.getPipeline(),
-          putKeyRequest.getPutKey());
+      // Get block must work on the closed container.
+      request = ContainerTestHelper.getBlockRequest(client.getPipeline(),
+          putBlockRequest.getPutBlock());
       response = client.sendCommand(request);
-      int chunksCount = putKeyRequest.getPutKey().getKeyData().getChunksCount();
-      ContainerTestHelper.verifyGetKey(request, response, chunksCount);
+      int chunksCount = putBlockRequest.getPutBlock().getBlockData()
+          .getChunksCount();
+      ContainerTestHelper.verifyGetBlock(request, response, chunksCount);
 
-      // Delete Key must fail on a closed container.
+      // Delete block must fail on a closed container.
       request =
-          ContainerTestHelper.getDeleteKeyRequest(client.getPipeline(),
-              putKeyRequest.getPutKey());
+          ContainerTestHelper.getDeleteBlockRequest(client.getPipeline(),
+              putBlockRequest.getPutBlock());
       response = client.sendCommand(request);
       Assert.assertNotNull(response);
       Assert.assertEquals(ContainerProtos.Result.CLOSED_CONTAINER_IO,
@@ -365,7 +372,7 @@ public class TestOzoneContainer {
     XceiverClientGrpc client = null;
     ContainerProtos.ContainerCommandResponseProto response;
     ContainerProtos.ContainerCommandRequestProto request,
-        writeChunkRequest, putKeyRequest;
+        writeChunkRequest, putBlockRequest;
     try {
       OzoneConfiguration conf = newOzoneConfiguration();
 
@@ -378,17 +385,18 @@ public class TestOzoneContainer {
 
       long containerID = ContainerTestHelper.getTestContainerID();
       createContainerForTesting(client, containerID);
-      writeChunkRequest = writeChunkForContainer(client, containerID, 1024);
+      writeChunkRequest = writeChunkForContainer(
+          client, containerID, 1024);
 
-      putKeyRequest = ContainerTestHelper.getPutKeyRequest(client.getPipeline(),
-          writeChunkRequest.getWriteChunk());
+      putBlockRequest = ContainerTestHelper.getPutBlockRequest(
+          client.getPipeline(), writeChunkRequest.getWriteChunk());
       // Put key before deleting.
-      response = client.sendCommand(putKeyRequest);
+      response = client.sendCommand(putBlockRequest);
       Assert.assertNotNull(response);
       Assert.assertEquals(ContainerProtos.Result.SUCCESS,
           response.getResult());
       Assert.assertTrue(
-          putKeyRequest.getTraceID().equals(response.getTraceID()));
+          putBlockRequest.getTraceID().equals(response.getTraceID()));
 
       // Container cannot be deleted forcibly because
       // the container is not closed.
@@ -529,7 +537,7 @@ public class TestOzoneContainer {
       writeChunkForContainer(XceiverClientSpi client,
       long containerID, int dataLen) throws Exception {
     // Write Chunk
-    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);;
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
     ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
         ContainerTestHelper.getWriteChunkRequest(client.getPipeline(),
             blockID, dataLen);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerStateMachine.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerStateMachine.java
index 8c83fd3..c875a7e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerStateMachine.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerStateMachine.java
@@ -89,7 +89,7 @@ public class TestContainerStateMachine {
 
     // add putKey request
     ContainerCommandRequestProto putKeyProto = ContainerTestHelper
-            .getPutKeyRequest(pipeline, writeChunkProto.getWriteChunk());
+            .getPutBlockRequest(pipeline, writeChunkProto.getWriteChunk());
     RaftClientRequest putKeyRequest = getRaftClientRequest(putKeyProto);
 
     TransactionContext createContainerCtxt =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
index f309715..7144005 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
@@ -1209,8 +1209,8 @@ public class TestOzoneManager {
   //Disabling this test
   @Ignore("Disabling this test until Open Key is fixed.")
   public void testExpiredOpenKey() throws Exception {
-//    BackgroundService openKeyCleanUpService = ((KeyManagerImpl)cluster
-//        .getOzoneManager().getKeyManager()).getOpenKeyCleanupService();
+//    BackgroundService openKeyCleanUpService = ((BlockManagerImpl)cluster
+//        .getOzoneManager().getBlockManager()).getOpenKeyCleanupService();
 
     String userName = "user" + RandomStringUtils.randomNumeric(5);
     String adminName = "admin" + RandomStringUtils.randomNumeric(5);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
index a2d95e8..84a4028 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
@@ -103,7 +103,7 @@ public class TestContainerSmallFile {
   }
 
   @Test
-  public void testInvalidKeyRead() throws Exception {
+  public void testInvalidBlockRead() throws Exception {
     String traceID = UUID.randomUUID().toString();
     ContainerWithPipeline container =
         storageContainerLocationClient.allocateContainer(
@@ -116,7 +116,7 @@ public class TestContainerSmallFile {
         container.getContainerInfo().getContainerID(), traceID);
 
     thrown.expect(StorageContainerException.class);
-    thrown.expectMessage("Unable to find the key");
+    thrown.expectMessage("Unable to find the block");
 
     BlockID blockID = ContainerTestHelper.getTestBlockID(
         container.getContainerInfo().getContainerID());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java
index f82b0d3..08e7808 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java
@@ -107,7 +107,7 @@ public class TestGetCommittedBlockLengthAndPutKey {
     // Now, explicitly make a putKey request for the block.
     ContainerProtos.ContainerCommandRequestProto putKeyRequest =
         ContainerTestHelper
-            .getPutKeyRequest(pipeline, writeChunkRequest.getWriteChunk());
+            .getPutBlockRequest(pipeline, writeChunkRequest.getWriteChunk());
     client.sendCommand(putKeyRequest);
     response = ContainerProtocolCalls
         .getCommittedBlockLength(client, blockID, traceID);
@@ -155,7 +155,7 @@ public class TestGetCommittedBlockLengthAndPutKey {
   }
 
   @Test
-  public void tesGetCommittedBlockLengthForInvalidBlock() throws Exception {
+  public void testGetCommittedBlockLengthForInvalidBlock() throws Exception {
     String traceID = UUID.randomUUID().toString();
     ContainerWithPipeline container = storageContainerLocationClient
         .allocateContainer(xceiverClientManager.getType(),
@@ -174,7 +174,7 @@ public class TestGetCommittedBlockLengthAndPutKey {
       ContainerProtocolCalls.getCommittedBlockLength(client, blockID, traceID);
       Assert.fail("Expected exception not thrown");
     } catch (StorageContainerException sce) {
-      Assert.assertTrue(sce.getMessage().contains("Unable to find the key"));
+      Assert.assertTrue(sce.getMessage().contains("Unable to find the block"));
     }
     xceiverClientManager.releaseClient(client);
   }
@@ -216,7 +216,7 @@ public class TestGetCommittedBlockLengthAndPutKey {
 
   @Test
   public void tesPutKeyResposne() throws Exception {
-    ContainerProtos.PutKeyResponseProto response;
+    ContainerProtos.PutBlockResponseProto response;
     String traceID = UUID.randomUUID().toString();
     ContainerWithPipeline container = storageContainerLocationClient
         .allocateContainer(xceiverClientManager.getType(),
@@ -239,8 +239,8 @@ public class TestGetCommittedBlockLengthAndPutKey {
     // Now, explicitly make a putKey request for the block.
     ContainerProtos.ContainerCommandRequestProto putKeyRequest =
         ContainerTestHelper
-            .getPutKeyRequest(pipeline, writeChunkRequest.getWriteChunk());
-    response = client.sendCommand(putKeyRequest).getPutKey();
+            .getPutBlockRequest(pipeline, writeChunkRequest.getWriteChunk());
+    response = client.sendCommand(putKeyRequest).getPutBlock();
     // make sure the block ids in the request and response are same.
     // This will also ensure that closing the container committed the block
     // on the Datanodes.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index 02cd985..7eb2ec2 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
@@ -278,29 +278,29 @@ public class TestKeys {
   }
 
   static void runTestPutKey(PutHelper helper) throws Exception {
-    final ClientProtocol client = helper.client;
+    final ClientProtocol helperClient = helper.client;
     helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
-    List<OzoneKey> keyList = client
+    List<OzoneKey> keyList = helperClient
         .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
             null, 10);
     Assert.assertEquals(1, keyList.size());
 
     // test list key using a more efficient call
     String newkeyName = OzoneUtils.getRequestID().toLowerCase();
-    OzoneOutputStream ozoneOutputStream = client
+    OzoneOutputStream ozoneOutputStream = helperClient
         .createKey(helper.getVol().getName(), helper.getBucket().getName(),
             newkeyName, 0, replicationType, replicationFactor);
     ozoneOutputStream.close();
-    keyList = client
+    keyList = helperClient
         .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
             null, 10);
     Assert.assertEquals(2, keyList.size());
 
     // test new put key with invalid volume/bucket name
     try {
-      ozoneOutputStream = client
+      ozoneOutputStream = helperClient
           .createKey("invalid-volume", helper.getBucket().getName(), newkeyName,
               0, replicationType, replicationFactor);
       ozoneOutputStream.close();
@@ -312,7 +312,7 @@ public class TestKeys {
     }
 
     try {
-      ozoneOutputStream = client
+      ozoneOutputStream = helperClient
           .createKey(helper.getVol().getName(), "invalid-bucket", newkeyName, 0,
               replicationType, replicationFactor);
       ozoneOutputStream.close();
@@ -380,7 +380,7 @@ public class TestKeys {
   }
 
   static void runTestPutAndGetKey(PutHelper helper) throws Exception {
-    final ClientProtocol client = helper.client;
+    final ClientProtocol helperClient = helper.client;
 
     String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
@@ -427,7 +427,8 @@ public class TestKeys {
 
       // test new get key with invalid volume/bucket name
       try {
-        client.getKey("invalid-volume", helper.getBucket().getName(), keyName);
+        helperClient.getKey(
+            "invalid-volume", helper.getBucket().getName(), keyName);
         fail("Get key should have thrown " + "when using invalid volume name.");
       } catch (IOException e) {
         GenericTestUtils
@@ -435,7 +436,8 @@ public class TestKeys {
       }
 
       try {
-        client.getKey(helper.getVol().getName(), "invalid-bucket", keyName);
+        helperClient.getKey(
+            helper.getVol().getName(), "invalid-bucket", keyName);
         fail("Get key should have thrown " + "when using invalid bucket name.");
       } catch (IOException e) {
         GenericTestUtils.assertExceptionContains(
@@ -476,7 +478,7 @@ public class TestKeys {
   }
 
   static void runTestPutAndListKey(PutHelper helper) throws Exception {
-    ClientProtocol client = helper.client;
+    ClientProtocol helperClient = helper.client;
     helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
@@ -495,7 +497,7 @@ public class TestKeys {
     List<OzoneKey> keyList1 =
         IteratorUtils.toList(helper.getBucket().listKeys(null, null));
     // test list key using a more efficient call
-    List<OzoneKey> keyList2 = client
+    List<OzoneKey> keyList2 = helperClient
         .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
             null, 100);
 
@@ -515,7 +517,7 @@ public class TestKeys {
     }
 
     // test maxLength parameter of list keys
-    keyList2 = client
+    keyList2 = helperClient
         .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
             null, 1);
     Assert.assertEquals(1, keyList2.size());
@@ -523,7 +525,7 @@ public class TestKeys {
     // test startKey parameter of list keys
     keyList1 = IteratorUtils
         .toList(helper.getBucket().listKeys("list-key", "list-key4"));
-    keyList2 = client
+    keyList2 = helperClient
         .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
             "list-key", "list-key4", 100);
     Assert.assertEquals(5, keyList1.size());
@@ -532,7 +534,7 @@ public class TestKeys {
     // test prefix parameter of list keys
     keyList1 =
         IteratorUtils.toList(helper.getBucket().listKeys("list-key2", null));
-    keyList2 = client
+    keyList2 = helperClient
         .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
             "list-key2", null, 100);
     Assert.assertTrue(
@@ -542,7 +544,7 @@ public class TestKeys {
 
     // test new list keys with invalid volume/bucket name
     try {
-      client.listKeys("invalid-volume", helper.getBucket().getName(),
+      helperClient.listKeys("invalid-volume", helper.getBucket().getName(),
           null, null, 100);
       fail("List keys should have thrown when using invalid volume name.");
     } catch (IOException e) {
@@ -551,7 +553,7 @@ public class TestKeys {
     }
 
     try {
-      client.listKeys(helper.getVol().getName(), "invalid-bucket", null,
+      helperClient.listKeys(helper.getVol().getName(), "invalid-bucket", null,
           null, 100);
       fail("List keys should have thrown when using invalid bucket name.");
     } catch (IOException e) {
@@ -697,10 +699,10 @@ public class TestKeys {
                   .KeyValueContainer);
           KeyValueContainer container = (KeyValueContainer) cm.getContainerSet()
               .getContainer(location.getBlockID().getContainerID());
-          KeyData blockInfo = keyValueHandler
-              .getKeyManager().getKey(container, location.getBlockID());
-          KeyValueContainerData containerData = (KeyValueContainerData) container
-              .getContainerData();
+          BlockData blockInfo = keyValueHandler
+              .getBlockManager().getBlock(container, location.getBlockID());
+          KeyValueContainerData containerData =
+              (KeyValueContainerData) container.getContainerData();
           File dataDir = new File(containerData.getChunksPath());
           for (ContainerProtos.ChunkInfo chunkInfo : blockInfo.getChunks()) {
             File chunkFile = dataDir.toPath()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
index 9c451e2..3e740d9 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
@@ -54,7 +54,7 @@ public class BucketManagerImpl implements BucketManager {
 
   /**
    * MetadataDB is maintained in MetadataManager and shared between
-   * BucketManager and VolumeManager. (and also by KeyManager)
+   * BucketManager and VolumeManager. (and also by BlockManager)
    *
    * BucketManager uses MetadataDB to store bucket level information.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestIngClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestIngClient.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestIngClient.java
index 2da60de..eb533e8 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestIngClient.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestIngClient.java
@@ -102,7 +102,7 @@ public class ScmBlockLocationTestIngClient implements ScmBlockLocationProtocol {
   }
 
   /**
-   * Returns Fake blocks to the KeyManager so we get blocks in the Database.
+   * Returns Fake blocks to the BlockManager so we get blocks in the Database.
    * @param size - size of the block.
    * @param type Replication Type
    * @param factor - Replication factor

http://git-wip-us.apache.org/repos/asf/hadoop/blob/096a7160/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
index 5ac7e0a..8811d91 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
@@ -48,17 +48,16 @@ import java.util.Random;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 
-
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .PutBlockRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .GetBlockRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ReadChunkRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .WriteChunkRequestProto;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .PutKeyRequestProto;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .GetKeyRequestProto;
 
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 
@@ -141,7 +140,7 @@ public class BenchMarkDatanodeDispatcher {
         long containerID = containers.get(y);
         BlockID  blockID = new BlockID(containerID, key);
         dispatcher
-            .dispatch(getPutKeyCommand(blockID, chunkName));
+            .dispatch(getPutBlockCommand(blockID, chunkName));
         dispatcher.dispatch(getWriteChunkCommand(blockID, chunkName));
       }
     }
@@ -213,38 +212,39 @@ public class BenchMarkDatanodeDispatcher {
     return builder.build();
   }
 
-  private ContainerCommandRequestProto getPutKeyCommand(
+  private ContainerCommandRequestProto getPutBlockCommand(
       BlockID blockID, String chunkKey) {
-    PutKeyRequestProto.Builder putKeyRequest = PutKeyRequestProto
+    PutBlockRequestProto.Builder putBlockRequest = PutBlockRequestProto
         .newBuilder()
-        .setKeyData(getKeyData(blockID, chunkKey));
+        .setBlockData(getBlockData(blockID, chunkKey));
 
     ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
         .newBuilder();
-    request.setCmdType(ContainerProtos.Type.PutKey)
+    request.setCmdType(ContainerProtos.Type.PutBlock)
         .setContainerID(blockID.getContainerID())
         .setTraceID(getBlockTraceID(blockID))
         .setDatanodeUuid(datanodeUuid)
-        .setPutKey(putKeyRequest);
+        .setPutBlock(putBlockRequest);
     return request.build();
   }
 
-  private ContainerCommandRequestProto getGetKeyCommand(BlockID blockID) {
-    GetKeyRequestProto.Builder readKeyRequest = GetKeyRequestProto.newBuilder()
+  private ContainerCommandRequestProto getGetBlockCommand(BlockID blockID) {
+    GetBlockRequestProto.Builder readBlockRequest =
+        GetBlockRequestProto.newBuilder()
         .setBlockID(blockID.getDatanodeBlockIDProtobuf());
     ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
         .newBuilder()
-        .setCmdType(ContainerProtos.Type.GetKey)
+        .setCmdType(ContainerProtos.Type.GetBlock)
         .setContainerID(blockID.getContainerID())
         .setTraceID(getBlockTraceID(blockID))
         .setDatanodeUuid(datanodeUuid)
-        .setGetKey(readKeyRequest);
+        .setGetBlock(readBlockRequest);
     return request.build();
   }
 
-  private ContainerProtos.KeyData getKeyData(
+  private ContainerProtos.BlockData getBlockData(
       BlockID blockID, String chunkKey) {
-    ContainerProtos.KeyData.Builder builder =  ContainerProtos.KeyData
+    ContainerProtos.BlockData.Builder builder =  ContainerProtos.BlockData
         .newBuilder()
         .setBlockID(blockID.getDatanodeBlockIDProtobuf())
         .addChunks(getChunkInfo(blockID, chunkKey));
@@ -275,16 +275,16 @@ public class BenchMarkDatanodeDispatcher {
   }
 
   @Benchmark
-  public void putKey(BenchMarkDatanodeDispatcher bmdd) {
+  public void putBlock(BenchMarkDatanodeDispatcher bmdd) {
     BlockID blockID = getRandomBlockID();
     String chunkKey = getNewChunkToWrite();
-    bmdd.dispatcher.dispatch(getPutKeyCommand(blockID, chunkKey));
+    bmdd.dispatcher.dispatch(getPutBlockCommand(blockID, chunkKey));
   }
 
   @Benchmark
-  public void getKey(BenchMarkDatanodeDispatcher bmdd) {
+  public void getBlock(BenchMarkDatanodeDispatcher bmdd) {
     BlockID blockID = getRandomBlockID();
-    bmdd.dispatcher.dispatch(getGetKeyCommand(blockID));
+    bmdd.dispatcher.dispatch(getGetBlockCommand(blockID));
   }
 
   // Chunks writes from benchmark only reaches certain containers


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