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/08/18 08:59:36 UTC

[GitHub] [ozone] Xushaohong opened a new pull request, #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Xushaohong opened a new pull request, #3691:
URL: https://github.com/apache/ozone/pull/3691

   ## What changes were proposed in this pull request?
   
   Supplement CLI for deleted block txn.
   
     1. Integrate the related CLI
     2. Add new CLI to list txns
     3. Support JSON format file as output and input
     4. Improve UT
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-7137
   
   ## How was this patch tested?
   UT
   ![WX20220818-165155](https://user-images.githubusercontent.com/10106574/185354599-6cf3e97a-4220-40f2-b901-aa60716274d3.png)
   ![WX20220818-165230](https://user-images.githubusercontent.com/10106574/185354610-74318d1e-8a6c-4473-be56-7cc7830654fe.png)
   ![WX20220818-165132](https://user-images.githubusercontent.com/10106574/185354615-e2f3078b-b345-4fae-acb5-182714fe6615.png)
   ![WX20220818-164911](https://user-images.githubusercontent.com/10106574/185354619-6441c928-73ed-4a04-95b4-38eb20a7fec9.png)
   
   


-- 
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] errose28 commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
errose28 commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1229062875

   > Moving the DeletedBlocksTransaction to common hdds.proto is an ideal solution,
   but it will lead to the compatibility problem, so I adds the import of ScmServerDatanodeHeartbeatProtocol.proto
   
   I don't think we want this dependency chain. hdds.proto exists to prevent the admin and heartbeat proto from depending on each other. Cross client and rolling upgrade compatability (future work) will be harder to maintain with this dependency. The DeletedBlocksTransaction message is not very complicated. Could we make a different version of that message to be used in the client response placed in ScmAdminProtocol.proto?


-- 
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] ChenSammi commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122586797


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/ResetDeletedBlockRetryCountSubcommand.java:
##########
@@ -41,25 +50,45 @@ public class ResetDeletedBlockRetryCountSubcommand extends ScmSubcommand {
 
   static class TransactionsOption {
     @CommandLine.Option(names = {"-a", "--all"},
-        description = "reset all expired deleted block transaction retry" +
+        description = "Reset all expired deleted block transaction retry" +
             " count from -1 to 0.")
     private boolean resetAll;
 
     @CommandLine.Option(names = {"-l", "--list"},
         split = ",",
-        description = "reset the only given deletedBlock transaction ID" +
+        paramLabel = "txId",
+        description = "Reset the only given deletedBlock transaction ID" +
             " list, e.g 100,101,102.(Separated by ',')")
     private List<Long> txList;
-  }
 
-  @CommandLine.ParentCommand
-  private ScmAdmin parent;
+    @CommandLine.Option(names = {"-i", "--in"},
+        description = "Use file as input, need to be JSON Array format and" +
+            " contains multi \"txID\" key.")

Review Comment:
   Can you add an example of JSON file here, can be just in code comments.



-- 
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] Xushaohong commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1420059414

   > @Xushaohong can you please rebase and resolve conflicts?
   
   @kerneltime  done


-- 
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] ChenSammi commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1455702444

   The last patch LGTM + 1. 
   Thanks @Xushaohong for the contribution and @errose28 @swamirishi @kerneltime for the code review.


-- 
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] swamirishi commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1406926022

   @errose28 Can this be merged?


-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1098115713


##########
hadoop-hdds/interface-client/src/main/proto/hdds.proto:
##########
@@ -451,3 +451,10 @@ message ContainerBalancerConfigurationProto {
     required bool shouldRun = 18;
     optional int32 nextIterationIndex = 19;
 }
+
+message DeletedBlocksTransactionInfo {
+    required int64 txID = 1;

Review Comment:
   @kerneltime I have a little confusion here, this ```DeletedBlocksTransactionInfo``` is returned by the server, and the client now would never construct it as request args. 



-- 
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] Xushaohong commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1219375453

   Moving the ```DeletedBlocksTransaction``` to common ```hdds.proto``` is an ideal solution, 
   but it will lead to the compatibility problem, here adds the import of  ```ScmServerDatanodeHeartbeatProtocol.proto```. 


-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1125869082


##########
hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto:
##########
@@ -488,6 +491,16 @@ message ReplicationManagerReportResponseProto {
   required ReplicationManagerReportProto report = 1;
 }
 
+message GetFailedDeletedBlocksTxnRequestProto {
+  optional string traceID = 1;
+  required int32 count = 2;
+  optional int32 startTxId = 3;

Review Comment:
   Good catch! Updated



-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122981291


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/ResetDeletedBlockRetryCountSubcommand.java:
##########
@@ -41,25 +50,45 @@ public class ResetDeletedBlockRetryCountSubcommand extends ScmSubcommand {
 
   static class TransactionsOption {
     @CommandLine.Option(names = {"-a", "--all"},
-        description = "reset all expired deleted block transaction retry" +
+        description = "Reset all expired deleted block transaction retry" +
             " count from -1 to 0.")
     private boolean resetAll;
 
     @CommandLine.Option(names = {"-l", "--list"},
         split = ",",
-        description = "reset the only given deletedBlock transaction ID" +
+        paramLabel = "txId",
+        description = "Reset the only given deletedBlock transaction ID" +
             " list, e.g 100,101,102.(Separated by ',')")
     private List<Long> txList;
-  }
 
-  @CommandLine.ParentCommand
-  private ScmAdmin parent;
+    @CommandLine.Option(names = {"-i", "--in"},
+        description = "Use file as input, need to be JSON Array format and" +
+            " contains multi \"txID\" key.")

Review Comment:
   
   ![image](https://user-images.githubusercontent.com/10106574/222421214-f773bd60-aa2b-4ebc-a5c2-c51a73459081.png)
   Updated as above



-- 
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] ChenSammi commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122590143


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/ResetDeletedBlockRetryCountSubcommand.java:
##########
@@ -41,25 +50,45 @@ public class ResetDeletedBlockRetryCountSubcommand extends ScmSubcommand {
 
   static class TransactionsOption {
     @CommandLine.Option(names = {"-a", "--all"},
-        description = "reset all expired deleted block transaction retry" +
+        description = "Reset all expired deleted block transaction retry" +
             " count from -1 to 0.")
     private boolean resetAll;
 
     @CommandLine.Option(names = {"-l", "--list"},
         split = ",",
-        description = "reset the only given deletedBlock transaction ID" +
+        paramLabel = "txId",
+        description = "Reset the only given deletedBlock transaction ID" +
             " list, e.g 100,101,102.(Separated by ',')")
     private List<Long> txList;
-  }
 
-  @CommandLine.ParentCommand
-  private ScmAdmin parent;
+    @CommandLine.Option(names = {"-i", "--in"},
+        description = "Use file as input, need to be JSON Array format and" +
+            " contains multi \"txID\" key.")
+    private String fileName;
+  }
 
   @Override
   public void execute(ScmClient client) throws IOException {
     int count;
     if (group.resetAll) {
       count = client.resetDeletedBlockRetryCount(new ArrayList<>());
+    } else if (group.fileName != null) {
+      Gson gson = new Gson();
+      List<Long> txIds;
+      try (InputStream in = new FileInputStream(group.fileName);
+           Reader fileReader = new InputStreamReader(in,
+               StandardCharsets.UTF_8)) {
+        DeletedBlocksTransactionInfoWrapper[] txns = gson.fromJson(fileReader,
+            DeletedBlocksTransactionInfoWrapper[].class);
+        txIds = Arrays.stream(txns).map(DeletedBlocksTransactionInfoWrapper::
+            getTxID).distinct().collect(Collectors.toList());
+        System.out.println("Load txIDS: " + txIds);

Review Comment:
   Please output the start txID, and total number of txIds here.  
   
   



-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122978811


##########
hadoop-hdds/interface-client/src/main/proto/hdds.proto:
##########
@@ -458,4 +458,11 @@ message TransferLeadershipRequestProto {
 }
 
 message TransferLeadershipResponseProto {
+}
+
+message DeletedBlocksTransactionInfo {
+    optional int64 txID = 1;
+    optional int64 containerID = 2;
+    repeated int64 localID = 3;
+    optional int32 count = 4;

Review Comment:
   As I discussed with Ethan above, it is a historical problem that the **DeletedBlocksTransaction**  in `ScmServerDatanodeHeartbeatProtocol.proto` is not in `hdds.proto` as a general PB, here if we want to reuse it we need to import `ScmServerDatanodeHeartbeatProtocol.proto` in `ScmAdminProtocol.proto` which may cause some possible compatibility issue in the future, and hence not a good option.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/GetFailedDeletedBlocksTxnSubcommand.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.admin.scm;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionInfo;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers.DeletedBlocksTransactionInfoWrapper;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import picocli.CommandLine;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Handler of getting expired deleted blocks from SCM side.
+ */
+@CommandLine.Command(
+    name = "ls",
+    description = "Print the failed DeletedBlocksTxn (retry count = -1)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class GetFailedDeletedBlocksTxnSubcommand extends ScmSubcommand {
+
+  @CommandLine.ArgGroup(multiplicity = "1")
+  private TransactionsOption group;
+
+  static class TransactionsOption {
+    @CommandLine.Option(names = {"-a", "--all"},
+        description = "Get all the failed transactions.")
+    private boolean getAll;
+
+    @CommandLine.Option(names = {"-n", "--num"},

Review Comment:
   done



-- 
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] kerneltime commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1418562222

   @Xushaohong can you please rebase and resolve conflicts?


-- 
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] Xushaohong commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1230197700

   > > Moving the DeletedBlocksTransaction to common hdds.proto is an ideal solution,
   > > but it will lead to the compatibility problem, so I adds the import of ScmServerDatanodeHeartbeatProtocol.proto
   > 
   > I don't think we want this dependency chain. hdds.proto exists to prevent the admin and heartbeat proto from depending on each other. Cross client and rolling upgrade compatability (future work) will be harder to maintain with this dependency. The DeletedBlocksTransaction message is not very complicated. Could we make a different version of that message to be used in the client response placed in ScmAdminProtocol.proto?
   
   The problem mentioned is right,  as the migration of proto would bring the compatibility issue,  I just considered the reuse form. I have now added the `DeletedBlocksTransactionInfo`


-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1098249038


##########
hadoop-hdds/interface-client/src/main/proto/hdds.proto:
##########
@@ -451,3 +451,10 @@ message ContainerBalancerConfigurationProto {
     required bool shouldRun = 18;
     optional int32 nextIterationIndex = 19;
 }
+
+message DeletedBlocksTransactionInfo {
+    required int64 txID = 1;

Review Comment:
   Got it, I have updated it. @kerneltime 



-- 
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] ChenSammi commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122583131


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -126,18 +128,27 @@ public DeletedBlockLogImpl(ConfigurationSource conf,
   }
 
   @Override
-  public List<DeletedBlocksTransaction> getFailedTransactions()
+  public List<DeletedBlocksTransaction> getFailedTransactions(int count)
       throws IOException {
     lock.lock();
     try {
       final List<DeletedBlocksTransaction> failedTXs = Lists.newArrayList();
       try (TableIterator<Long,
           ? extends Table.KeyValue<Long, DeletedBlocksTransaction>> iter =
                deletedBlockLogStateManager.getReadOnlyIterator()) {
-        while (iter.hasNext()) {
-          DeletedBlocksTransaction delTX = iter.next().getValue();
-          if (delTX.getCount() == -1) {
-            failedTXs.add(delTX);
+        if (count <= 0) {

Review Comment:
   We should use LIST_ALL_FAILED_TRANSACTIONS check here.



-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122984908


##########
hadoop-hdds/common/pom.xml:
##########
@@ -226,6 +226,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.apache.ozone</groupId>
       <artifactId>hdds-interface-admin</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-interface-server</artifactId>
+    </dependency>

Review Comment:
   Originally it is for DeletedBlocksTransaction in **DeletedBlocksTransactionInfoWrapper**,  I have now moved the wrapper class  to a better place.



##########
hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto:
##########
@@ -488,6 +491,15 @@ message ReplicationManagerReportResponseProto {
   required ReplicationManagerReportProto report = 1;
 }
 
+message GetFailedDeletedBlocksTxnRequestProto {
+  optional string traceID = 1;
+  required int32 count = 2;

Review Comment:
   Updated



-- 
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] ChenSammi commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122588629


##########
hadoop-hdds/common/pom.xml:
##########
@@ -226,6 +226,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.apache.ozone</groupId>
       <artifactId>hdds-interface-admin</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-interface-server</artifactId>
+    </dependency>

Review Comment:
   @Xushaohong , which class reference requires this new dependency? 



-- 
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] ChenSammi commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122589368


##########
hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto:
##########
@@ -488,6 +491,15 @@ message ReplicationManagerReportResponseProto {
   required ReplicationManagerReportProto report = 1;
 }
 
+message GetFailedDeletedBlocksTxnRequestProto {
+  optional string traceID = 1;
+  required int32 count = 2;

Review Comment:
   In batch fetch support, we need a start ID and batch size, so user can do the iterate using different start ID and same batch size.  So we should add a start TxnID here. 



-- 
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] kerneltime commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1420086500

   @Xushaohong this looks like a pretty straightforward change, please address the minor nits.


-- 
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] kerneltime commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1098101481


##########
hadoop-hdds/interface-client/src/main/proto/hdds.proto:
##########
@@ -451,3 +451,10 @@ message ContainerBalancerConfigurationProto {
     required bool shouldRun = 18;
     optional int32 nextIterationIndex = 19;
 }
+
+message DeletedBlocksTransactionInfo {
+    required int64 txID = 1;

Review Comment:
   In general, we should let the proto layer mark args as optional and deal with it on the server side? I would mark most of these args as optional. cc @errose28 



-- 
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] Xushaohong commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1222110096

   @adoroszlai @ChenSammi PTAL~


-- 
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] Xushaohong commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1225607563

   > Hey @Xushaohong , could you past some example output of this command? For both console output and file output.
   
   @ChenSammi  Sure,  I screenshotted three pics, and the result of `reset` is not actually worked due to the non-instant flush as you know
   
   ![show3](https://user-images.githubusercontent.com/10106574/186409840-1de7d0ae-5d4b-45ed-8559-6ad8e757e9e5.png)
   ![show2](https://user-images.githubusercontent.com/10106574/186409848-4cd15d7d-be9c-40c9-9580-0c5c43d0468c.png)
   ![show1](https://user-images.githubusercontent.com/10106574/186409851-12e9d885-ec3c-4e67-8fa9-36f3a426c3a8.png)
   
   


-- 
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] Xushaohong commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1231078771

   > Thanks for the proto update @Xushaohong. I probably won't have time to review the rest of this PR but that part looks good. Just a question: I see the new proto is in hdds.proto. Would it make more sense to be in SCMAdminProtocol.proto since it is used as the response to an SCM admin command?
   
   Hi @errose28, hdds.proto is a common client proto,  my motivation for putting the definition here has two reasons:
   1. The `DeletedBlocksTransactionInfo` could be used for the future client request, it is a prospective placement. Refer to the proto of `ContainerInfoProto` and `Pipeline`.
   2. I think the placement in  ScmServerDatanodeHeartbeatProtocol.proto is a historical problem. Originally it is only considered to be used in HB, not in client queries. Such common proto shall be unified and could help reduce confusion for the user and reader. So I wonder if someday we change the reference of this TXN proto from the ScmServerDatanodeHeartbeatProtocol.proto to the hdds.proto. And we could deprecate the old reference.
   
   Pls feel free to correct me if i am wrong~
   


-- 
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] kerneltime commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1098101828


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -126,18 +126,27 @@ public DeletedBlockLogImpl(ConfigurationSource conf,
   }
 
   @Override
-  public List<DeletedBlocksTransaction> getFailedTransactions()
+  public List<DeletedBlocksTransaction> getFailedTransactions(int count)
       throws IOException {
     lock.lock();
     try {
       final List<DeletedBlocksTransaction> failedTXs = Lists.newArrayList();
       try (TableIterator<Long,
           ? extends Table.KeyValue<Long, DeletedBlocksTransaction>> iter =
                deletedBlockLogStateManager.getReadOnlyIterator()) {
-        while (iter.hasNext()) {
-          DeletedBlocksTransaction delTX = iter.next().getValue();
-          if (delTX.getCount() == -1) {
-            failedTXs.add(delTX);
+        if (count < 0) {

Review Comment:
   ```suggestion
           if (count <= 0) {
   ```



-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122967241


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -126,18 +128,27 @@ public DeletedBlockLogImpl(ConfigurationSource conf,
   }
 
   @Override
-  public List<DeletedBlocksTransaction> getFailedTransactions()
+  public List<DeletedBlocksTransaction> getFailedTransactions(int count)
       throws IOException {
     lock.lock();
     try {
       final List<DeletedBlocksTransaction> failedTXs = Lists.newArrayList();
       try (TableIterator<Long,
           ? extends Table.KeyValue<Long, DeletedBlocksTransaction>> iter =
                deletedBlockLogStateManager.getReadOnlyIterator()) {
-        while (iter.hasNext()) {
-          DeletedBlocksTransaction delTX = iter.next().getValue();
-          if (delTX.getCount() == -1) {
-            failedTXs.add(delTX);
+        if (count <= 0) {

Review Comment:
   Updated



-- 
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] ChenSammi commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1222454979

   Hey @Xushaohong , could you past some example output of this command? For both console output and file output.


-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1098115713


##########
hadoop-hdds/interface-client/src/main/proto/hdds.proto:
##########
@@ -451,3 +451,10 @@ message ContainerBalancerConfigurationProto {
     required bool shouldRun = 18;
     optional int32 nextIterationIndex = 19;
 }
+
+message DeletedBlocksTransactionInfo {
+    required int64 txID = 1;

Review Comment:
   @kerneltime I have a little confusion here, this ```DeletedBlocksTransactionInfo``` is returned by the server, and the client now would never construct it as request args. 



-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1098388086


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -126,18 +126,27 @@ public DeletedBlockLogImpl(ConfigurationSource conf,
   }
 
   @Override
-  public List<DeletedBlocksTransaction> getFailedTransactions()
+  public List<DeletedBlocksTransaction> getFailedTransactions(int count)
       throws IOException {
     lock.lock();
     try {
       final List<DeletedBlocksTransaction> failedTXs = Lists.newArrayList();
       try (TableIterator<Long,
           ? extends Table.KeyValue<Long, DeletedBlocksTransaction>> iter =
                deletedBlockLogStateManager.getReadOnlyIterator()) {
-        while (iter.hasNext()) {
-          DeletedBlocksTransaction delTX = iter.next().getValue();
-          if (delTX.getCount() == -1) {
-            failedTXs.add(delTX);
+        if (count < 0) {

Review Comment:
   done



-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1119535267


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -126,18 +128,27 @@ public DeletedBlockLogImpl(ConfigurationSource conf,
   }
 
   @Override
-  public List<DeletedBlocksTransaction> getFailedTransactions()
+  public List<DeletedBlocksTransaction> getFailedTransactions(int count)
       throws IOException {
     lock.lock();
     try {
       final List<DeletedBlocksTransaction> failedTXs = Lists.newArrayList();
       try (TableIterator<Long,
           ? extends Table.KeyValue<Long, DeletedBlocksTransaction>> iter =
                deletedBlockLogStateManager.getReadOnlyIterator()) {
-        while (iter.hasNext()) {
-          DeletedBlocksTransaction delTX = iter.next().getValue();
-          if (delTX.getCount() == -1) {
-            failedTXs.add(delTX);
+        if (count <= 0) {
+          while (iter.hasNext()) {
+            DeletedBlocksTransaction delTX = iter.next().getValue();
+            if (delTX.getCount() == -1) {
+              failedTXs.add(delTX);
+            }
+          }
+        } else {
+          while (iter.hasNext() && failedTXs.size() < count) {
+            DeletedBlocksTransaction delTX = iter.next().getValue();
+            if (delTX.getCount() == -1) {

Review Comment:
   Oops, this is the special value of DeletedBlocksTransaction, not for the clear instant description.       
   See `DeletedBlocksTransaction.Builder builder = block.toBuilder().setCount(-1);`



-- 
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] kerneltime commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1119106456


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -126,18 +128,27 @@ public DeletedBlockLogImpl(ConfigurationSource conf,
   }
 
   @Override
-  public List<DeletedBlocksTransaction> getFailedTransactions()
+  public List<DeletedBlocksTransaction> getFailedTransactions(int count)
       throws IOException {
     lock.lock();
     try {
       final List<DeletedBlocksTransaction> failedTXs = Lists.newArrayList();
       try (TableIterator<Long,
           ? extends Table.KeyValue<Long, DeletedBlocksTransaction>> iter =
                deletedBlockLogStateManager.getReadOnlyIterator()) {
-        while (iter.hasNext()) {
-          DeletedBlocksTransaction delTX = iter.next().getValue();
-          if (delTX.getCount() == -1) {
-            failedTXs.add(delTX);
+        if (count <= 0) {
+          while (iter.hasNext()) {
+            DeletedBlocksTransaction delTX = iter.next().getValue();
+            if (delTX.getCount() == -1) {
+              failedTXs.add(delTX);
+            }
+          }
+        } else {
+          while (iter.hasNext() && failedTXs.size() < count) {
+            DeletedBlocksTransaction delTX = iter.next().getValue();
+            if (delTX.getCount() == -1) {

Review Comment:
   Should this be switched as well?
   ```suggestion
               if (delTX.getCount() == LIST_ALL_FAILED_TRANSACTIONS) {
   ```



-- 
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] Xushaohong commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1123013901


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/ResetDeletedBlockRetryCountSubcommand.java:
##########
@@ -41,25 +50,45 @@ public class ResetDeletedBlockRetryCountSubcommand extends ScmSubcommand {
 
   static class TransactionsOption {
     @CommandLine.Option(names = {"-a", "--all"},
-        description = "reset all expired deleted block transaction retry" +
+        description = "Reset all expired deleted block transaction retry" +
             " count from -1 to 0.")
     private boolean resetAll;
 
     @CommandLine.Option(names = {"-l", "--list"},
         split = ",",
-        description = "reset the only given deletedBlock transaction ID" +
+        paramLabel = "txId",
+        description = "Reset the only given deletedBlock transaction ID" +
             " list, e.g 100,101,102.(Separated by ',')")
     private List<Long> txList;
-  }
 
-  @CommandLine.ParentCommand
-  private ScmAdmin parent;
+    @CommandLine.Option(names = {"-i", "--in"},
+        description = "Use file as input, need to be JSON Array format and" +
+            " contains multi \"txID\" key.")
+    private String fileName;
+  }
 
   @Override
   public void execute(ScmClient client) throws IOException {
     int count;
     if (group.resetAll) {
       count = client.resetDeletedBlockRetryCount(new ArrayList<>());
+    } else if (group.fileName != null) {
+      Gson gson = new Gson();
+      List<Long> txIds;
+      try (InputStream in = new FileInputStream(group.fileName);
+           Reader fileReader = new InputStreamReader(in,
+               StandardCharsets.UTF_8)) {
+        DeletedBlocksTransactionInfoWrapper[] txns = gson.fromJson(fileReader,
+            DeletedBlocksTransactionInfoWrapper[].class);
+        txIds = Arrays.stream(txns).map(DeletedBlocksTransactionInfoWrapper::
+            getTxID).distinct().collect(Collectors.toList());
+        System.out.println("Load txIDS: " + txIds);

Review Comment:
   ![image](https://user-images.githubusercontent.com/10106574/222427449-30a53002-aa60-4e67-a60c-21a540ef96e3.png)
   



-- 
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] ChenSammi commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1125842902


##########
hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto:
##########
@@ -488,6 +491,16 @@ message ReplicationManagerReportResponseProto {
   required ReplicationManagerReportProto report = 1;
 }
 
+message GetFailedDeletedBlocksTxnRequestProto {
+  optional string traceID = 1;
+  required int32 count = 2;
+  optional int32 startTxId = 3;

Review Comment:
   Tx ID is int64, and long in java code. 



-- 
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] errose28 commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
errose28 commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1234811211

   > I think the placement in ScmServerDatanodeHeartbeatProtocol.proto is a historical problem. Originally it is only considered to be used in HB, not in client queries. Such common proto shall be unified and could help reduce confusion for the user and reader. So I wonder if someday we change the reference of this TXN proto from the ScmServerDatanodeHeartbeatProtocol.proto to the hdds.proto. And we could deprecate the old reference.
   
   I think this is a good point. If we put the new message in ScmAdminProtocol.proto we will have the two messages forever since we need to support all old clients. If we put the new message in hdds.proto then later we could switch datanode and SCM communication to use the new message in hdds.proto instead of the old one in ScmServerDatanodeHeartbeatProtocol.proto  since we do not support rolling upgrades yet.


-- 
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] errose28 commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by GitBox <gi...@apache.org>.
errose28 commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1230713418

   Thanks for the proto update @Xushaohong. I probably won't have time to review the rest of this PR but that part looks good.


-- 
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] kerneltime commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1098102621


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java:
##########
@@ -191,7 +200,7 @@ public int resetCount(List<Long> txIDs) throws IOException, TimeoutException {
     lock.lock();
     try {
       if (txIDs == null || txIDs.isEmpty()) {
-        txIDs = getFailedTransactions().stream()
+        txIDs = getFailedTransactions(-1).stream()

Review Comment:
   I would avoid magic numbers, you can define some constants for these which make it obvious what is intended Example: `LIST_ALL_FAILED_TRANSACTIONS`



-- 
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] ChenSammi commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122584631


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/GetFailedDeletedBlocksTxnSubcommand.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.admin.scm;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionInfo;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers.DeletedBlocksTransactionInfoWrapper;
+import org.apache.hadoop.hdds.server.JsonUtils;
+import picocli.CommandLine;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Handler of getting expired deleted blocks from SCM side.
+ */
+@CommandLine.Command(
+    name = "ls",
+    description = "Print the failed DeletedBlocksTxn (retry count = -1)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class GetFailedDeletedBlocksTxnSubcommand extends ScmSubcommand {
+
+  @CommandLine.ArgGroup(multiplicity = "1")
+  private TransactionsOption group;
+
+  static class TransactionsOption {
+    @CommandLine.Option(names = {"-a", "--all"},
+        description = "Get all the failed transactions.")
+    private boolean getAll;
+
+    @CommandLine.Option(names = {"-n", "--num"},

Review Comment:
   "-n", "--num" -> "-c", "--count"



-- 
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] ChenSammi commented on a diff in pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi commented on code in PR #3691:
URL: https://github.com/apache/ozone/pull/3691#discussion_r1122584063


##########
hadoop-hdds/interface-client/src/main/proto/hdds.proto:
##########
@@ -458,4 +458,11 @@ message TransferLeadershipRequestProto {
 }
 
 message TransferLeadershipResponseProto {
+}
+
+message DeletedBlocksTransactionInfo {
+    optional int64 txID = 1;
+    optional int64 containerID = 2;
+    repeated int64 localID = 3;
+    optional int32 count = 4;

Review Comment:
   @Xushaohong , there is a DeletedBlocksTransaction defined in StorageContainerDatanodeProtocolProtos, which has the same fields as this one. Could we use that directly? 



-- 
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] Xushaohong commented on pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "Xushaohong (via GitHub)" <gi...@apache.org>.
Xushaohong commented on PR #3691:
URL: https://github.com/apache/ozone/pull/3691#issuecomment-1451784498

   ![image](https://user-images.githubusercontent.com/10106574/222427721-b522dee8-8355-43ea-aa3e-89b33aa6b1a8.png)
   ![image](https://user-images.githubusercontent.com/10106574/222427773-87047543-3b5b-4eff-8c9e-5a3ee57451b5.png)
   The latest usage is as above


-- 
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] ChenSammi merged pull request #3691: HDDS-7137. Add CLI for Getting the failed deleted block txn

Posted by "ChenSammi (via GitHub)" <gi...@apache.org>.
ChenSammi merged PR #3691:
URL: https://github.com/apache/ozone/pull/3691


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