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

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


The following commit(s) were added to refs/heads/master by this push:
     new 2bc5f531e2 HDDS-10541. Replace GSON with Jackson in debug commands 
(#6401)
2bc5f531e2 is described below

commit 2bc5f531e27e1c4dc457ff0859061069f66ed0fc
Author: Arafat2198 <[email protected]>
AuthorDate: Sun Mar 24 11:50:04 2024 +0530

    HDDS-10541. Replace GSON with Jackson in debug commands (#6401)
---
 .../apache/hadoop/ozone/debug/ChunkKeyHandler.java | 71 ++++++++++------------
 .../hadoop/ozone/debug/ContainerChunkInfo.java     | 22 +++++++
 .../apache/hadoop/ozone/debug/ReadReplicas.java    | 52 +++++++---------
 3 files changed, 75 insertions(+), 70 deletions(-)

diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ChunkKeyHandler.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ChunkKeyHandler.java
index 012ab989d5..5c311d49c9 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ChunkKeyHandler.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ChunkKeyHandler.java
@@ -24,11 +24,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.HashSet;
-import com.google.gson.GsonBuilder;
-import com.google.gson.Gson;
-import com.google.gson.JsonObject;
-import com.google.gson.JsonArray;
-import com.google.gson.JsonElement;
+
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.hadoop.hdds.cli.SubcommandWithParent;
 import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
@@ -40,6 +38,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
+import org.apache.hadoop.hdds.server.JsonUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientException;
@@ -80,13 +79,12 @@ public class ChunkKeyHandler extends KeyHandler implements
         XceiverClientManager xceiverClientManager = 
containerOperationClient.getXceiverClientManager()) {
       OzoneManagerProtocol ozoneManagerClient = 
client.getObjectStore().getClientProxy().getOzoneManagerClient();
       address.ensureKeyAddress();
-      JsonElement element;
-      JsonObject result = new JsonObject();
+      ObjectNode result = JsonUtils.createObjectNode(null);
       String volumeName = address.getVolumeName();
       String bucketName = address.getBucketName();
       String keyName = address.getKeyName();
-      List<ContainerProtos.ChunkInfo> tempchunks = null;
-      List<ChunkDetails> chunkDetailsList = new ArrayList<ChunkDetails>();
+      List<ContainerProtos.ChunkInfo> tempchunks;
+      List<ChunkDetails> chunkDetailsList = new ArrayList<>();
       HashSet<String> chunkPaths = new HashSet<>();
       OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
           .setBucketName(bucketName).setKeyName(keyName).build();
@@ -102,7 +100,7 @@ public class ChunkKeyHandler extends KeyHandler implements
       }
       ContainerLayoutVersion containerLayoutVersion = ContainerLayoutVersion
           .getConfiguredVersion(getConf());
-      JsonArray responseArrayList = new JsonArray();
+      ArrayNode responseArrayList = JsonUtils.createArrayNode();
       for (OmKeyLocationInfo keyLocation : locationInfos) {
         ContainerChunkInfo containerChunkInfoVerbose = new 
ContainerChunkInfo();
         ContainerChunkInfo containerChunkInfo = new ContainerChunkInfo();
@@ -128,24 +126,17 @@ public class ChunkKeyHandler extends KeyHandler implements
               keyLocation.getBlockID().getDatanodeBlockIDProtobuf();
           // doing a getBlock on all nodes
           Map<DatanodeDetails, ContainerProtos.GetBlockResponseProto>
-              responses = null;
-          Map<DatanodeDetails, ContainerProtos.ReadContainerResponseProto>
-              readContainerResponses = null;
-          try {
-            responses = 
ContainerProtocolCalls.getBlockFromAllNodes(xceiverClient,
-                datanodeBlockID, keyLocation.getToken());
-            readContainerResponses =
-                containerOperationClient.readContainerFromAllNodes(
-                    keyLocation.getContainerID(), pipeline);
-          } catch (InterruptedException e) {
-            LOG.error("Execution interrupted due to " + e);
-            Thread.currentThread().interrupt();
-          }
-          JsonArray responseFromAllNodes = new JsonArray();
-          for (Map.Entry<DatanodeDetails, 
ContainerProtos.GetBlockResponseProto>
-              entry : responses.entrySet()) {
+              responses =
+              ContainerProtocolCalls.getBlockFromAllNodes(xceiverClient,
+                  keyLocation.getBlockID().getDatanodeBlockIDProtobuf(),
+                  keyLocation.getToken());
+          Map<DatanodeDetails, ContainerProtos.ReadContainerResponseProto> 
readContainerResponses =
+              containerOperationClient.readContainerFromAllNodes(
+                  keyLocation.getContainerID(), pipeline);
+          ArrayNode responseFromAllNodes = JsonUtils.createArrayNode();
+          for (Map.Entry<DatanodeDetails, 
ContainerProtos.GetBlockResponseProto> entry : responses.entrySet()) {
             chunkPaths.clear();
-            JsonObject jsonObj = new JsonObject();
+            ObjectNode jsonObj = JsonUtils.createObjectNode(null);
             if (entry.getValue() == null) {
               LOG.error("Cant execute getBlock on this node");
               continue;
@@ -177,29 +168,29 @@ public class ChunkKeyHandler extends KeyHandler implements
               containerChunkInfoVerbose.setChunkType(blockChunksType);
               containerChunkInfo.setChunkType(blockChunksType);
             }
-            Gson gson = new GsonBuilder().create();
+
             if (isVerbose()) {
-              element = gson.toJsonTree(containerChunkInfoVerbose);
+              jsonObj.set("Locations",
+                  JsonUtils.createObjectNode(containerChunkInfoVerbose));
             } else {
-              element = gson.toJsonTree(containerChunkInfo);
+              jsonObj.set("Locations",
+                  JsonUtils.createObjectNode(containerChunkInfo));
             }
-            jsonObj.addProperty("Datanode-HostName", entry.getKey()
-                .getHostName());
-            jsonObj.addProperty("Datanode-IP", entry.getKey()
-                .getIpAddress());
-            jsonObj.addProperty("Container-ID", containerId);
-            jsonObj.addProperty("Block-ID", keyLocation.getLocalID());
-            jsonObj.add("Locations", element);
+            jsonObj.put("Datanode-HostName", entry.getKey().getHostName());
+            jsonObj.put("Datanode-IP", entry.getKey().getIpAddress());
+            jsonObj.put("Container-ID", containerId);
+            jsonObj.put("Block-ID", keyLocation.getLocalID());
             responseFromAllNodes.add(jsonObj);
           }
           responseArrayList.add(responseFromAllNodes);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
         } finally {
           xceiverClientManager.releaseClientForReadData(xceiverClient, false);
         }
       }
-      result.add("KeyLocations", responseArrayList);
-      Gson gson2 = new GsonBuilder().setPrettyPrinting().create();
-      String prettyJson = gson2.toJson(result);
+      result.set("KeyLocations", responseArrayList);
+      String prettyJson = 
JsonUtils.toJsonStringWithDefaultPrettyPrinter(result);
       System.out.println(prettyJson);
     }
   }
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerChunkInfo.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerChunkInfo.java
index f88e08413d..130c1bca0f 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerChunkInfo.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ContainerChunkInfo.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 public class ContainerChunkInfo {
   private String containerPath;
   private List<ChunkDetails> chunkInfos;
+
   private HashSet<String> files;
   private UUID pipelineID;
   private Pipeline pipeline;
@@ -65,6 +66,27 @@ public class ContainerChunkInfo {
     this.chunkType = chunkType;
   }
 
+  public String getContainerPath() {
+    return containerPath;
+  }
+
+  public List<ChunkDetails> getChunkInfos() {
+    return chunkInfos;
+  }
+
+  public HashSet<String> getFiles() {
+    return files;
+  }
+
+  public UUID getPipelineID() {
+    return pipelineID;
+  }
+
+  public ChunkType getChunkType() {
+    return chunkType;
+  }
+
+
   @Override
   public String toString() {
     return "Container{"
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
index ac9fc7854a..48ed7c74ae 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/ReadReplicas.java
@@ -17,14 +17,11 @@
 
 package org.apache.hadoop.ozone.debug;
 
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonArray;
-import com.google.gson.JsonObject;
 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.server.JsonUtils;
 import org.apache.hadoop.hdds.utils.IOUtils;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientException;
@@ -36,6 +33,8 @@ 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 com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import jakarta.annotation.Nonnull;
 import org.kohsuke.MetaInfServices;
 import picocli.CommandLine;
@@ -129,18 +128,17 @@ public class ReadReplicas extends KeyHandler implements 
SubcommandWithParent {
           replicasWithoutChecksum = noChecksumClient
           .getKeysEveryReplicas(volumeName, bucketName, keyName);
 
-      JsonObject result = new JsonObject();
-      result.addProperty(JSON_PROPERTY_FILE_NAME,
+      ObjectNode result = JsonUtils.createObjectNode(null);
+      result.put(JSON_PROPERTY_FILE_NAME,
           volumeName + "/" + bucketName + "/" + keyName);
-      result.addProperty(JSON_PROPERTY_FILE_SIZE, 
keyInfoDetails.getDataSize());
+      result.put(JSON_PROPERTY_FILE_SIZE, keyInfoDetails.getDataSize());
 
-      JsonArray blocks = new JsonArray();
+      ArrayNode blocks = JsonUtils.createArrayNode();
       downloadReplicasAndCreateManifest(keyName, replicas,
           replicasWithoutChecksum, dir, blocks);
-      result.add(JSON_PROPERTY_FILE_BLOCKS, blocks);
+      result.set(JSON_PROPERTY_FILE_BLOCKS, blocks);
 
-      Gson gson = new GsonBuilder().setPrettyPrinting().create();
-      String prettyJson = gson.toJson(result);
+      String prettyJson = 
JsonUtils.toJsonStringWithDefaultPrettyPrinter(result);
 
       String manifestFileName = keyName + "_manifest";
       System.out.println("Writing manifest file : " + manifestFileName);
@@ -158,25 +156,22 @@ public class ReadReplicas extends KeyHandler implements 
SubcommandWithParent {
       Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>> replicas,
       Map<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
           replicasWithoutChecksum,
-      File dir, JsonArray blocks) throws IOException {
+      File dir, ArrayNode blocks) throws IOException {
     int blockIndex = 0;
 
     for (Map.Entry<OmKeyLocationInfo, Map<DatanodeDetails, OzoneInputStream>>
         block : replicas.entrySet()) {
-      JsonObject blockJson = new JsonObject();
-      JsonArray replicasJson = new JsonArray();
+      ObjectNode blockJson = JsonUtils.createObjectNode(null);
+      ArrayNode replicasJson = JsonUtils.createArrayNode();
 
       blockIndex += 1;
-      blockJson.addProperty(JSON_PROPERTY_BLOCK_INDEX, blockIndex);
+      blockJson.put(JSON_PROPERTY_BLOCK_INDEX, blockIndex);
       OmKeyLocationInfo locationInfo = block.getKey();
-      blockJson.addProperty(JSON_PROPERTY_BLOCK_CONTAINERID,
+      blockJson.put(JSON_PROPERTY_BLOCK_CONTAINERID,
           locationInfo.getContainerID());
-      blockJson.addProperty(JSON_PROPERTY_BLOCK_LOCALID,
-          locationInfo.getLocalID());
-      blockJson.addProperty(JSON_PROPERTY_BLOCK_LENGTH,
-          locationInfo.getLength());
-      blockJson.addProperty(JSON_PROPERTY_BLOCK_OFFSET,
-          locationInfo.getOffset());
+      blockJson.put(JSON_PROPERTY_BLOCK_LOCALID, locationInfo.getLocalID());
+      blockJson.put(JSON_PROPERTY_BLOCK_LENGTH, locationInfo.getLength());
+      blockJson.put(JSON_PROPERTY_BLOCK_OFFSET, locationInfo.getOffset());
 
       BlockID blockID = locationInfo.getBlockID();
       Map<DatanodeDetails, OzoneInputStream> blockReplicasWithoutChecksum =
@@ -186,12 +181,10 @@ public class ReadReplicas extends KeyHandler implements 
SubcommandWithParent {
           replica : block.getValue().entrySet()) {
         DatanodeDetails datanode = replica.getKey();
 
-        JsonObject replicaJson = new JsonObject();
+        ObjectNode replicaJson = JsonUtils.createObjectNode(null);
 
-        replicaJson.addProperty(JSON_PROPERTY_REPLICA_HOSTNAME,
-            datanode.getHostName());
-        replicaJson.addProperty(JSON_PROPERTY_REPLICA_UUID,
-            datanode.getUuidString());
+        replicaJson.put(JSON_PROPERTY_REPLICA_HOSTNAME, 
datanode.getHostName());
+        replicaJson.put(JSON_PROPERTY_REPLICA_UUID, datanode.getUuidString());
 
         String fileName = keyName + "_block" + blockIndex + "_" +
             datanode.getHostName();
@@ -202,8 +195,7 @@ public class ReadReplicas extends KeyHandler implements 
SubcommandWithParent {
           Files.copy(is, path, StandardCopyOption.REPLACE_EXISTING);
         } catch (IOException e) {
           Throwable cause = e.getCause();
-          replicaJson.addProperty(JSON_PROPERTY_REPLICA_EXCEPTION,
-              e.getMessage());
+          replicaJson.put(JSON_PROPERTY_REPLICA_EXCEPTION, e.getMessage());
           if (cause instanceof OzoneChecksumException) {
             try (InputStream is = getReplica(
                 blockReplicasWithoutChecksum, datanode)) {
@@ -213,7 +205,7 @@ public class ReadReplicas extends KeyHandler implements 
SubcommandWithParent {
         }
         replicasJson.add(replicaJson);
       }
-      blockJson.add(JSON_PROPERTY_BLOCK_REPLICAS, replicasJson);
+      blockJson.set(JSON_PROPERTY_BLOCK_REPLICAS, replicasJson);
       blocks.add(blockJson);
 
       IOUtils.close(LOG, blockReplicasWithoutChecksum.values());


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

Reply via email to