This is an automated email from the ASF dual-hosted git repository.

adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 557449e916 HDDS-8546. Ozone debug chunkinfo command throws 
GroupMismatchException for closed container (#4661)
557449e916 is described below

commit 557449e916758bb1e4801e061fe6c7a5a4417de0
Author: Sadanand Shenoy <[email protected]>
AuthorDate: Mon May 8 21:42:09 2023 +0530

    HDDS-8546. Ozone debug chunkinfo command throws GroupMismatchException for 
closed container (#4661)
---
 .../org/apache/hadoop/ozone/OzoneTestUtils.java    |  17 ++
 .../hadoop/ozone/scm/TestCloseContainer.java       |  10 +-
 .../hadoop/ozone/shell/TestOzoneDebugShell.java    | 181 ++++++++++++++++++
 .../apache/hadoop/ozone/debug/ChunkKeyHandler.java | 211 +++++++++++----------
 .../org/apache/hadoop/ozone/debug/OzoneDebug.java  |   7 +
 5 files changed, 313 insertions(+), 113 deletions(-)

diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java
index dba9739780..0e3101be6f 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java
@@ -25,17 +25,20 @@ import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.ozone.test.GenericTestUtils;
 import org.apache.ozone.test.LambdaTestUtils.VoidCallable;
 
 import org.apache.ratis.util.IOUtils;
@@ -169,4 +172,18 @@ public final class OzoneTestUtils {
     }
     return sockets;
   }
+
+  /**
+    * Close container & Wait till container state becomes CLOSED.
+   */
+  public static void closeContainer(StorageContainerManager scm,
+      ContainerInfo container)
+      throws IOException, TimeoutException, InterruptedException {
+    Pipeline pipeline = scm.getPipelineManager()
+        .getPipeline(container.getPipelineID());
+    scm.getPipelineManager().closePipeline(pipeline, false);
+    GenericTestUtils.waitFor(() ->
+            container.getState() == HddsProtos.LifeCycleState.CLOSED,
+        200, 30000);
+  }
 }
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCloseContainer.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCloseContainer.java
index 635ff14ecc..c019a540b3 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCloseContainer.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCloseContainer.java
@@ -21,13 +21,12 @@ import org.apache.hadoop.hdds.utils.IOUtils;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import 
org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration;
-import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneTestUtils;
 import org.apache.hadoop.ozone.TestDataUtil;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneClient;
@@ -115,12 +114,7 @@ public class TestCloseContainer {
     // Pick any container on the cluster, get its pipeline, close it and then
     // wait for the container to close
     ContainerInfo container = scm.getContainerManager().getContainers().get(0);
-    Pipeline pipeline = scm.getPipelineManager()
-        .getPipeline(container.getPipelineID());
-    scm.getPipelineManager().closePipeline(pipeline, false);
-    GenericTestUtils.waitFor(() ->
-            container.getState() == HddsProtos.LifeCycleState.CLOSED,
-        200, 30000);
+    OzoneTestUtils.closeContainer(scm, container);
 
     long originalSeq = container.getSequenceId();
 
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneDebugShell.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneDebugShell.java
new file mode 100644
index 0000000000..da96bd1c83
--- /dev/null
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneDebugShell.java
@@ -0,0 +1,181 @@
+/*
+ * 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.shell;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneTestUtils;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.debug.OzoneDebug;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static 
org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL;
+import static 
org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL;
+import static 
org.apache.hadoop.hdds.HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL;
+import static 
org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
+
+/**
+ * Test Ozone Debug shell.
+ */
+public class TestOzoneDebugShell {
+
+  private static String omServiceId;
+  private static String clusterId;
+  private static String scmId;
+
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneClient client;
+
+  private static OzoneConfiguration conf = null;
+
+  protected static void startCluster() throws Exception {
+    // Init HA cluster
+    omServiceId = "om-service-test1";
+    clusterId = UUID.randomUUID().toString();
+    scmId = UUID.randomUUID().toString();
+    final int numDNs = 3;
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setClusterId(clusterId)
+        .setScmId(scmId)
+        .setOMServiceId(omServiceId)
+        .setNumDatanodes(numDNs)
+        .build();
+    cluster.waitForClusterToBeReady();
+    client = cluster.newClient();
+  }
+
+
+  @BeforeAll
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
+        100, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, 1, SECONDS);
+    conf.setTimeDuration(HDDS_PIPELINE_REPORT_INTERVAL, 1, SECONDS);
+    conf.setTimeDuration(HDDS_COMMAND_STATUS_REPORT_INTERVAL, 1, SECONDS);
+    conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 1, SECONDS);
+    ReplicationManager.ReplicationManagerConfiguration replicationConf =
+        conf.getObject(
+            ReplicationManager.ReplicationManagerConfiguration.class);
+    replicationConf.setInterval(Duration.ofSeconds(1));
+    conf.setFromObject(replicationConf);
+    startCluster();
+  }
+
+  @Test
+  public void testChunkInfoCmdBeforeAfterCloseContainer() throws Exception {
+    final String volumeName = UUID.randomUUID().toString();
+    final String bucketName = UUID.randomUUID().toString();
+    final String keyName = UUID.randomUUID().toString();
+
+    writeKey(volumeName, bucketName, keyName);
+
+    int exitCode = runChunkInfoCommand(volumeName, bucketName, keyName);
+    Assertions.assertEquals(0, exitCode);
+
+    closeContainerForKey(volumeName, bucketName, keyName);
+
+    exitCode = runChunkInfoCommand(volumeName, bucketName, keyName);
+    Assertions.assertEquals(0, exitCode);
+  }
+
+  private static void writeKey(String volumeName, String bucketName,
+      String keyName) throws IOException {
+    try (OzoneClient client = OzoneClientFactory.getRpcClient(conf)) {
+      TestDataUtil.createVolumeAndBucket(client, volumeName, bucketName);
+      TestDataUtil.createKey(
+          client.getObjectStore().getVolume(volumeName).getBucket(bucketName),
+          keyName, ReplicationFactor.THREE, ReplicationType.RATIS, "test");
+    }
+  }
+
+  private int runChunkInfoCommand(String volumeName, String bucketName,
+      String keyName) {
+    String bucketPath =
+        Path.SEPARATOR + volumeName + Path.SEPARATOR + bucketName;
+
+    String[] args = new String[] {
+        getSetConfStringFromConf(OMConfigKeys.OZONE_OM_ADDRESS_KEY),
+        "chunkinfo", bucketPath + Path.SEPARATOR + keyName };
+
+    OzoneDebug ozoneDebugShell = new OzoneDebug(conf);
+    int exitCode = ozoneDebugShell.execute(args);
+    return exitCode;
+  }
+
+  /**
+   * Generate string to pass as extra arguments to the
+   * ozone debug command line, This is necessary for client to
+   * connect to OM by setting the right om address.
+   */
+  private String getSetConfStringFromConf(String key) {
+    return String.format("--set=%s=%s", key, conf.get(key));
+  }
+
+  private static void closeContainerForKey(String volumeName, String 
bucketName,
+      String keyName)
+      throws IOException, TimeoutException, InterruptedException {
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
+        .setBucketName(bucketName).setKeyName(keyName).build();
+
+    OmKeyLocationInfo omKeyLocationInfo =
+        cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
+            .get(0).getBlocksLatestVersionOnly().get(0);
+
+    ContainerInfo container =
+        
cluster.getStorageContainerManager().getContainerManager().getContainer(
+            ContainerID.valueOf(omKeyLocationInfo.getContainerID()));
+    OzoneTestUtils.closeContainer(cluster.getStorageContainerManager(),
+        container);
+  }
+
+  /**
+   * shutdown MiniOzoneCluster.
+   */
+  @AfterAll
+  public static void shutdown() {
+    IOUtils.closeQuietly(client);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+}
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ChunkKeyHandler.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ChunkKeyHandler.java
index 82d4bbd980..38845eb81e 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ChunkKeyHandler.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ChunkKeyHandler.java
@@ -52,6 +52,7 @@ import 
org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
 import org.apache.hadoop.ozone.shell.OzoneAddress;
 import org.apache.hadoop.ozone.shell.keys.KeyHandler;
 import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
 import picocli.CommandLine.Command;
 
 import static 
org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
@@ -66,11 +67,13 @@ import static 
org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor
 public class ChunkKeyHandler extends KeyHandler implements
     SubcommandWithParent {
 
-  private ContainerOperationClient containerOperationClient;
   private  XceiverClientManager xceiverClientManager;
   private XceiverClientSpi xceiverClient;
   private OzoneManagerProtocol ozoneManagerClient;
 
+  @CommandLine.ParentCommand
+  private OzoneDebug parent;
+
   private String getChunkLocationPath(String containerLocation) {
     return containerLocation + File.separator + OzoneConsts.STORAGE_DIR_CHUNKS;
   }
@@ -78,117 +81,115 @@ public class ChunkKeyHandler extends KeyHandler implements
   @Override
   protected void execute(OzoneClient client, OzoneAddress address)
           throws IOException, OzoneClientException {
-    containerOperationClient = new
-            ContainerOperationClient(createOzoneConfiguration());
-    xceiverClientManager = containerOperationClient
-            .getXceiverClientManager();
-    ozoneManagerClient = client.getObjectStore().getClientProxy()
-            .getOzoneManagerClient();
-    address.ensureKeyAddress();
-    JsonElement element;
-    JsonObject result = new JsonObject();
-    String volumeName = address.getVolumeName();
-    String bucketName = address.getBucketName();
-    String keyName = address.getKeyName();
-    List<ContainerProtos.ChunkInfo> tempchunks = null;
-    List<ChunkDetails> chunkDetailsList = new ArrayList<ChunkDetails>();
-    HashSet<String> chunkPaths = new HashSet<>();
-    OmKeyArgs keyArgs = new OmKeyArgs.Builder()
-            .setVolumeName(volumeName)
-            .setBucketName(bucketName)
-            .setKeyName(keyName)
-            .build();
-    OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
-    // querying  the keyLocations.The OM is queried to get containerID and
-    // localID pertaining to a given key
-    List<OmKeyLocationInfo> locationInfos = keyInfo
-            .getLatestVersionLocations().getBlocksLatestVersionOnly();
-    // for zero-sized key
-    if (locationInfos.isEmpty()) {
-      System.out.println("No Key Locations Found");
-      return;
-    }
-    ContainerLayoutVersion containerLayoutVersion = ContainerLayoutVersion
-            .getConfiguredVersion(getConf());
-    JsonArray responseArrayList = new JsonArray();
-    for (OmKeyLocationInfo keyLocation:locationInfos) {
-      ContainerChunkInfo containerChunkInfoVerbose = new ContainerChunkInfo();
-      ContainerChunkInfo containerChunkInfo = new ContainerChunkInfo();
-      long containerId = keyLocation.getContainerID();
-      chunkPaths.clear();
-      Pipeline pipeline = keyLocation.getPipeline();
-      if (pipeline.getType() != HddsProtos.ReplicationType.STAND_ALONE) {
-        pipeline = Pipeline.newBuilder(pipeline)
-            .setReplicationConfig(StandaloneReplicationConfig.getInstance(ONE))
-            .build();
-      }
-      xceiverClient = xceiverClientManager
-              .acquireClientForReadData(pipeline);
-      // Datanode is queried to get chunk information.Thus querying the
-      // OM,SCM and datanode helps us get chunk location information
-      ContainerProtos.DatanodeBlockID datanodeBlockID = 
keyLocation.getBlockID()
-              .getDatanodeBlockIDProtobuf();
-      // doing a getBlock on all nodes
-      HashMap<DatanodeDetails, ContainerProtos.GetBlockResponseProto>
-              responses = null;
-      try {
-        responses = ContainerProtocolCalls.getBlockFromAllNodes(
-            xceiverClient, datanodeBlockID, keyLocation.getToken());
-      } catch (InterruptedException e) {
-        LOG.error("Execution interrupted due to " + e);
-        Thread.currentThread().interrupt();
+    try (ContainerOperationClient containerOperationClient = new
+        ContainerOperationClient(parent.getOzoneConf())) {
+      xceiverClientManager = 
containerOperationClient.getXceiverClientManager();
+      ozoneManagerClient =
+          client.getObjectStore().getClientProxy().getOzoneManagerClient();
+      address.ensureKeyAddress();
+      JsonElement element;
+      JsonObject result = new JsonObject();
+      String volumeName = address.getVolumeName();
+      String bucketName = address.getBucketName();
+      String keyName = address.getKeyName();
+      List<ContainerProtos.ChunkInfo> tempchunks = null;
+      List<ChunkDetails> chunkDetailsList = new ArrayList<ChunkDetails>();
+      HashSet<String> chunkPaths = new HashSet<>();
+      OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
+          .setBucketName(bucketName).setKeyName(keyName).build();
+      OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
+      // querying  the keyLocations.The OM is queried to get containerID and
+      // localID pertaining to a given key
+      List<OmKeyLocationInfo> locationInfos =
+          keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly();
+      // for zero-sized key
+      if (locationInfos.isEmpty()) {
+        System.out.println("No Key Locations Found");
+        return;
       }
-      JsonArray responseFromAllNodes = new JsonArray();
-      for (Map.Entry<DatanodeDetails, ContainerProtos.GetBlockResponseProto>
-              entry: responses.entrySet()) {
-        JsonObject jsonObj = new JsonObject();
-        if (entry.getValue() == null) {
-          LOG.error("Cant execute getBlock on this node");
-          continue;
+      ContainerLayoutVersion containerLayoutVersion = ContainerLayoutVersion
+          .getConfiguredVersion(getConf());
+      JsonArray responseArrayList = new JsonArray();
+      for (OmKeyLocationInfo keyLocation : locationInfos) {
+        ContainerChunkInfo containerChunkInfoVerbose = new 
ContainerChunkInfo();
+        ContainerChunkInfo containerChunkInfo = new ContainerChunkInfo();
+        long containerId = keyLocation.getContainerID();
+        chunkPaths.clear();
+        Pipeline pipeline = keyLocation.getPipeline();
+        if (pipeline.getType() != HddsProtos.ReplicationType.STAND_ALONE) {
+          pipeline = Pipeline.newBuilder(pipeline)
+              .setReplicationConfig(StandaloneReplicationConfig
+                  .getInstance(ONE)).build();
         }
-        tempchunks = entry.getValue().getBlockData().getChunksList();
-        ContainerProtos.ContainerDataProto containerData =
-                containerOperationClient.readContainer(
-                        keyLocation.getContainerID(),
-                        keyLocation.getPipeline());
-        for (ContainerProtos.ChunkInfo chunkInfo : tempchunks) {
-          String fileName = containerLayoutVersion.getChunkFile(new File(
-              getChunkLocationPath(containerData.getContainerPath())),
-                  keyLocation.getBlockID(),
-                  ChunkInfo.getFromProtoBuf(chunkInfo)).toString();
-          chunkPaths.add(fileName);
-          ChunkDetails chunkDetails = new ChunkDetails();
-          chunkDetails.setChunkName(fileName);
-          chunkDetails.setChunkOffset(chunkInfo.getOffset());
-          chunkDetailsList.add(chunkDetails);
+        xceiverClient = 
xceiverClientManager.acquireClientForReadData(pipeline);
+        // Datanode is queried to get chunk information.Thus querying the
+        // OM,SCM and datanode helps us get chunk location information
+        ContainerProtos.DatanodeBlockID datanodeBlockID =
+            keyLocation.getBlockID().getDatanodeBlockIDProtobuf();
+        // doing a getBlock on all nodes
+        HashMap<DatanodeDetails, ContainerProtos.GetBlockResponseProto>
+            responses = null;
+        try {
+          responses = 
ContainerProtocolCalls.getBlockFromAllNodes(xceiverClient,
+              datanodeBlockID, keyLocation.getToken());
+        } catch (InterruptedException e) {
+          LOG.error("Execution interrupted due to " + e);
+          Thread.currentThread().interrupt();
         }
-        containerChunkInfoVerbose
-                .setContainerPath(containerData.getContainerPath());
-        containerChunkInfoVerbose.setPipeline(keyLocation.getPipeline());
-        containerChunkInfoVerbose.setChunkInfos(chunkDetailsList);
-        containerChunkInfo.setFiles(chunkPaths);
-        containerChunkInfo.setPipelineID(
-                keyLocation.getPipeline().getId().getId());
-        Gson gson = new GsonBuilder().create();
-        if (isVerbose()) {
-          element = gson.toJsonTree(containerChunkInfoVerbose);
-        } else {
-          element = gson.toJsonTree(containerChunkInfo);
+        JsonArray responseFromAllNodes = new JsonArray();
+        for (Map.Entry<DatanodeDetails, ContainerProtos.GetBlockResponseProto>
+            entry : responses.entrySet()) {
+          JsonObject jsonObj = new JsonObject();
+          if (entry.getValue() == null) {
+            LOG.error("Cant execute getBlock on this node");
+            continue;
+          }
+          tempchunks = entry.getValue().getBlockData().getChunksList();
+          ContainerProtos.ContainerDataProto containerData =
+              containerOperationClient.readContainer(keyLocation
+                  .getContainerID(), pipeline);
+          for (ContainerProtos.ChunkInfo chunkInfo : tempchunks) {
+            String fileName = containerLayoutVersion.getChunkFile(new File(
+                    getChunkLocationPath(containerData.getContainerPath())),
+                keyLocation.getBlockID(),
+                ChunkInfo.getFromProtoBuf(chunkInfo)).toString();
+            chunkPaths.add(fileName);
+            ChunkDetails chunkDetails = new ChunkDetails();
+            chunkDetails.setChunkName(fileName);
+            chunkDetails.setChunkOffset(chunkInfo.getOffset());
+            chunkDetailsList.add(chunkDetails);
+          }
+          containerChunkInfoVerbose.setContainerPath(containerData
+              .getContainerPath());
+          containerChunkInfoVerbose.setPipeline(keyLocation.getPipeline());
+          containerChunkInfoVerbose.setChunkInfos(chunkDetailsList);
+          containerChunkInfo.setFiles(chunkPaths);
+          containerChunkInfo.setPipelineID(
+              keyLocation.getPipeline().getId().getId());
+          Gson gson = new GsonBuilder().create();
+          if (isVerbose()) {
+            element = gson.toJsonTree(containerChunkInfoVerbose);
+          } else {
+            element = gson.toJsonTree(containerChunkInfo);
+          }
+          jsonObj.addProperty("Datanode-HostName", entry.getKey()
+              .getHostName());
+          jsonObj.addProperty("Datanode-IP", entry.getKey()
+              .getIpAddress());
+          jsonObj.addProperty("Container-ID", containerId);
+          jsonObj.addProperty("Block-ID", keyLocation.getLocalID());
+          jsonObj.add("Locations", element);
+          responseFromAllNodes.add(jsonObj);
         }
-        jsonObj.addProperty("Datanode-HostName", entry.getKey().getHostName());
-        jsonObj.addProperty("Datanode-IP", entry.getKey().getIpAddress());
-        jsonObj.addProperty("Container-ID", containerId);
-        jsonObj.addProperty("Block-ID", keyLocation.getLocalID());
-        jsonObj.add("Locations", element);
-        responseFromAllNodes.add(jsonObj);
+        responseArrayList.add(responseFromAllNodes);
         xceiverClientManager.releaseClientForReadData(xceiverClient, false);
+        xceiverClient = null;
       }
-      responseArrayList.add(responseFromAllNodes);
+      result.add("KeyLocations", responseArrayList);
+      Gson gson2 = new GsonBuilder().setPrettyPrinting().create();
+      String prettyJson = gson2.toJson(result);
+      System.out.println(prettyJson);
     }
-    result.add("KeyLocations", responseArrayList);
-    Gson gson2 = new GsonBuilder().setPrettyPrinting().create();
-    String prettyJson = gson2.toJson(result);
-    System.out.println(prettyJson);
   }
 
   @Override
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/OzoneDebug.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/OzoneDebug.java
index 41b96da7b3..3d6cf57093 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/OzoneDebug.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/OzoneDebug.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.debug;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 
@@ -39,6 +40,12 @@ public class OzoneDebug extends GenericCli {
     super(OzoneDebug.class);
   }
 
+  @VisibleForTesting
+  public OzoneDebug(OzoneConfiguration configuration) {
+    super(OzoneDebug.class);
+    this.ozoneConf = configuration;
+  }
+
   public OzoneConfiguration getOzoneConf() {
     if (ozoneConf == null) {
       ozoneConf = createOzoneConfiguration();


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to