You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2022/06/21 15:29:01 UTC

[hbase-operator-tools] branch master updated: HBASE-27119 [HBCK2] Some commands are broken after HBASE-24587 (#107)

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

wchevreuil pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase-operator-tools.git


The following commit(s) were added to refs/heads/master by this push:
     new 7b21879  HBASE-27119 [HBCK2] Some commands are broken after HBASE-24587 (#107)
7b21879 is described below

commit 7b218791dd7b68278152e2507a667b82ded431f4
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Tue Jun 21 16:28:56 2022 +0100

    HBASE-27119 [HBCK2] Some commands are broken after HBASE-24587 (#107)
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
    Reviewed-by: clarax <cl...@gmail.com>
---
 hbase-hbck2/README.md                              |   1 -
 .../main/java/org/apache/hbase/FileSystemFsck.java |  23 +-
 .../src/main/java/org/apache/hbase/HBCK2.java      |  53 ++++-
 .../java/org/apache/hbase/ReplicationFsck.java     |  26 +--
 .../apache/hbase/TestHBCKCommandLineParsing.java   | 259 ++++++++++++++++++++-
 5 files changed, 299 insertions(+), 63 deletions(-)

diff --git a/hbase-hbck2/README.md b/hbase-hbck2/README.md
index 14ad004..84009aa 100644
--- a/hbase-hbck2/README.md
+++ b/hbase-hbck2/README.md
@@ -102,7 +102,6 @@ Options:
 Command:
  addFsRegionsMissingInMeta <NAMESPACE|NAMESPACE:TABLENAME>...|-i <INPUT_FILE>...
    Options:
-    -d,--force_disable aborts fix for table if disable fails.
     -i,--inputFiles  take one or more input files of namespace of table names
    To be used when regions missing from hbase:meta but directories
    are present still in HDFS. Can happen if user has run _hbck1_
diff --git a/hbase-hbck2/src/main/java/org/apache/hbase/FileSystemFsck.java b/hbase-hbck2/src/main/java/org/apache/hbase/FileSystemFsck.java
index d062a1d..f97995c 100644
--- a/hbase-hbck2/src/main/java/org/apache/hbase/FileSystemFsck.java
+++ b/hbase-hbck2/src/main/java/org/apache/hbase/FileSystemFsck.java
@@ -20,6 +20,7 @@ package org.apache.hbase;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
@@ -30,12 +31,6 @@ import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hbase.hbck1.HBaseFsck;
 import org.apache.hbase.hbck1.HFileCorruptionChecker;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
 
 /**
  * Checks and repairs for hbase filesystem.
@@ -57,20 +52,7 @@ public class FileSystemFsck implements Closeable {
     // Nothing to do.
   }
 
-  int fsck(String[] args) throws IOException {
-    Options options = new Options();
-    Option fixOption = Option.builder("f").longOpt("fix").build();
-    options.addOption(fixOption);
-    // Parse command-line.
-    CommandLineParser parser = new DefaultParser();
-    CommandLine commandLine;
-    try {
-      commandLine = parser.parse(options, args, false);
-    } catch(ParseException e) {
-      HBCK2.showErrorMessage(e.getMessage());
-      return -1;
-    }
-    boolean fix = commandLine.hasOption(fixOption.getOpt());
+  int fsck(List<String> tables, boolean fix) throws IOException {
     // Before we start make sure of the version file.
     if (fix && !HBaseFsck.versionFileExists(this.fs, this.rootDir)) {
       HBaseFsck.versionFileCreate(this.configuration, this.fs, this.rootDir);
@@ -80,7 +62,6 @@ public class FileSystemFsck implements Closeable {
       // Check hfiles.
       HFileCorruptionChecker hfcc = hbaseFsck.createHFileCorruptionChecker(fix);
       hbaseFsck.setHFileCorruptionChecker(hfcc);
-      Collection<String> tables = commandLine.getArgList();
       Collection<Path> tableDirs = tables.isEmpty()?
           FSUtils.getTableDirs(this.fs, this.rootDir):
           tables.stream().map(t -> CommonFSUtils.getTableDir(this.rootDir, TableName.valueOf(t))).
diff --git a/hbase-hbck2/src/main/java/org/apache/hbase/HBCK2.java b/hbase-hbck2/src/main/java/org/apache/hbase/HBCK2.java
index a0afc6c..a4e180b 100644
--- a/hbase-hbck2/src/main/java/org/apache/hbase/HBCK2.java
+++ b/hbase-hbck2/src/main/java/org/apache/hbase/HBCK2.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.filter.RowFilter;
 import org.apache.hadoop.hbase.filter.SubstringComparator;
 import org.apache.hadoop.hbase.master.RegionState;
 
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.core.config.Configurator;
 import org.slf4j.Logger;
@@ -559,7 +560,6 @@ public class HBCK2 extends Configured implements org.apache.hadoop.util.Tool {
     writer.println(" " + ADD_MISSING_REGIONS_IN_META_FOR_TABLES + " [<NAMESPACE|"
         + "NAMESPACE:TABLENAME>...|-i <INPUTFILES>...]");
     writer.println("   Options:");
-    writer.println("    -d,--force_disable aborts fix for table if disable fails.");
     writer.println("    -i,--inputFiles  take one or more files of namespace or table names");
     writer.println("   To be used when regions missing from hbase:meta but directories");
     writer.println("   are present still in HDFS. Can happen if user has run _hbck1_");
@@ -1070,8 +1070,10 @@ public class HBCK2 extends Configured implements org.apache.hadoop.util.Tool {
         try (ClusterConnection connection = connect()) {
           checkHBCKSupport(connection, command);
           try (FileSystemFsck fsfsck = new FileSystemFsck(getConf())) {
-            return fsfsck.fsck(getInputList(purgeFirst(commands))
-                    .toArray(new String[0])) != 0? EXIT_FAILURE : EXIT_SUCCESS;
+            Pair<CommandLine, List<String>> pair =
+              parseCommandWithFixAndInputOptions(purgeFirst(commands));
+            return fsfsck.fsck(pair.getSecond(),
+              pair.getFirst().hasOption("f"))!= 0? EXIT_FAILURE : EXIT_SUCCESS;
           }
         }
 
@@ -1079,8 +1081,10 @@ public class HBCK2 extends Configured implements org.apache.hadoop.util.Tool {
         try (ClusterConnection connection = connect()) {
           checkHBCKSupport(connection, command, "2.1.1", "2.2.0", "3.0.0");
           try (ReplicationFsck replicationFsck = new ReplicationFsck(getConf())) {
-            return replicationFsck.fsck(getInputList(purgeFirst(commands))
-                    .toArray(new String[0])) != 0? EXIT_FAILURE : EXIT_SUCCESS;
+            Pair<CommandLine, List<String>> pair =
+              parseCommandWithFixAndInputOptions(purgeFirst(commands));
+            return replicationFsck.fsck(pair.getSecond(),
+              pair.getFirst().hasOption("f")) != 0? EXIT_FAILURE : EXIT_SUCCESS;
           }
         }
 
@@ -1331,18 +1335,43 @@ public class HBCK2 extends Configured implements org.apache.hadoop.util.Tool {
    * @return the list of input from arguments or parsed from input files
    */
   private List<String> getInputList(String[] args) throws IOException {
+    CommandLine commandLine = parseCommandWithInputList(args, null);
+    if (commandLine == null) {
+      return null;
+    }
+    return getFromArgsOrFiles(commandLine.getArgList(),
+            commandLine.hasOption("i"));
+  }
+
+  private CommandLine parseCommandWithInputList(String[] args, Options options) {
     if (args == null) {
       return null;
     }
-    Options options = new Options();
+    if (options == null) {
+      options = new Options();
+    }
     Option inputFile = Option.builder("i").longOpt("inputFiles").build();
     options.addOption(inputFile);
-    CommandLine commandLine = getCommandLine(args, options);
-    if (commandLine == null) {
-      return null;
-    }
-    return getFromArgsOrFiles(commandLine.getArgList(),
-            commandLine.hasOption(inputFile.getOpt()));
+    return getCommandLine(args, options);
+  }
+
+  private Pair<CommandLine,List<String>> parseAndGetCommandLineWithInputOption(String[] args,
+    Options options) throws IOException {
+    CommandLine commandLine = parseCommandWithInputList(args, options);
+    List<String> params = getFromArgsOrFiles(commandLine.getArgList(),
+      commandLine.hasOption("i"));
+    return Pair.newPair(commandLine, params);
+  }
+
+  private Pair<CommandLine,List<String>> parseCommandWithFixAndInputOptions(String[] args)
+    throws IOException {
+    Options options = new Options();
+    Option fixOption = Option.builder("f").longOpt("fix").build();
+    options.addOption(fixOption);
+    CommandLine commandLine = parseCommandWithInputList(args, options);
+    List<String> params = getFromArgsOrFiles(commandLine.getArgList(),
+      commandLine.hasOption("i"));
+    return Pair.newPair(commandLine, params);
   }
 
   /**
diff --git a/hbase-hbck2/src/main/java/org/apache/hbase/ReplicationFsck.java b/hbase-hbck2/src/main/java/org/apache/hbase/ReplicationFsck.java
index 12965a6..15baca1 100644
--- a/hbase-hbck2/src/main/java/org/apache/hbase/ReplicationFsck.java
+++ b/hbase-hbck2/src/main/java/org/apache/hbase/ReplicationFsck.java
@@ -19,17 +19,11 @@ package org.apache.hbase;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.Collection;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hbase.hbck1.HBaseFsck;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
 
 /**
  * Checks and repairs for hbase replication.
@@ -37,7 +31,7 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
 public class ReplicationFsck implements Closeable {
   private final Configuration configuration;
 
-  ReplicationFsck(Configuration conf) throws IOException {
+  ReplicationFsck(Configuration conf) {
     this.configuration = conf;
   }
 
@@ -46,24 +40,10 @@ public class ReplicationFsck implements Closeable {
     // Nothing to do.
   }
 
-  int fsck(String[] args) throws IOException {
-    Options options = new Options();
-    Option fixOption = Option.builder("f").longOpt("fix").build();
-    options.addOption(fixOption);
-    // Parse command-line.
-    CommandLineParser parser = new DefaultParser();
-    CommandLine commandLine;
-    try {
-      commandLine = parser.parse(options, args, false);
-    } catch(ParseException e) {
-      HBCK2.showErrorMessage(e.getMessage());
-      return -1;
-    }
-    boolean fix = commandLine.hasOption(fixOption.getOpt());
+  int fsck(List<String> tables, boolean fix) throws IOException {
     try (HBaseFsck hbaseFsck = new HBaseFsck(this.configuration)) {
       hbaseFsck.setFixReplication(fix);
       hbaseFsck.checkAndFixReplication();
-      Collection<String> tables = commandLine.getArgList();
       if (tables != null && !tables.isEmpty()) {
         // Below needs connection to be up; uses admin.
         hbaseFsck.connect();
diff --git a/hbase-hbck2/src/test/java/org/apache/hbase/TestHBCKCommandLineParsing.java b/hbase-hbck2/src/test/java/org/apache/hbase/TestHBCKCommandLineParsing.java
index 8cd2976..6dd3b91 100644
--- a/hbase-hbck2/src/test/java/org/apache/hbase/TestHBCKCommandLineParsing.java
+++ b/hbase-hbck2/src/test/java/org/apache/hbase/TestHBCKCommandLineParsing.java
@@ -18,12 +18,17 @@
 package org.apache.hbase;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.BufferedWriter;
 import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStreamWriter;
 import java.io.PrintStream;
 import java.util.Properties;
 
@@ -65,7 +70,7 @@ public class TestHBCKCommandLineParsing {
     assertTrue(output, output.startsWith("usage: HBCK2"));
 
     // Passing -h/--help does the same
-    output = retrieveOptionOutput("-h");
+    output = retrieveOptionOutput(new String[]{"-h"});
     assertTrue(output, output.startsWith("usage: HBCK2"));
   }
 
@@ -75,7 +80,7 @@ public class TestHBCKCommandLineParsing {
     String[] cmds = new String[]{"setTableState", "bypass", "scheduleRecoveries"};
     String output;
     for(String cmd: cmds){
-      output = retrieveOptionOutput(cmd);
+      output = retrieveOptionOutput(new String[]{cmd});
       assertTrue(output, output.startsWith("ERROR: "));
       assertTrue(output, output.contains("FOR USAGE, use the -h or --help option"));
     }
@@ -108,17 +113,259 @@ public class TestHBCKCommandLineParsing {
     properties.load(inputStream);
     String expectedVersionOutput = properties.getProperty("version");
     // Get hbck version option output.
-    String actualVersionOutput = retrieveOptionOutput("-v").trim();
+    String actualVersionOutput = retrieveOptionOutput(new String[]{"-v"}).trim();
     assertEquals(expectedVersionOutput, actualVersionOutput);
   }
 
-  private String retrieveOptionOutput(String option) throws IOException {
+  @Test
+  public void testReplicationNoOption() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"replication"});
+    assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+  }
+
+  @Test
+  public void testReplicationFixShortOption() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"replication",  "-f"});
+    assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+  }
+
+  @Test
+  public void testReplicationFixShortOptionTable() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"replication",  "-f", "table"});
+    assertTrue(
+      output.contains("ERROR: No replication barrier(s) on table: table\n"));
+    assertFalse(output.contains("ERROR: Unrecognized option: --fix"));
+  }
+
+  @Test
+  public void testReplicationFixShortOptionInputFile() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "replication", "-f", "-i", input.getPath() });
+      assertTrue(
+        output.contains("ERROR: No replication barrier(s) on table: table\n"));
+      assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+      assertFalse(output.contains("ERROR: Unrecognized option: -i"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testReplicationFixLongOption() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"replication",  "--fix"});
+    assertFalse(output.contains("ERROR: Unrecognized option: --fix"));
+  }
+
+  @Test
+  public void testReplicationFixLongOptionTable() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"replication",  "--fix", "table"});
+    assertTrue(output.contains("ERROR: No replication barrier(s) on table: table\n"));
+    assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+  }
+
+  @Test
+  public void testReplicationFixLongOptionInputFile() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "replication", "--fix", "-i", input.getPath() });
+      assertTrue(
+        output.contains("ERROR: No replication barrier(s) on table: table\n"));
+      assertFalse(output.contains("ERROR: Unrecognized option: --fix"));
+      assertFalse(output.contains("ERROR: Unrecognized option: -i"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testReplicationFixShortOptionInputFileLong() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "replication", "-f", "--inputFiles", input.getPath() });
+      assertTrue(
+        output.contains("ERROR: No replication barrier(s) on table: table\n"));
+      assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+      assertFalse(output.contains("ERROR: Unrecognized option: --inputFiles"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testReplicationFixLongOptionInputFileLong() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "replication", "--fix", "--inputFiles",
+          input.getPath() });
+      assertTrue(
+        output.contains("ERROR: No replication barrier(s) on table: table\n"));
+      assertFalse(output.contains("ERROR: Unrecognized option: --fix"));
+      assertFalse(output.contains("ERROR: Unrecognized option: --inoutFiles"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testReplicationInputFileLong() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "replication", "--inputFiles", input.getPath() });
+      assertFalse(output.contains("ERROR: Unrecognized option: --inputFiles"));
+
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testReplicationInputFile() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "replication", "-i", input.getPath() });
+      assertFalse(output.contains("ERROR: Unrecognized option: -i"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  private File createInputFile() throws Exception {
+    File f = new File("input");
+    try(BufferedWriter writer = new BufferedWriter(
+      new OutputStreamWriter(new FileOutputStream(f)))) {
+      writer.write("table");
+      writer.flush();
+    }
+    return f;
+  }
+
+  @Test
+  public void testFilesystemFixShortOption() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"filesystem",  "-f"});
+    assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+  }
+
+  @Test
+  public void testFilesystemFixShortOptionTable() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"filesystem",  "-f", "table"});
+    assertFalse(output.contains("ERROR: Unrecognized option: --fix"));
+  }
+
+  @Test
+  public void testFilesystemFixShortOptionInputFile() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "filesystem", "-f", "-i", input.getPath() });
+      assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+      assertFalse(output.contains("ERROR: Unrecognized option: -i"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testFilesystemFixLongOption() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"filesystem",  "--fix"});
+    assertFalse(output.contains("ERROR: Unrecognized option: --fix"));
+  }
+
+  @Test
+  public void testFilesystemFixLongOptionTable() throws IOException {
+    String output = retrieveOptionOutput(new String[]{"filesystem",  "--fix", "table"});
+    assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+  }
+
+  @Test
+  public void testFilesystemFixLongOptionInputFile() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "filesystem", "--fix", "-i", input.getPath() });
+      assertFalse(output.contains("ERROR: Unrecognized option: --fix"));
+      assertFalse(output.contains("ERROR: Unrecognized option: -i"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testFilesystemFixShortOptionInputFileLong() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "filesystem", "-f", "--inputFiles", input.getPath() });
+      assertFalse(output.contains("ERROR: Unrecognized option: -f"));
+      assertFalse(output.contains("ERROR: Unrecognized option: --inputFiles"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testFilesystemFixLongOptionInputFileLong() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "filesystem", "--fix", "--inputFiles",
+          input.getPath() });
+      assertFalse(output.contains("ERROR: Unrecognized option: --fix"));
+      assertFalse(output.contains("ERROR: Unrecognized option: --inputFiles"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testFilesystemInputFileLong() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "filesystem", "--inputFiles", input.getPath() });
+      assertFalse(output.contains("ERROR: Unrecognized option: --inputFiles"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  @Test
+  public void testFilesystemInputFile() throws Exception {
+    File input = null;
+    try {
+      input = createInputFile();
+      String output =
+        retrieveOptionOutput(new String[] { "filesystem", "-i", input.getPath() });
+      assertFalse(output.contains("ERROR: Unrecognized option: -i"));
+    } finally {
+      input.delete();
+    }
+  }
+
+  private String retrieveOptionOutput(String[] options) throws IOException {
     ByteArrayOutputStream os = new ByteArrayOutputStream();
     PrintStream stream = new PrintStream(os);
     PrintStream oldOut = System.out;
     System.setOut(stream);
-    if (option != null) {
-      this.hbck2.run(new String[] { option });
+    if (options != null) {
+      this.hbck2.run(options);
     } else {
       this.hbck2.run(null);
     }