You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/01/13 17:35:10 UTC

[GitHub] [ozone] dombizita opened a new pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

dombizita opened a new pull request #2987:
URL: https://github.com/apache/ozone/pull/2987


   ## What changes were proposed in this pull request?
   
   In this change I added a debug CLI tool, called read-replicas. The tool for a given key will download every blocks every replicas (from each datanode) and also creates a manifest file with informations about the key, its blocks and replicas. 
   
   The command can be used like this:
   `ozone debug read-replicas /vol1/bucket1/testfile
   `
   
   It will create a directory with the volume, bucket and key name; it also adds a timestamp, so you can re-run it. The downloaded replicas and the manifest file will be in this directory. The manifest file will look something like this:
   
   ```
   cat vol1_bucket1_testfile_20220113142642/testfile_manifest 
   {
     "filename": "vol1/bucket1/testfile",
     "datasize": 20000000,
     "blocks": [
       {
         "block index": 1,
         "container id": 1,
         "local id": 109611004723200001,
         "length": 20000000,
         "offset": 0,
         "replicas": [
           {
             "hostname": "ozone_datanode_3.ozone_default",
             "uuid": "075c52cf-0876-4cd8-94d1-104de5ba700f"
           },
           {
             "hostname": "ozone_datanode_2.ozone_default",
             "uuid": "ee456701-95fa-4277-a44f-7fff9cd07ace"
           },
           {
             "hostname": "ozone_datanode_1.ozone_default",
             "uuid": "6554007e-ec0b-46b0-aa29-9ba162bdbd9c"
           }
         ]
       }
     ]
   }
   ```
   If there is a checksum mismatch with one of the replicas it will be marked in the manifest file and the corrupt replica is still going to be downloaded. If one of the datanodes is down (and still alive) it will also be marked in the manifest file.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6128
   
   ## How was this patch tested?
   
   Built the project successfully and tested the command locally in docker environment. Specific tests will be added as the next sub-task.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r786569068



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads a given keys every blocks every replica.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);
+
+    address.ensureKeyAddress();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
+
+    OzoneKeyDetails keyInfoDetails
+        = clientProtocol.getKeyDetails(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas
+        = clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        replicasWithoutChecksum = clientProtocolWithoutChecksum
+        .getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    String directoryName = createDirectory(volumeName, bucketName, keyName);
+
+    JsonObject result = new JsonObject();
+    result.addProperty("filename",
+        volumeName + "/" + bucketName + "/" + keyName);
+    result.addProperty("datasize", keyInfoDetails.getDataSize());
+
+    JsonArray blocks = new JsonArray();
+    downloadReplicasAndCreateManifest(keyName, replicas,
+        replicasWithoutChecksum, directoryName, blocks);
+    result.add("blocks", blocks);
+
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    String prettyJson = gson.toJson(result);
+
+    String manifestFileName = keyName + "_manifest";
+    System.out.println("Writing manifest file : " + manifestFileName);
+    File manifestFile
+        = new File("/opt/hadoop/" + directoryName + "/" + manifestFileName);
+    Files.write(manifestFile.toPath(),
+        prettyJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private void downloadReplicasAndCreateManifest(
+      String keyName,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum,
+      String directoryName, JsonArray blocks) throws IOException {
+    int blockIndex = 0;
+
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicas.entrySet()) {
+      JsonObject blockJson = new JsonObject();
+      JsonArray replicasJson = new JsonArray();
+
+      blockIndex += 1;
+      blockJson.addProperty("block index", blockIndex);
+      blockJson.addProperty("container id", block.getKey().getContainerID());
+      blockJson.addProperty("local id", block.getKey().getLocalID());

Review comment:
       That seems a reasonable, as it will be easier to work with JSON file. Changed it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r785130040



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -82,28 +80,7 @@
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.om.helpers.OmDeleteKeys;
-import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
-import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteList;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
-import org.apache.hadoop.ozone.om.helpers.OmPartInfo;
-import org.apache.hadoop.ozone.om.helpers.OmRenameKeys;
-import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
-import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
-import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
-import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
-import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
-import org.apache.hadoop.ozone.om.helpers.ServiceInfoEx;
+import org.apache.hadoop.ozone.om.helpers.*;

Review comment:
       Please replace star import with individual imports.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r784907695



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.

Review comment:
       Could you please change the comment to be more clear - `Class that downloads every replica for all the blocks associated with a given key. It also generates a manifest file with information about the downloaded replicas.`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r791792598



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);

Review comment:
       Thanks, I didn't think it through, that the original configuration can have the checksum verification turned off. Changed the code, so whether the configuration is set to check or not the checksum, I will have two clients, one with checksum verification and one without.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r791598977



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);
+
+    address.ensureKeyAddress();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
+
+    OzoneKeyDetails keyInfoDetails
+        = clientProtocol.getKeyDetails(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas
+        = clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        replicasWithoutChecksum = clientProtocolWithoutChecksum
+        .getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    String directoryName = createDirectory(volumeName, bucketName, keyName);
+
+    JsonObject result = new JsonObject();
+    result.addProperty("filename",
+        volumeName + "/" + bucketName + "/" + keyName);
+    result.addProperty("datasize", keyInfoDetails.getDataSize());
+
+    JsonArray blocks = new JsonArray();
+    downloadReplicasAndCreateManifest(keyName, replicas,
+        replicasWithoutChecksum, directoryName, blocks);
+    result.add("blocks", blocks);
+
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    String prettyJson = gson.toJson(result);
+
+    String manifestFileName = keyName + "_manifest";
+    System.out.println("Writing manifest file : " + manifestFileName);
+    File manifestFile
+        = new File("/opt/hadoop/" + directoryName + "/" + manifestFileName);
+    Files.write(manifestFile.toPath(),
+        prettyJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private void downloadReplicasAndCreateManifest(
+      String keyName,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum,
+      String directoryName, JsonArray blocks) throws IOException {
+    int blockIndex = 0;
+
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicas.entrySet()) {
+      JsonObject blockJson = new JsonObject();
+      JsonArray replicasJson = new JsonArray();
+
+      blockIndex += 1;
+      blockJson.addProperty("blockIndex", blockIndex);
+      blockJson.addProperty("containerId", block.getKey().getContainerID());
+      blockJson.addProperty("localId", block.getKey().getLocalID());
+      blockJson.addProperty("length", block.getKey().getLength());
+      blockJson.addProperty("offset", block.getKey().getOffset());
+
+      for (Map.Entry<DatanodeDetails, OzoneInputStream>
+          replica : block.getValue().entrySet()) {
+        JsonObject replicaJson = new JsonObject();
+
+        replicaJson.addProperty("hostname",
+            replica.getKey().getHostName());
+        replicaJson.addProperty("uuid",
+            replica.getKey().getUuidString());
+
+        OzoneInputStream is = replica.getValue();
+        String fileName = keyName + "_block" + blockIndex + "_" +
+            replica.getKey().getHostName();
+        System.out.println("Writing : " + fileName);
+        File replicaFile
+            = new File("/opt/hadoop/" + directoryName + "/" + fileName);
+
+        try {
+          Files.copy(is, replicaFile.toPath(),
+              StandardCopyOption.REPLACE_EXISTING);
+        } catch (IOException e) {
+          Throwable cause = e.getCause();
+          replicaJson.addProperty("exception", e.getMessage());
+          if(cause instanceof OzoneChecksumException) {
+            BlockID blockID = block.getKey().getBlockID();
+            String datanodeUUID = replica.getKey().getUuidString();
+            is = getInputStreamWithoutChecksum(replicasWithoutChecksum,
+                datanodeUUID, blockID);
+            Files.copy(is, replicaFile.toPath(),
+                StandardCopyOption.REPLACE_EXISTING);
+          } else if(cause instanceof StatusRuntimeException) {
+            break;
+          }
+        } finally {
+          is.close();
+        }
+        replicasJson.add(replicaJson);
+      }
+      blockJson.add("replicas", replicasJson);
+      blocks.add(blockJson);
+    }
+  }
+
+  private OzoneInputStream getInputStreamWithoutChecksum(
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum, String datanodeUUID, BlockID blockID) {
+    OzoneInputStream is = new OzoneInputStream();
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicasWithoutChecksum.entrySet()) {
+      if(block.getKey().getBlockID().equals(blockID)) {
+        for (Map.Entry<DatanodeDetails, OzoneInputStream>
+            replica : block.getValue().entrySet()) {
+          if(replica.getKey().getUuidString().equals(datanodeUUID)) {
+            is = replica.getValue();
+          }
+        }
+      }
+    }
+    return is;
+  }
+
+  @NotNull
+  private String createDirectory(String volumeName, String bucketName,
+                                 String keyName) {
+    String fileSuffix
+        = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
+    String directoryName = volumeName + "_" + bucketName + "_" + keyName +
+        "_" + fileSuffix;
+    System.out.println("Creating directory : " + directoryName);
+    File dir = new File("/opt/hadoop/" + directoryName);
+    if (!dir.exists()){
+      if(dir.mkdir()) {
+        System.out.println("Successfully created!");
+      } else {
+        System.out.println("Something went wrong.");

Review comment:
       Thanks, this is a good idea. I moved the execution earlier and changed the handling when the directory cannot be created. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r791705233



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);

Review comment:
       Simplification of the code is done, and great, but the other problem I mentioned have been remained.
   
   This code will not give you any error status if in the original configuration that comes from the getConf() method is already has the checksum verification set to false.
   Note that this means that the first client we use does not check checksum failures, and then we create a second client that is meant to read data from blocks with checksum failures, but we will not detect any checksum failures with the first client, as checksum verification is turned off there as well.
   
   Ultimately we would like to get the checksum errors, so if the original config turns off checksum verification, then we need a client set to verify the checksum, and use that for initial reads here I guess, so that we still detect checksum errors, and still we can download the data with the client which is set to skip the checksum validation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
fapifta commented on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1021257017


   Thank you @dombizita for addressing my comments.
   
   +1, let's wait for CI, and if green, I will go ahead and merge this one later on.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r785129835



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
##########
@@ -42,11 +43,7 @@
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
-import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
-import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.helpers.*;

Review comment:
       Please replace star import with individual imports.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1015244302


   > Please add tests to verify correct behavior for the newly added `hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java` class.
   
   Hi @JyotinderSingh, thanks for taking your time to look at my work. I didn't add any tests yet, as that is my next subtask on my jira (https://issues.apache.org/jira/browse/HDDS-5739). I will add integration level tests to the read-replicas CLI tool and I am already working on it, I'm planning to finish it this week.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
fapifta commented on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1021184266






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r784938257



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads a given keys every blocks every replica.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);
+
+    address.ensureKeyAddress();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
+
+    OzoneKeyDetails keyInfoDetails
+        = clientProtocol.getKeyDetails(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas
+        = clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        replicasWithoutChecksum = clientProtocolWithoutChecksum
+        .getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    String directoryName = createDirectory(volumeName, bucketName, keyName);
+
+    JsonObject result = new JsonObject();
+    result.addProperty("filename",
+        volumeName + "/" + bucketName + "/" + keyName);
+    result.addProperty("datasize", keyInfoDetails.getDataSize());
+
+    JsonArray blocks = new JsonArray();
+    downloadReplicasAndCreateManifest(keyName, replicas,
+        replicasWithoutChecksum, directoryName, blocks);
+    result.add("blocks", blocks);
+
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    String prettyJson = gson.toJson(result);
+
+    String manifestFileName = keyName + "_manifest";
+    System.out.println("Writing manifest file : " + manifestFileName);
+    File manifestFile
+        = new File("/opt/hadoop/" + directoryName + "/" + manifestFileName);
+    Files.write(manifestFile.toPath(),
+        prettyJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private void downloadReplicasAndCreateManifest(
+      String keyName,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum,
+      String directoryName, JsonArray blocks) throws IOException {
+    int blockIndex = 0;
+
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicas.entrySet()) {
+      JsonObject blockJson = new JsonObject();
+      JsonArray replicasJson = new JsonArray();
+
+      blockIndex += 1;
+      blockJson.addProperty("block index", blockIndex);
+      blockJson.addProperty("container id", block.getKey().getContainerID());
+      blockJson.addProperty("local id", block.getKey().getLocalID());

Review comment:
       nit: could you change the property names to be in camel case instead of having spaces?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
fapifta commented on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1021184266


   Thank you @dombizita to address my comments, I still have one thing that has to be solved, and one suggestion regarding the constant names.
   The problematic one I added as an additional inline command around configuration and secondary client creation.
   
   With the constants, I would do one more thing, I would try to add a bit more descriptive names, like:
   JSON_PROPERTY_FILE_NAME
   JSON_PROPERTY_FILE_SIZE
   JSON_PROPERTY_FILE_BLOCKS
   JSON_PROPERTY_BLOCKS_INDEX
   JSON_PROPERTY_BLOCKS_CONTAINERID
   ...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r786569898



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -82,28 +80,7 @@
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.om.helpers.OmDeleteKeys;
-import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
-import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteList;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
-import org.apache.hadoop.ozone.om.helpers.OmPartInfo;
-import org.apache.hadoop.ozone.om.helpers.OmRenameKeys;
-import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
-import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
-import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
-import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
-import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
-import org.apache.hadoop.ozone.om.helpers.ServiceInfoEx;
+import org.apache.hadoop.ozone.om.helpers.*;

Review comment:
       Thanks, replaced them.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh edited a comment on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh edited a comment on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1013206300


   Please add tests to verify correct behavior for the newly added `hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java` class.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r786567961



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads a given keys every blocks every replica.")

Review comment:
       Thanks for the correction, I changed it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r791600779



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);

Review comment:
       Thanks, it was much easier this way, I changed it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r790667350



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);
+
+    address.ensureKeyAddress();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
+
+    OzoneKeyDetails keyInfoDetails
+        = clientProtocol.getKeyDetails(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas
+        = clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        replicasWithoutChecksum = clientProtocolWithoutChecksum
+        .getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    String directoryName = createDirectory(volumeName, bucketName, keyName);
+
+    JsonObject result = new JsonObject();
+    result.addProperty("filename",
+        volumeName + "/" + bucketName + "/" + keyName);
+    result.addProperty("datasize", keyInfoDetails.getDataSize());
+
+    JsonArray blocks = new JsonArray();
+    downloadReplicasAndCreateManifest(keyName, replicas,
+        replicasWithoutChecksum, directoryName, blocks);
+    result.add("blocks", blocks);
+
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    String prettyJson = gson.toJson(result);
+
+    String manifestFileName = keyName + "_manifest";
+    System.out.println("Writing manifest file : " + manifestFileName);
+    File manifestFile
+        = new File("/opt/hadoop/" + directoryName + "/" + manifestFileName);
+    Files.write(manifestFile.toPath(),
+        prettyJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private void downloadReplicasAndCreateManifest(
+      String keyName,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum,
+      String directoryName, JsonArray blocks) throws IOException {
+    int blockIndex = 0;
+
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicas.entrySet()) {
+      JsonObject blockJson = new JsonObject();
+      JsonArray replicasJson = new JsonArray();
+
+      blockIndex += 1;
+      blockJson.addProperty("blockIndex", blockIndex);
+      blockJson.addProperty("containerId", block.getKey().getContainerID());
+      blockJson.addProperty("localId", block.getKey().getLocalID());
+      blockJson.addProperty("length", block.getKey().getLength());
+      blockJson.addProperty("offset", block.getKey().getOffset());
+
+      for (Map.Entry<DatanodeDetails, OzoneInputStream>
+          replica : block.getValue().entrySet()) {
+        JsonObject replicaJson = new JsonObject();
+
+        replicaJson.addProperty("hostname",
+            replica.getKey().getHostName());
+        replicaJson.addProperty("uuid",
+            replica.getKey().getUuidString());
+
+        OzoneInputStream is = replica.getValue();
+        String fileName = keyName + "_block" + blockIndex + "_" +
+            replica.getKey().getHostName();
+        System.out.println("Writing : " + fileName);
+        File replicaFile
+            = new File("/opt/hadoop/" + directoryName + "/" + fileName);
+
+        try {
+          Files.copy(is, replicaFile.toPath(),
+              StandardCopyOption.REPLACE_EXISTING);
+        } catch (IOException e) {
+          Throwable cause = e.getCause();
+          replicaJson.addProperty("exception", e.getMessage());
+          if(cause instanceof OzoneChecksumException) {
+            BlockID blockID = block.getKey().getBlockID();
+            String datanodeUUID = replica.getKey().getUuidString();
+            is = getInputStreamWithoutChecksum(replicasWithoutChecksum,
+                datanodeUUID, blockID);
+            Files.copy(is, replicaFile.toPath(),
+                StandardCopyOption.REPLACE_EXISTING);
+          } else if(cause instanceof StatusRuntimeException) {
+            break;
+          }
+        } finally {
+          is.close();
+        }
+        replicasJson.add(replicaJson);
+      }
+      blockJson.add("replicas", replicasJson);
+      blocks.add(blockJson);
+    }
+  }
+
+  private OzoneInputStream getInputStreamWithoutChecksum(
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum, String datanodeUUID, BlockID blockID) {
+    OzoneInputStream is = new OzoneInputStream();
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicasWithoutChecksum.entrySet()) {
+      if(block.getKey().getBlockID().equals(blockID)) {
+        for (Map.Entry<DatanodeDetails, OzoneInputStream>
+            replica : block.getValue().entrySet()) {
+          if(replica.getKey().getUuidString().equals(datanodeUUID)) {
+            is = replica.getValue();
+          }
+        }
+      }
+    }
+    return is;
+  }
+
+  @NotNull
+  private String createDirectory(String volumeName, String bucketName,
+                                 String keyName) {
+    String fileSuffix
+        = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
+    String directoryName = volumeName + "_" + bucketName + "_" + keyName +
+        "_" + fileSuffix;
+    System.out.println("Creating directory : " + directoryName);
+    File dir = new File("/opt/hadoop/" + directoryName);
+    if (!dir.exists()){
+      if(dir.mkdir()) {
+        System.out.println("Successfully created!");
+      } else {
+        System.out.println("Something went wrong.");

Review comment:
       We might call this method at the very beginning of execution, and throw an exception here if the output directory can not be created, so that we fail early, and do not start gathering data that we can not save.

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);
+
+    address.ensureKeyAddress();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
+
+    OzoneKeyDetails keyInfoDetails
+        = clientProtocol.getKeyDetails(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas
+        = clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        replicasWithoutChecksum = clientProtocolWithoutChecksum
+        .getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    String directoryName = createDirectory(volumeName, bucketName, keyName);
+
+    JsonObject result = new JsonObject();
+    result.addProperty("filename",
+        volumeName + "/" + bucketName + "/" + keyName);
+    result.addProperty("datasize", keyInfoDetails.getDataSize());
+
+    JsonArray blocks = new JsonArray();
+    downloadReplicasAndCreateManifest(keyName, replicas,
+        replicasWithoutChecksum, directoryName, blocks);
+    result.add("blocks", blocks);
+
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    String prettyJson = gson.toJson(result);
+
+    String manifestFileName = keyName + "_manifest";
+    System.out.println("Writing manifest file : " + manifestFileName);
+    File manifestFile
+        = new File("/opt/hadoop/" + directoryName + "/" + manifestFileName);
+    Files.write(manifestFile.toPath(),
+        prettyJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private void downloadReplicasAndCreateManifest(
+      String keyName,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum,
+      String directoryName, JsonArray blocks) throws IOException {
+    int blockIndex = 0;
+
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicas.entrySet()) {
+      JsonObject blockJson = new JsonObject();
+      JsonArray replicasJson = new JsonArray();
+
+      blockIndex += 1;
+      blockJson.addProperty("blockIndex", blockIndex);
+      blockJson.addProperty("containerId", block.getKey().getContainerID());
+      blockJson.addProperty("localId", block.getKey().getLocalID());
+      blockJson.addProperty("length", block.getKey().getLength());
+      blockJson.addProperty("offset", block.getKey().getOffset());
+
+      for (Map.Entry<DatanodeDetails, OzoneInputStream>
+          replica : block.getValue().entrySet()) {
+        JsonObject replicaJson = new JsonObject();
+
+        replicaJson.addProperty("hostname",
+            replica.getKey().getHostName());
+        replicaJson.addProperty("uuid",
+            replica.getKey().getUuidString());
+
+        OzoneInputStream is = replica.getValue();
+        String fileName = keyName + "_block" + blockIndex + "_" +
+            replica.getKey().getHostName();
+        System.out.println("Writing : " + fileName);
+        File replicaFile
+            = new File("/opt/hadoop/" + directoryName + "/" + fileName);
+
+        try {
+          Files.copy(is, replicaFile.toPath(),
+              StandardCopyOption.REPLACE_EXISTING);
+        } catch (IOException e) {
+          Throwable cause = e.getCause();
+          replicaJson.addProperty("exception", e.getMessage());
+          if(cause instanceof OzoneChecksumException) {
+            BlockID blockID = block.getKey().getBlockID();
+            String datanodeUUID = replica.getKey().getUuidString();
+            is = getInputStreamWithoutChecksum(replicasWithoutChecksum,
+                datanodeUUID, blockID);
+            Files.copy(is, replicaFile.toPath(),
+                StandardCopyOption.REPLACE_EXISTING);
+          } else if(cause instanceof StatusRuntimeException) {
+            break;
+          }
+        } finally {
+          is.close();
+        }
+        replicasJson.add(replicaJson);
+      }
+      blockJson.add("replicas", replicasJson);
+      blocks.add(blockJson);
+    }
+  }
+
+  private OzoneInputStream getInputStreamWithoutChecksum(
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum, String datanodeUUID, BlockID blockID) {
+    OzoneInputStream is = new OzoneInputStream();
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicasWithoutChecksum.entrySet()) {
+      if(block.getKey().getBlockID().equals(blockID)) {
+        for (Map.Entry<DatanodeDetails, OzoneInputStream>
+            replica : block.getValue().entrySet()) {
+          if(replica.getKey().getUuidString().equals(datanodeUUID)) {
+            is = replica.getValue();
+          }
+        }
+      }
+    }
+    return is;
+  }
+
+  @NotNull
+  private String createDirectory(String volumeName, String bucketName,
+                                 String keyName) {
+    String fileSuffix
+        = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
+    String directoryName = volumeName + "_" + bucketName + "_" + keyName +
+        "_" + fileSuffix;
+    System.out.println("Creating directory : " + directoryName);
+    File dir = new File("/opt/hadoop/" + directoryName);

Review comment:
       `/opt/hadoop/` as a prefix for the directory name should not be hardcoded here, we should take that as a parameter defined for the command. For example named as -o, or --outputDir or something similar.
   
   This also goes for the path used for block files in line 154.

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);

Review comment:
       The configuration object here can be created a bit more easily. The ultimate goal would be to have an identical configuration for the two clients one with checksum verification turned on, the other with it turned off.
   
   You may use the copy constructor of OzoneConfiguration and get the original config by getConf() as the base config to copy, while to set the property you can use the setBoolean(String, boolean) method to set to property value without converting OzoneConfiguration to OzoneClientConfig and then set it back.
   
   An other case I just realized while writing this up is when checksum verification in the client's base config is turned off, as in that case both clients will have it turned off, and none of them will show checksum problems.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -920,6 +926,64 @@ public OzoneInputStream getKey(
     return getInputStreamWithRetryFunction(keyInfo);
   }
 
+  @Override
+  public Map< OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream> >
+      getKeysEveryReplicas(String volumeName,
+                         String bucketName,
+                         String keyName) throws IOException {
+
+    Map< OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream> > result
+        = new LinkedHashMap<>();
+
+    verifyVolumeName(volumeName);
+    verifyBucketName(bucketName);
+    Preconditions.checkNotNull(keyName);
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .setRefreshPipeline(true)
+        .setSortDatanodesInPipeline(topologyAwareReadEnabled)

Review comment:
       As we download all data, we do not need to sort the DataNodes for this request even in topology aware environments, and if we skip this, we spare some cycles on the OM side if we skip sorting, so I would like to ask to remove this call, as it defaults to false.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r784909336



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads a given keys every blocks every replica.")

Review comment:
       Could you modify the description to say `Reads every replica for all the blocks associated with a given key.`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r786636877



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads a given keys every blocks every replica.")

Review comment:
       Oh, I didn't changed it here, I changed it in the documentation at the ClientProtocol interface. Thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r786567719



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.

Review comment:
       Hi @JyotinderSingh, thanks for the correction, I changed it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r791600263



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);
+
+    address.ensureKeyAddress();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
+
+    OzoneKeyDetails keyInfoDetails
+        = clientProtocol.getKeyDetails(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas
+        = clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        replicasWithoutChecksum = clientProtocolWithoutChecksum
+        .getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    String directoryName = createDirectory(volumeName, bucketName, keyName);
+
+    JsonObject result = new JsonObject();
+    result.addProperty("filename",
+        volumeName + "/" + bucketName + "/" + keyName);
+    result.addProperty("datasize", keyInfoDetails.getDataSize());
+
+    JsonArray blocks = new JsonArray();
+    downloadReplicasAndCreateManifest(keyName, replicas,
+        replicasWithoutChecksum, directoryName, blocks);
+    result.add("blocks", blocks);
+
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    String prettyJson = gson.toJson(result);
+
+    String manifestFileName = keyName + "_manifest";
+    System.out.println("Writing manifest file : " + manifestFileName);
+    File manifestFile
+        = new File("/opt/hadoop/" + directoryName + "/" + manifestFileName);
+    Files.write(manifestFile.toPath(),
+        prettyJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private void downloadReplicasAndCreateManifest(
+      String keyName,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum,
+      String directoryName, JsonArray blocks) throws IOException {
+    int blockIndex = 0;
+
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicas.entrySet()) {
+      JsonObject blockJson = new JsonObject();
+      JsonArray replicasJson = new JsonArray();
+
+      blockIndex += 1;
+      blockJson.addProperty("blockIndex", blockIndex);
+      blockJson.addProperty("containerId", block.getKey().getContainerID());
+      blockJson.addProperty("localId", block.getKey().getLocalID());
+      blockJson.addProperty("length", block.getKey().getLength());
+      blockJson.addProperty("offset", block.getKey().getOffset());
+
+      for (Map.Entry<DatanodeDetails, OzoneInputStream>
+          replica : block.getValue().entrySet()) {
+        JsonObject replicaJson = new JsonObject();
+
+        replicaJson.addProperty("hostname",
+            replica.getKey().getHostName());
+        replicaJson.addProperty("uuid",
+            replica.getKey().getUuidString());
+
+        OzoneInputStream is = replica.getValue();
+        String fileName = keyName + "_block" + blockIndex + "_" +
+            replica.getKey().getHostName();
+        System.out.println("Writing : " + fileName);
+        File replicaFile
+            = new File("/opt/hadoop/" + directoryName + "/" + fileName);
+
+        try {
+          Files.copy(is, replicaFile.toPath(),
+              StandardCopyOption.REPLACE_EXISTING);
+        } catch (IOException e) {
+          Throwable cause = e.getCause();
+          replicaJson.addProperty("exception", e.getMessage());
+          if(cause instanceof OzoneChecksumException) {
+            BlockID blockID = block.getKey().getBlockID();
+            String datanodeUUID = replica.getKey().getUuidString();
+            is = getInputStreamWithoutChecksum(replicasWithoutChecksum,
+                datanodeUUID, blockID);
+            Files.copy(is, replicaFile.toPath(),
+                StandardCopyOption.REPLACE_EXISTING);
+          } else if(cause instanceof StatusRuntimeException) {
+            break;
+          }
+        } finally {
+          is.close();
+        }
+        replicasJson.add(replicaJson);
+      }
+      blockJson.add("replicas", replicasJson);
+      blocks.add(blockJson);
+    }
+  }
+
+  private OzoneInputStream getInputStreamWithoutChecksum(
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum, String datanodeUUID, BlockID blockID) {
+    OzoneInputStream is = new OzoneInputStream();
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicasWithoutChecksum.entrySet()) {
+      if(block.getKey().getBlockID().equals(blockID)) {
+        for (Map.Entry<DatanodeDetails, OzoneInputStream>
+            replica : block.getValue().entrySet()) {
+          if(replica.getKey().getUuidString().equals(datanodeUUID)) {
+            is = replica.getValue();
+          }
+        }
+      }
+    }
+    return is;
+  }
+
+  @NotNull
+  private String createDirectory(String volumeName, String bucketName,
+                                 String keyName) {
+    String fileSuffix
+        = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
+    String directoryName = volumeName + "_" + bucketName + "_" + keyName +
+        "_" + fileSuffix;
+    System.out.println("Creating directory : " + directoryName);
+    File dir = new File("/opt/hadoop/" + directoryName);

Review comment:
       Thanks for pointing this out, I added the --outputDir parameter to the `read-replicas` command. I didn't set it to be required as I set the default value to `/opt/hadoop/`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta merged pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
fapifta merged pull request #2987:
URL: https://github.com/apache/ozone/pull/2987


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1021081782


   > Hi Zita, thank you for contributing this new debug tool, I would like to request one general thing, and also I have a few comments inline.
   > 
   > I would generally suggest to create String constants and put them at the top of the class, for property names used in the manifest file, it would make it easier to see what properties we expect to have in the manifest if they are at one place. This is a stylistic suggestion, I can accept the current code in this regard, so I let this change for you to decide if it is worth it.
   
   Hi @fapifta, thank you so much for the review. I have taken your advices and changed the mentioned things. I also created the String constants, the code is much clearer this way.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r791705233



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);

Review comment:
       Simplification of the code is done, and great, but the other problem I mentioned have been remained.
   
   This code will not give you any error status if in the original configuration that comes from the getConf() method is already has the checksum verification set to false.
   Note that this means that the first client we use does not check checksum failures, and then we create a second client that is meant to read data from blocks with checksum failures, but we will not detect any checksum failures with the first client, as checksum verification is turned off there as well.
   
   Ultimately we would like to get the checksum errors, so if the original config turns off checksum verification, then we need a client set to verify the checksum, and use that for initial reads here I guess, so that we still detect checksum errors, and still we can download the data with the client which is set to skip the checksum validation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r791601326



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -920,6 +926,64 @@ public OzoneInputStream getKey(
     return getInputStreamWithRetryFunction(keyInfo);
   }
 
+  @Override
+  public Map< OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream> >
+      getKeysEveryReplicas(String volumeName,
+                         String bucketName,
+                         String keyName) throws IOException {
+
+    Map< OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream> > result
+        = new LinkedHashMap<>();
+
+    verifyVolumeName(volumeName);
+    verifyBucketName(bucketName);
+    Preconditions.checkNotNull(keyName);
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .setRefreshPipeline(true)
+        .setSortDatanodesInPipeline(topologyAwareReadEnabled)

Review comment:
       Thanks, I removed the unnecessary call.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r786569650



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
##########
@@ -42,11 +43,7 @@
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
-import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
-import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.om.helpers.*;

Review comment:
       Thanks, replaced it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1015282948


   Thanks for the contribution and for updating the patch @dombizita, the changes look good to me. I just had one minor comment that I've added inline.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r791598977



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);
+
+    address.ensureKeyAddress();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
+
+    OzoneKeyDetails keyInfoDetails
+        = clientProtocol.getKeyDetails(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas
+        = clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        replicasWithoutChecksum = clientProtocolWithoutChecksum
+        .getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    String directoryName = createDirectory(volumeName, bucketName, keyName);
+
+    JsonObject result = new JsonObject();
+    result.addProperty("filename",
+        volumeName + "/" + bucketName + "/" + keyName);
+    result.addProperty("datasize", keyInfoDetails.getDataSize());
+
+    JsonArray blocks = new JsonArray();
+    downloadReplicasAndCreateManifest(keyName, replicas,
+        replicasWithoutChecksum, directoryName, blocks);
+    result.add("blocks", blocks);
+
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    String prettyJson = gson.toJson(result);
+
+    String manifestFileName = keyName + "_manifest";
+    System.out.println("Writing manifest file : " + manifestFileName);
+    File manifestFile
+        = new File("/opt/hadoop/" + directoryName + "/" + manifestFileName);
+    Files.write(manifestFile.toPath(),
+        prettyJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private void downloadReplicasAndCreateManifest(
+      String keyName,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum,
+      String directoryName, JsonArray blocks) throws IOException {
+    int blockIndex = 0;
+
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicas.entrySet()) {
+      JsonObject blockJson = new JsonObject();
+      JsonArray replicasJson = new JsonArray();
+
+      blockIndex += 1;
+      blockJson.addProperty("blockIndex", blockIndex);
+      blockJson.addProperty("containerId", block.getKey().getContainerID());
+      blockJson.addProperty("localId", block.getKey().getLocalID());
+      blockJson.addProperty("length", block.getKey().getLength());
+      blockJson.addProperty("offset", block.getKey().getOffset());
+
+      for (Map.Entry<DatanodeDetails, OzoneInputStream>
+          replica : block.getValue().entrySet()) {
+        JsonObject replicaJson = new JsonObject();
+
+        replicaJson.addProperty("hostname",
+            replica.getKey().getHostName());
+        replicaJson.addProperty("uuid",
+            replica.getKey().getUuidString());
+
+        OzoneInputStream is = replica.getValue();
+        String fileName = keyName + "_block" + blockIndex + "_" +
+            replica.getKey().getHostName();
+        System.out.println("Writing : " + fileName);
+        File replicaFile
+            = new File("/opt/hadoop/" + directoryName + "/" + fileName);
+
+        try {
+          Files.copy(is, replicaFile.toPath(),
+              StandardCopyOption.REPLACE_EXISTING);
+        } catch (IOException e) {
+          Throwable cause = e.getCause();
+          replicaJson.addProperty("exception", e.getMessage());
+          if(cause instanceof OzoneChecksumException) {
+            BlockID blockID = block.getKey().getBlockID();
+            String datanodeUUID = replica.getKey().getUuidString();
+            is = getInputStreamWithoutChecksum(replicasWithoutChecksum,
+                datanodeUUID, blockID);
+            Files.copy(is, replicaFile.toPath(),
+                StandardCopyOption.REPLACE_EXISTING);
+          } else if(cause instanceof StatusRuntimeException) {
+            break;
+          }
+        } finally {
+          is.close();
+        }
+        replicasJson.add(replicaJson);
+      }
+      blockJson.add("replicas", replicasJson);
+      blocks.add(blockJson);
+    }
+  }
+
+  private OzoneInputStream getInputStreamWithoutChecksum(
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum, String datanodeUUID, BlockID blockID) {
+    OzoneInputStream is = new OzoneInputStream();
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicasWithoutChecksum.entrySet()) {
+      if(block.getKey().getBlockID().equals(blockID)) {
+        for (Map.Entry<DatanodeDetails, OzoneInputStream>
+            replica : block.getValue().entrySet()) {
+          if(replica.getKey().getUuidString().equals(datanodeUUID)) {
+            is = replica.getValue();
+          }
+        }
+      }
+    }
+    return is;
+  }
+
+  @NotNull
+  private String createDirectory(String volumeName, String bucketName,
+                                 String keyName) {
+    String fileSuffix
+        = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
+    String directoryName = volumeName + "_" + bucketName + "_" + keyName +
+        "_" + fileSuffix;
+    System.out.println("Creating directory : " + directoryName);
+    File dir = new File("/opt/hadoop/" + directoryName);
+    if (!dir.exists()){
+      if(dir.mkdir()) {
+        System.out.println("Successfully created!");
+      } else {
+        System.out.println("Something went wrong.");

Review comment:
       Thanks, this is a good idea. I moved the execution earlier and changed the handling when the directory cannot be created. 

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);
+
+    address.ensureKeyAddress();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
+
+    OzoneKeyDetails keyInfoDetails
+        = clientProtocol.getKeyDetails(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas
+        = clientProtocol.getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        replicasWithoutChecksum = clientProtocolWithoutChecksum
+        .getKeysEveryReplicas(volumeName, bucketName, keyName);
+
+    String directoryName = createDirectory(volumeName, bucketName, keyName);
+
+    JsonObject result = new JsonObject();
+    result.addProperty("filename",
+        volumeName + "/" + bucketName + "/" + keyName);
+    result.addProperty("datasize", keyInfoDetails.getDataSize());
+
+    JsonArray blocks = new JsonArray();
+    downloadReplicasAndCreateManifest(keyName, replicas,
+        replicasWithoutChecksum, directoryName, blocks);
+    result.add("blocks", blocks);
+
+    Gson gson = new GsonBuilder().setPrettyPrinting().create();
+    String prettyJson = gson.toJson(result);
+
+    String manifestFileName = keyName + "_manifest";
+    System.out.println("Writing manifest file : " + manifestFileName);
+    File manifestFile
+        = new File("/opt/hadoop/" + directoryName + "/" + manifestFileName);
+    Files.write(manifestFile.toPath(),
+        prettyJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private void downloadReplicasAndCreateManifest(
+      String keyName,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum,
+      String directoryName, JsonArray blocks) throws IOException {
+    int blockIndex = 0;
+
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicas.entrySet()) {
+      JsonObject blockJson = new JsonObject();
+      JsonArray replicasJson = new JsonArray();
+
+      blockIndex += 1;
+      blockJson.addProperty("blockIndex", blockIndex);
+      blockJson.addProperty("containerId", block.getKey().getContainerID());
+      blockJson.addProperty("localId", block.getKey().getLocalID());
+      blockJson.addProperty("length", block.getKey().getLength());
+      blockJson.addProperty("offset", block.getKey().getOffset());
+
+      for (Map.Entry<DatanodeDetails, OzoneInputStream>
+          replica : block.getValue().entrySet()) {
+        JsonObject replicaJson = new JsonObject();
+
+        replicaJson.addProperty("hostname",
+            replica.getKey().getHostName());
+        replicaJson.addProperty("uuid",
+            replica.getKey().getUuidString());
+
+        OzoneInputStream is = replica.getValue();
+        String fileName = keyName + "_block" + blockIndex + "_" +
+            replica.getKey().getHostName();
+        System.out.println("Writing : " + fileName);
+        File replicaFile
+            = new File("/opt/hadoop/" + directoryName + "/" + fileName);
+
+        try {
+          Files.copy(is, replicaFile.toPath(),
+              StandardCopyOption.REPLACE_EXISTING);
+        } catch (IOException e) {
+          Throwable cause = e.getCause();
+          replicaJson.addProperty("exception", e.getMessage());
+          if(cause instanceof OzoneChecksumException) {
+            BlockID blockID = block.getKey().getBlockID();
+            String datanodeUUID = replica.getKey().getUuidString();
+            is = getInputStreamWithoutChecksum(replicasWithoutChecksum,
+                datanodeUUID, blockID);
+            Files.copy(is, replicaFile.toPath(),
+                StandardCopyOption.REPLACE_EXISTING);
+          } else if(cause instanceof StatusRuntimeException) {
+            break;
+          }
+        } finally {
+          is.close();
+        }
+        replicasJson.add(replicaJson);
+      }
+      blockJson.add("replicas", replicasJson);
+      blocks.add(blockJson);
+    }
+  }
+
+  private OzoneInputStream getInputStreamWithoutChecksum(
+      Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+          replicasWithoutChecksum, String datanodeUUID, BlockID blockID) {
+    OzoneInputStream is = new OzoneInputStream();
+    for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
+        block : replicasWithoutChecksum.entrySet()) {
+      if(block.getKey().getBlockID().equals(blockID)) {
+        for (Map.Entry<DatanodeDetails, OzoneInputStream>
+            replica : block.getValue().entrySet()) {
+          if(replica.getKey().getUuidString().equals(datanodeUUID)) {
+            is = replica.getValue();
+          }
+        }
+      }
+    }
+    return is;
+  }
+
+  @NotNull
+  private String createDirectory(String volumeName, String bucketName,
+                                 String keyName) {
+    String fileSuffix
+        = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
+    String directoryName = volumeName + "_" + bucketName + "_" + keyName +
+        "_" + fileSuffix;
+    System.out.println("Creating directory : " + directoryName);
+    File dir = new File("/opt/hadoop/" + directoryName);

Review comment:
       Thanks for pointing this out, I added the --outputDir parameter to the `read-replicas` command. I didn't set it to be required as I set the default value to `/opt/hadoop/`.

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);

Review comment:
       Thanks, it was much easier this way, I changed it.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -920,6 +926,64 @@ public OzoneInputStream getKey(
     return getInputStreamWithRetryFunction(keyInfo);
   }
 
+  @Override
+  public Map< OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream> >
+      getKeysEveryReplicas(String volumeName,
+                         String bucketName,
+                         String keyName) throws IOException {
+
+    Map< OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream> > result
+        = new LinkedHashMap<>();
+
+    verifyVolumeName(volumeName);
+    verifyBucketName(bucketName);
+    Preconditions.checkNotNull(keyName);
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .setRefreshPipeline(true)
+        .setSortDatanodesInPipeline(topologyAwareReadEnabled)

Review comment:
       Thanks, I removed the unnecessary call.

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that downloads every replica for all the blocks associated with a
+ * given key. It also generates a manifest file with information about the
+ * downloaded replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads every replica for all the blocks associated with a " +
+        "given key.")
+@MetaInfServices(SubcommandWithParent.class)
+public class ReadReplicas extends KeyHandler implements SubcommandWithParent {
+
+  private ClientProtocol clientProtocol;
+  private ClientProtocol clientProtocolWithoutChecksum;
+
+  @Override
+  public Class<?> getParentType() {
+    return OzoneDebug.class;
+  }
+
+  @Override
+  protected void execute(OzoneClient client, OzoneAddress address)
+      throws IOException, OzoneClientException {
+
+    clientProtocol = client.getObjectStore().getClientProxy();
+
+    OzoneConfiguration configuration = new OzoneConfiguration();
+    OzoneClientConfig clientConfig = configuration
+        .getObject(OzoneClientConfig.class);
+    clientConfig.setChecksumVerify(false);
+    configuration.setFromObject(clientConfig);
+    clientProtocolWithoutChecksum = new RpcClient(configuration, null);

Review comment:
       Thanks, I didn't think it through, that the original configuration can have the checksum verification turned off. Changed the code, so whether the configuration is set to check or not the checksum, I will have two clients, one with checksum verification and one without.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] dombizita commented on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
dombizita commented on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1021081782


   > Hi Zita, thank you for contributing this new debug tool, I would like to request one general thing, and also I have a few comments inline.
   > 
   > I would generally suggest to create String constants and put them at the top of the class, for property names used in the manifest file, it would make it easier to see what properties we expect to have in the manifest if they are at one place. This is a stylistic suggestion, I can accept the current code in this regard, so I let this change for you to decide if it is worth it.
   
   Hi @fapifta, thank you so much for the review. I have taken your advices and changed the mentioned things. I also created the String constants, the code is much clearer this way.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r786618547



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads a given keys every blocks every replica.")

Review comment:
       I think you missed changing this description in the updated patch, could you look into it? Thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on a change in pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on a change in pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#discussion_r784909336



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.debug;
+
+import com.google.gson.*;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.keys.KeyHandler;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.jetbrains.annotations.NotNull;
+import org.kohsuke.MetaInfServices;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * Class that for a given key downloads every blocks every replica and creates
+ * a manifest file with information about the replicas.
+ */
+@CommandLine.Command(name = "read-replicas",
+    description = "Reads a given keys every blocks every replica.")

Review comment:
       Could you modify the description to say `Read every replica for all the blocks associated with a given key.`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JyotinderSingh commented on pull request #2987: HDDS-6128. CLI tool that downloads all the block replicas and creates a manifest file

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on pull request #2987:
URL: https://github.com/apache/ozone/pull/2987#issuecomment-1013206300


   Please add unit tests to verify correct behavior for the newly added `hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java` class.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org