You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2019/06/07 22:26:46 UTC

[kudu] branch master updated: [backup] Make KuduBackupCLI executable

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 76c72ee  [backup] Make KuduBackupCLI executable
76c72ee is described below

commit 76c72ee9f9b168782e4630edf99e7d7425e64680
Author: Grant Henke <gr...@apache.org>
AuthorDate: Thu Jun 6 19:16:21 2019 -0500

    [backup] Make KuduBackupCLI executable
    
    This patch changes the KuduBackupCLI to be the
    defacto main class of the kudu-backup-tools. It
    does this by combining both the GC tool and the
    list tool into a single CLI parser.
    
    I tested this by running:
    
       `java -jar build/libs/kudu-backup-tools-1.10.0-SNAPSHOT.jar`
    
    Change-Id: I15e0270a94844ffb7cb804a182dcf9699dd12ec7
    Reviewed-on: http://gerrit.cloudera.org:8080/13551
    Tested-by: Kudu Jenkins
    Reviewed-by: Will Berkeley <wd...@gmail.com>
---
 java/kudu-backup-tools/build.gradle                |   8 +
 .../org/apache/kudu/backup/KuduBackupCLI.scala     | 196 ++++++++-------------
 .../org/apache/kudu/backup/KuduBackupCleaner.scala |  78 +-------
 ...{KuduBackupCLI.scala => KuduBackupLister.scala} |  91 +---------
 .../apache/kudu/backup/TestKuduBackupCleaner.scala |  38 ++--
 ...uBackupCLI.scala => TestKuduBackupLister.scala} |  58 +++---
 6 files changed, 156 insertions(+), 313 deletions(-)

diff --git a/java/kudu-backup-tools/build.gradle b/java/kudu-backup-tools/build.gradle
index df89c76..e08fb4e 100644
--- a/java/kudu-backup-tools/build.gradle
+++ b/java/kudu-backup-tools/build.gradle
@@ -23,6 +23,14 @@ shadow {
   isToolJar = true
 }
 
+
+// Add the main class to the manifest so the jar can be run via `java -jar`
+jar {
+  manifest {
+    attributes 'Main-Class': 'org.apache.kudu.backup.KuduBackupCLI'
+  }
+}
+
 dependencies {
   compile project(path: ":kudu-backup-common")
   compile project(path: ":kudu-client", configuration: "shadow")
diff --git a/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCLI.scala b/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCLI.scala
index 6c2123b..4ac95ea 100644
--- a/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCLI.scala
+++ b/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCLI.scala
@@ -16,18 +16,27 @@
 // under the License.
 package org.apache.kudu.backup
 
-import java.text.SimpleDateFormat
-
-import org.apache.hadoop.conf.Configuration
+import java.time.Duration
+import java.time.temporal.ChronoUnit
+
+import org.apache.kudu.backup.BackupCLIOptions.DefaultDryRun
+import org.apache.kudu.backup.BackupCLIOptions.DefaultExpirationAge
+import org.apache.kudu.backup.BackupCLIOptions.DefaultFormat
+import org.apache.kudu.backup.BackupCLIOptions.DefaultListType
+import org.apache.kudu.backup.BackupCLIOptions.DefaultVerbose
 import org.apache.yetus.audience.InterfaceAudience
 import org.apache.yetus.audience.InterfaceStability
 import scopt.OptionParser
 
-// The possible backup CLI tool actions.
+object Mode extends Enumeration {
+  val LIST, CLEAN = Value
+}
+
+// The possible backup CLI tool list types.
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-object Action extends Enumeration {
-  val LIST_LATEST, LIST_RESTORE_SEQUENCE, LIST_ALL = Value
+object ListType extends Enumeration {
+  val LATEST, RESTORE_SEQUENCE, ALL = Value
 }
 
 // The possible backup CLI print formats.
@@ -40,41 +49,79 @@ object Format extends Enumeration {
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 case class BackupCLIOptions(
-    action: Action.Value,
-    format: Format.Value,
-    tables: Seq[String],
-    rootPath: String)
+    rootPath: String,
+    mode: Mode.Value,
+    tables: Seq[String] = Seq(),
+    listType: ListType.Value = DefaultListType,
+    format: Format.Value = DefaultFormat,
+    expirationAge: Duration = DefaultExpirationAge,
+    dryRun: Boolean = DefaultDryRun,
+    verbose: Boolean = DefaultVerbose)
 
 object BackupCLIOptions {
 
-  val ProgramName: String =
-    KuduBackupCLI.getClass.getCanonicalName.dropRight(1) // Remove trailing `$`
+  val DefaultDryRun: Boolean = false
+  val DefaultExpirationAge: Duration = Duration.of(30, ChronoUnit.DAYS)
+  val DefaultFormat: Format.Value = Format.PRETTY
+  val DefaultListType: ListType.Value = ListType.LATEST
+  val DefaultVerbose: Boolean = false
+
+  val ProgramName: String = "kudu-backup-tools"
 
   val parser: OptionParser[BackupCLIOptions] =
     new OptionParser[BackupCLIOptions](ProgramName) {
+
       opt[String]("rootPath")
         .action((v, o) => o.copy(rootPath = v))
         .text("The root path to search for backups. Accepts any Hadoop compatible path.")
         .required()
 
-      arg[String]("format")
-        .validate(validateEnumeratedOption("format", Format.values.map(_.toString.toLowerCase)))
-        .action((v, o) => o.copy(format = Format.withName(v.toUpperCase)))
-        .text("The output format. One of 'pretty', 'tsv', 'csv'.")
-        .optional()
-
-      arg[String]("<action>")
-        .validate(validateEnumeratedOption("action", Action.values.map(_.toString.toLowerCase)))
-        .action((v, o) => o.copy(action = Action.withName(v.toUpperCase)))
-        .text("The action to perform. One of 'list_latest', 'list_restore_sequence', 'list_all'.")
-
       arg[String]("<table>...")
         .unbounded()
         .action((v, o) => o.copy(tables = o.tables :+ v))
-        .text("A list of tables about which to print backup information. Specifying no tables includes all tables.")
+        .text("A list of tables to be included. Specifying no tables includes all tables.")
         .optional()
 
       help("help").text("Prints this usage text")
+
+      cmd("list")
+        .action((_, c) => c.copy(mode = Mode.LIST))
+        .text("List the backups in the rootPath.")
+        .children(
+          opt[String]("type")
+            .action((v, o) => o.copy(listType = ListType.withName(v.toUpperCase)))
+            .text("The type of listing to perform. One of 'latest', 'restore_sequence', 'all'. " +
+              s"Default: ${DefaultListType.toString.toLowerCase()}")
+            .validate(
+              validateEnumeratedOption("type", ListType.values.map(_.toString.toLowerCase))),
+          opt[String]("format")
+            .action((v, o) => o.copy(format = Format.withName(v.toUpperCase)))
+            .text(s"The output format. One of 'pretty', 'tsv', 'csv'. " +
+              s"Default: ${DefaultFormat.toString.toLowerCase()}")
+            .validate(validateEnumeratedOption("format", Format.values.map(_.toString.toLowerCase)))
+            .optional()
+        )
+
+      cmd("clean")
+        .action((_, c) => c.copy(mode = Mode.CLEAN))
+        .text("Cleans up old backup data in the rootPath.")
+        .children(
+          opt[String]("expirationAgeDays")
+            .action((v, o) => o.copy(expirationAge = Duration.of(v.toLong, ChronoUnit.DAYS)))
+            .text("The age at which old backups should be expired. " +
+              "Backups that are part of the current restore path are never expired. " +
+              s"Default: ${DefaultExpirationAge.toDays} Days")
+            .optional(),
+          opt[Boolean]("dryRun")
+            .action((v, o) => o.copy(dryRun = v))
+            .text("Report on what backups will be deleted, but don't delete anything. " +
+              s"Overrides --verbose. Default: $DefaultDryRun")
+            .optional(),
+          opt[Boolean]("verbose")
+            .action((v, o) => o.copy(verbose = v))
+            .text(s"Report on what backups are deleted. Default: $DefaultVerbose")
+            .optional()
+        )
     }
 
   def validateEnumeratedOption(
@@ -89,7 +136,7 @@ object BackupCLIOptions {
     }
 
   def parse(args: Seq[String]): Option[BackupCLIOptions] = {
-    parser.parse(args, BackupCLIOptions(null, Format.PRETTY, Seq(), null))
+    parser.parse(args, BackupCLIOptions(null, null))
   }
 }
 
@@ -97,101 +144,14 @@ object BackupCLIOptions {
 @InterfaceStability.Unstable
 object KuduBackupCLI {
 
-  // The header for all tables printed by the tool.
-  val HEADER: Seq[String] =
-    Seq("table name", "table id", "end time", "start timestamp", "end timestamp", "type")
-
-  // Run the backup CLI tool with the given options. Like a command, returns 0 if successful, or
-  // a nonzero error code.
+  // Run the backup CLI tool with the given options.
+  // Like a command, returns 0 if successful, or a nonzero error code.
   def run(options: BackupCLIOptions): Int = {
-    // Sort by table name for a consistent ordering (at least if there's no duplicate names).
-    val sortedTables = options.tables.sorted
-
-    val io: BackupIO = new BackupIO(new Configuration(), options.rootPath)
-    val backupGraphs =
-      if (sortedTables.isEmpty)
-        io.readAllBackupGraphs()
-      else
-        io.readBackupGraphsByTableName(sortedTables)
-
-    options.action match {
-      case Action.LIST_LATEST => {
-        val rows = backupGraphs.map(graph => rowForBackupNode(graph.restorePath.lastBackup))
-        printTable(options.format, rows)
-      }
-      case Action.LIST_RESTORE_SEQUENCE => {
-        val tablesOfBackups =
-          backupGraphs.map(_.restorePath.backups.map(node => rowForBackupNode(node)))
-        tablesOfBackups.foreach(table => printTable(options.format, table))
-      }
-      case Action.LIST_ALL => {
-        val tablesOfBackups = backupGraphs.map(
-          _.allBackups.sortBy(node => node.metadata.getToMs).map(node => rowForBackupNode(node)))
-        tablesOfBackups.foreach(table => printTable(options.format, table))
-      }
-    }
-    // Because of renames, one table name might map to multiple backup directories, so it's not
-    // sufficient to check the size of 'options.tables' against the size of 'backupGraphs'.
-    val foundTables = backupGraphs.map(graph => graph.backupBase.metadata.getTableName).toSet
-    val notFoundTables = options.tables.filter(table => !foundTables.contains(table))
-    if (notFoundTables.nonEmpty) {
-      Console.err.println(s"No backups were found for ${notFoundTables.size} table(s):")
-      notFoundTables.foreach(Console.err.println)
-      return 1
-    }
-    0
-  }
-
-  private def rowForBackupNode(backup: BackupNode): Seq[String] = {
-    val metadata = backup.metadata
-    val tableName = metadata.getTableName
-    val tableId = metadata.getTableId
-    val fromMs = metadata.getFromMs
-    val toMs = metadata.getToMs
-    val toDatetime = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ").format(toMs)
-    val backupType = if (fromMs == 0) "full" else "incremental"
-    Seq(tableName, tableId, toDatetime, s"$fromMs", s"$toMs", backupType)
-  }
-
-  private def formatDsv(delimiter: String, table: Seq[Seq[String]]): String = {
-    table.map(_.mkString(delimiter)).mkString("\n")
-  }
-
-  private def formatPrettyTable(table: Seq[Seq[String]]): String = {
-    if (table.isEmpty) {
-      return ""
-    }
-    // The width of a column is the width of largest cell, plus a padding of 2.
-    val colWidths = table.transpose.map(_.map(_.length).max + 2)
-    val rows = table.map { row =>
-      (row, colWidths).zipped
-        .map {
-          // 1 space on left, then pad to (padding - 1) spaces.
-          case (cell, width) => s" %-${width - 1}s".format(cell)
-        }
-        .mkString("|")
-    }
-    val separatorRow = colWidths.map("-" * _).mkString("+")
-    (rows.head +: separatorRow +: rows.tail).mkString("\n")
-  }
-
-  private def printTable(format: Format.Value, rows: Seq[Seq[String]]): Unit = {
-    if (rows.isEmpty) {
-      return
-    }
-    val table = HEADER +: rows
-    format match {
-      case Format.PRETTY => {
-        println(formatPrettyTable(table))
-      }
-      case Format.TSV => {
-        println(formatDsv("\t", table))
-      }
-      case Format.CSV => {
-        println(formatDsv(",", table))
-      }
+    options.mode match {
+      case Mode.LIST => KuduBackupLister.run(options)
+      case Mode.CLEAN => KuduBackupCleaner.run(options)
+      case _ => throw new IllegalArgumentException("Arguments must come after the command")
     }
-    println() // Spacing after the table.
   }
 
   def main(args: Array[String]): Unit = {
diff --git a/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCleaner.scala b/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCleaner.scala
index 0170c6e..7d7d3ff 100644
--- a/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCleaner.scala
+++ b/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCleaner.scala
@@ -16,79 +16,24 @@
 // under the License.
 package org.apache.kudu.backup
 
-import java.time.temporal.ChronoUnit
-import java.time.Duration
 import java.time.Instant
 
+import com.google.common.base.Preconditions
 import org.apache.hadoop.conf.Configuration
-import scopt.OptionParser
 import org.apache.kudu.backup.Backup.TableMetadataPB
-import org.apache.yetus.audience.InterfaceAudience
-import org.apache.yetus.audience.InterfaceStability
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-case class BackupCleanerOptions(
-    tables: Seq[String],
-    rootPath: String,
-    expirationAge: Duration,
-    dryRun: Boolean,
-    verbose: Boolean)
-
-object BackupCleanerOptions {
-  val DefaultExpirationAge: Duration = Duration.of(30, ChronoUnit.DAYS)
-
-  val ProgramName: String =
-    KuduBackupCleaner.getClass.getCanonicalName.dropRight(1) // Remove trailing `$`
-
-  val parser: OptionParser[BackupCleanerOptions] =
-    new OptionParser[BackupCleanerOptions](ProgramName) {
-      opt[String]("rootPath")
-        .action((v, o) => o.copy(rootPath = v))
-        .text("The root path to search for backups. Accepts any Hadoop compatible path.")
-        .required()
-
-      opt[String]("expirationAgeDays")
-        .action((v, o) => o.copy(expirationAge = Duration.of(v.toLong, ChronoUnit.DAYS)))
-        .text("The age at which old backups should be expired. Backups that are part of the current restore path are never expired.")
-        .optional()
-
-      opt[Boolean]("dryRun")
-        .action((v, o) => o.copy(dryRun = v))
-        .text(
-          "Report on what backups will be deleted, but don't delete anything. Overrides --verbose.")
-        .optional()
-
-      opt[Boolean]("verbose")
-        .action((v, o) => o.copy(verbose = v))
-        .text("Report on what backups are deleted.")
-        .optional()
-
-      arg[String]("<table>...")
-        .unbounded()
-        .action((v, o) => o.copy(tables = o.tables :+ v))
-        .text("A list of tables whose backups should be garbage-collected. Specifying no tables includes all tables.")
-        .optional()
-
-      help("help").text("Prints this usage text")
-    }
-
-  def parse(args: Seq[String]): Option[BackupCleanerOptions] = {
-    parser.parse(
-      args,
-      BackupCleanerOptions(Seq(), null, DefaultExpirationAge, dryRun = false, verbose = false))
-  }
-}
 
 object KuduBackupCleaner {
 
   private def backupToShortString(metadata: TableMetadataPB): String = {
-    s"name: ${metadata.getTableName}, id: ${metadata.getTableId}, fromMs: ${metadata.getFromMs}, toMs: ${metadata.getToMs}"
+    s"name: ${metadata.getTableName}, id: ${metadata.getTableId}, fromMs: ${metadata.getFromMs}, " +
+      s"toMs: ${metadata.getToMs}"
   }
 
   // Run the cleanup tool with the given options. Like a command, returns 0 if successful, or
   // a nonzero error code.
-  def run(options: BackupCleanerOptions): Int = {
+  def run(options: BackupCLIOptions): Int = {
+    Preconditions.checkArgument(options.mode == Mode.CLEAN)
+
     // Delete the metadata for all backups that satisfy the following three conditions:
     // 1. The table name matches the provided names (does not apply if no names were specified).
     // 2. The backup is part of a path whose latest backup is older than the expiration age.
@@ -130,8 +75,8 @@ object KuduBackupCleaner {
                 if (options.verbose) {
                   println(s"Delete backup ${backupToShortString(backup.metadata)}")
                 }
-                // TODO(wdberkeley): Make this crash-consistent by handling backup directories with no
-                //  metadata.
+                // TODO(wdberkeley): Make this crash-consistent by handling backup directories
+                //  with no metadata.
                 io.deleteBackup(backup.metadata)
               }
             })
@@ -141,11 +86,4 @@ object KuduBackupCleaner {
 
     0
   }
-
-  def main(args: Array[String]): Unit = {
-    val options = BackupCleanerOptions
-      .parse(args)
-      .getOrElse(throw new IllegalArgumentException("could not parse the arguments"))
-    System.exit(run(options))
-  }
 }
diff --git a/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCLI.scala b/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupLister.scala
similarity index 62%
copy from java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCLI.scala
copy to java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupLister.scala
index 6c2123b..e2e4f34 100644
--- a/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupCLI.scala
+++ b/java/kudu-backup-tools/src/main/scala/org/apache/kudu/backup/KuduBackupLister.scala
@@ -18,84 +18,14 @@ package org.apache.kudu.backup
 
 import java.text.SimpleDateFormat
 
+import com.google.common.base.Preconditions
 import org.apache.hadoop.conf.Configuration
 import org.apache.yetus.audience.InterfaceAudience
 import org.apache.yetus.audience.InterfaceStability
-import scopt.OptionParser
 
-// The possible backup CLI tool actions.
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-object Action extends Enumeration {
-  val LIST_LATEST, LIST_RESTORE_SEQUENCE, LIST_ALL = Value
-}
-
-// The possible backup CLI print formats.
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-object Format extends Enumeration {
-  val PRETTY, TSV, CSV = Value
-}
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-case class BackupCLIOptions(
-    action: Action.Value,
-    format: Format.Value,
-    tables: Seq[String],
-    rootPath: String)
-
-object BackupCLIOptions {
-
-  val ProgramName: String =
-    KuduBackupCLI.getClass.getCanonicalName.dropRight(1) // Remove trailing `$`
-
-  val parser: OptionParser[BackupCLIOptions] =
-    new OptionParser[BackupCLIOptions](ProgramName) {
-      opt[String]("rootPath")
-        .action((v, o) => o.copy(rootPath = v))
-        .text("The root path to search for backups. Accepts any Hadoop compatible path.")
-        .required()
-
-      arg[String]("format")
-        .validate(validateEnumeratedOption("format", Format.values.map(_.toString.toLowerCase)))
-        .action((v, o) => o.copy(format = Format.withName(v.toUpperCase)))
-        .text("The output format. One of 'pretty', 'tsv', 'csv'.")
-        .optional()
-
-      arg[String]("<action>")
-        .validate(validateEnumeratedOption("action", Action.values.map(_.toString.toLowerCase)))
-        .action((v, o) => o.copy(action = Action.withName(v.toUpperCase)))
-        .text("The action to perform. One of 'list_latest', 'list_restore_sequence', 'list_all'.")
-
-      arg[String]("<table>...")
-        .unbounded()
-        .action((v, o) => o.copy(tables = o.tables :+ v))
-        .text("A list of tables about which to print backup information. Specifying no tables includes all tables.")
-        .optional()
-
-      help("help").text("Prints this usage text")
-    }
-
-  def validateEnumeratedOption(
-      name: String,
-      optionStrings: Iterable[String]): String => Either[String, Unit] =
-    (v: String) => {
-      if (optionStrings.exists(_.equalsIgnoreCase(v))) {
-        Right(())
-      } else {
-        Left(s"$name must be one of ${optionStrings.mkString(", ")}: $v")
-      }
-    }
-
-  def parse(args: Seq[String]): Option[BackupCLIOptions] = {
-    parser.parse(args, BackupCLIOptions(null, Format.PRETTY, Seq(), null))
-  }
-}
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-object KuduBackupCLI {
+object KuduBackupLister {
 
   // The header for all tables printed by the tool.
   val HEADER: Seq[String] =
@@ -104,6 +34,8 @@ object KuduBackupCLI {
   // Run the backup CLI tool with the given options. Like a command, returns 0 if successful, or
   // a nonzero error code.
   def run(options: BackupCLIOptions): Int = {
+    Preconditions.checkArgument(options.mode == Mode.LIST);
+
     // Sort by table name for a consistent ordering (at least if there's no duplicate names).
     val sortedTables = options.tables.sorted
 
@@ -114,17 +46,17 @@ object KuduBackupCLI {
       else
         io.readBackupGraphsByTableName(sortedTables)
 
-    options.action match {
-      case Action.LIST_LATEST => {
+    options.listType match {
+      case ListType.LATEST => {
         val rows = backupGraphs.map(graph => rowForBackupNode(graph.restorePath.lastBackup))
         printTable(options.format, rows)
       }
-      case Action.LIST_RESTORE_SEQUENCE => {
+      case ListType.RESTORE_SEQUENCE => {
         val tablesOfBackups =
           backupGraphs.map(_.restorePath.backups.map(node => rowForBackupNode(node)))
         tablesOfBackups.foreach(table => printTable(options.format, table))
       }
-      case Action.LIST_ALL => {
+      case ListType.ALL => {
         val tablesOfBackups = backupGraphs.map(
           _.allBackups.sortBy(node => node.metadata.getToMs).map(node => rowForBackupNode(node)))
         tablesOfBackups.foreach(table => printTable(options.format, table))
@@ -193,11 +125,4 @@ object KuduBackupCLI {
     }
     println() // Spacing after the table.
   }
-
-  def main(args: Array[String]): Unit = {
-    val options = BackupCLIOptions
-      .parse(args)
-      .getOrElse(throw new IllegalArgumentException("could not parse the arguments"))
-    System.exit(run(options))
-  }
 }
diff --git a/java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupCleaner.scala b/java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupCleaner.scala
index ffc7c45..879d269 100644
--- a/java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupCleaner.scala
+++ b/java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupCleaner.scala
@@ -35,16 +35,16 @@ import org.slf4j.LoggerFactory
 class TestKuduBackupCleaner {
   val log: Logger = LoggerFactory.getLogger(getClass)
 
-  var rootDir: Path = _
+  var rootPath: Path = _
 
   @Before
   def setUp(): Unit = {
-    rootDir = Files.createTempDirectory("backupcli")
+    rootPath = Files.createTempDirectory("backupcli")
   }
 
   @After
   def tearDown(): Unit = {
-    FileUtils.deleteDirectory(rootDir.toFile)
+    FileUtils.deleteDirectory(rootPath.toFile)
   }
 
   // Return the epoch time in milliseconds that is 'secsBefore' seconds before 'current'.
@@ -54,7 +54,7 @@ class TestKuduBackupCleaner {
 
   @Test
   def testBackupCleaner(): Unit = {
-    val io = new BackupIO(new Configuration(), rootDir.toUri.toString)
+    val io = new BackupIO(new Configuration(), rootPath.toUri.toString)
     val expirationAge = Duration.of(15, ChronoUnit.SECONDS)
     val now = Instant.now
     val tableName = "taco"
@@ -74,13 +74,7 @@ class TestKuduBackupCleaner {
     createPath(pathA)
 
     // Nothing should be cleaned up because all backups are on the restore path.
-    val options =
-      BackupCleanerOptions(
-        Seq(),
-        rootDir.toUri.toString,
-        expirationAge,
-        dryRun = false,
-        verbose = true)
+    val options = createOptions(rootPath, expirationAge, verbose = true)
     assertEquals(0, KuduBackupCleaner.run(options))
 
     val backupExists = (secsAgo: Long) => {
@@ -104,12 +98,7 @@ class TestKuduBackupCleaner {
 
     // Running the cleaner should delete path A and the forked backup, but first do a dry run and
     // make sure nothing gets deleted.
-    val dryRunOptions = BackupCleanerOptions(
-      Seq(),
-      rootDir.toUri.toString,
-      expirationAge,
-      dryRun = true,
-      verbose = false)
+    val dryRunOptions = createOptions(rootPath, expirationAge, dryRun = true)
     assertEquals(0, KuduBackupCleaner.run(dryRunOptions))
 
     assertTrue(pathA.forall(backupExists(_)))
@@ -132,4 +121,19 @@ class TestKuduBackupCleaner {
     assertTrue(pathB.forall(backupExists(_)))
     assertTrue(pathC.forall(backupExists(_)))
   }
+
+  def createOptions(
+      rootPath: Path,
+      expirationAge: Duration,
+      tables: Seq[String] = Seq(),
+      dryRun: Boolean = false,
+      verbose: Boolean = false): BackupCLIOptions = {
+    new BackupCLIOptions(
+      rootPath.toUri.toString,
+      Mode.CLEAN,
+      tables = tables,
+      expirationAge = expirationAge,
+      dryRun = dryRun,
+      verbose = verbose)
+  }
 }
diff --git a/java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupCLI.scala b/java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupLister.scala
similarity index 78%
rename from java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupCLI.scala
rename to java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupLister.scala
index aec1686..cf75ba1 100644
--- a/java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupCLI.scala
+++ b/java/kudu-backup-tools/src/test/scala/org/apache/kudu/backup/TestKuduBackupLister.scala
@@ -24,26 +24,26 @@ import java.text.SimpleDateFormat
 
 import org.apache.commons.io.FileUtils
 import org.apache.hadoop.conf.Configuration
-import org.junit.After
 import org.junit.Assert._
+import org.junit.After
 import org.junit.Before
 import org.junit.Test
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
 
-class TestKuduBackupCLI {
+class TestKuduBackupLister {
   val log: Logger = LoggerFactory.getLogger(getClass)
 
-  var rootDir: Path = _
+  var rootPath: Path = _
 
   @Before
   def setUp(): Unit = {
-    rootDir = Files.createTempDirectory("backupcli")
+    rootPath = Files.createTempDirectory("backupcli")
   }
 
   @After
   def tearDown(): Unit = {
-    FileUtils.deleteDirectory(rootDir.toFile)
+    FileUtils.deleteDirectory(rootPath.toFile)
   }
 
   // Helper to write a standard collection of backup metadata useful for a few tests.
@@ -71,17 +71,16 @@ class TestKuduBackupCLI {
 
   @Test
   def testListAllBackups(): Unit = {
-    val io = new BackupIO(new Configuration(), rootDir.toUri.toString)
+    val io = new BackupIO(new Configuration(), rootPath.toUri.toString)
     createStandardTableMetadata(io)
 
-    val options =
-      BackupCLIOptions(Action.LIST_ALL, Format.CSV, Seq(), rootDir.toUri.toString)
+    val options = createOptions(rootPath, ListType.ALL)
     val stdout = new ByteArrayOutputStream
     Console.withOut(new PrintStream(stdout)) {
       assertEquals(0, KuduBackupCLI.run(options))
     }
 
-    val headerString = KuduBackupCLI.HEADER.mkString(",")
+    val headerString = KuduBackupLister.HEADER.mkString(",")
     val expected = Seq(
       headerString,
       s"pizza,id_pizza,${endTime(200)},0,200,full",
@@ -98,17 +97,16 @@ class TestKuduBackupCLI {
 
   @Test
   def testListLatestBackups(): Unit = {
-    val io = new BackupIO(new Configuration(), rootDir.toUri.toString)
+    val io = new BackupIO(new Configuration(), rootPath.toUri.toString)
     createStandardTableMetadata(io)
 
-    val options =
-      BackupCLIOptions(Action.LIST_LATEST, Format.CSV, Seq(), rootDir.toUri.toString)
+    val options = createOptions(rootPath, ListType.LATEST)
     val stdout = new ByteArrayOutputStream
     Console.withOut(new PrintStream(stdout)) {
       assertEquals(0, KuduBackupCLI.run(options))
     }
 
-    val headerString = KuduBackupCLI.HEADER.mkString(",")
+    val headerString = KuduBackupLister.HEADER.mkString(",")
     val expected = Seq(
       headerString,
       s"pizza,id_pizza,${endTime(600)},400,600,incremental",
@@ -119,17 +117,16 @@ class TestKuduBackupCLI {
 
   @Test
   def testListRestorePath(): Unit = {
-    val io = new BackupIO(new Configuration(), rootDir.toUri.toString)
+    val io = new BackupIO(new Configuration(), rootPath.toUri.toString)
     createStandardTableMetadata(io)
 
-    val options =
-      BackupCLIOptions(Action.LIST_RESTORE_SEQUENCE, Format.CSV, Seq(), rootDir.toUri.toString)
+    val options = createOptions(rootPath, ListType.RESTORE_SEQUENCE)
     val stdout = new ByteArrayOutputStream
     Console.withOut(new PrintStream(stdout)) {
       assertEquals(0, KuduBackupCLI.run(options))
     }
 
-    val headerString = KuduBackupCLI.HEADER.mkString(",")
+    val headerString = KuduBackupLister.HEADER.mkString(",")
     val expected = Seq(
       headerString,
       s"pizza,id_pizza,${endTime(200)},0,200,full",
@@ -145,17 +142,16 @@ class TestKuduBackupCLI {
 
   @Test
   def testTableFilter(): Unit = {
-    val io = new BackupIO(new Configuration(), rootDir.toUri.toString)
+    val io = new BackupIO(new Configuration(), rootPath.toUri.toString)
     createStandardTableMetadata(io)
 
-    val options =
-      BackupCLIOptions(Action.LIST_ALL, Format.CSV, Seq("taco"), rootDir.toUri.toString)
+    val options = createOptions(rootPath, ListType.ALL, Seq("taco"))
     val stdout = new ByteArrayOutputStream
     Console.withOut(new PrintStream(stdout)) {
       assertEquals(0, KuduBackupCLI.run(options))
     }
 
-    val headerString = KuduBackupCLI.HEADER.mkString(",")
+    val headerString = KuduBackupLister.HEADER.mkString(",")
     val expected = Seq(
       headerString,
       s"taco,id_taco,${endTime(100)},0,100,full",
@@ -167,11 +163,10 @@ class TestKuduBackupCLI {
 
   @Test
   def testMissingTable(): Unit = {
-    val io = new BackupIO(new Configuration(), rootDir.toUri.toString)
+    val io = new BackupIO(new Configuration(), rootPath.toUri.toString)
     createStandardTableMetadata(io)
 
-    val options =
-      BackupCLIOptions(Action.LIST_ALL, Format.CSV, Seq("pizza", "nope"), rootDir.toUri.toString)
+    val options = createOptions(rootPath, ListType.ALL, Seq("pizza", "nope"))
     val stdout = new ByteArrayOutputStream
     val stderr = new ByteArrayOutputStream
     Console.withOut(new PrintStream(stdout)) {
@@ -180,7 +175,7 @@ class TestKuduBackupCLI {
       }
     }
 
-    val headerString = KuduBackupCLI.HEADER.mkString(",")
+    val headerString = KuduBackupLister.HEADER.mkString(",")
     val expected = Seq(
       headerString,
       s"pizza,id_pizza,${endTime(200)},0,200,full",
@@ -191,4 +186,17 @@ class TestKuduBackupCLI {
 
     assertEquals("No backups were found for 1 table(s):\nnope", stderr.toString.trim)
   }
+
+  def createOptions(
+      rootPath: Path,
+      listType: ListType.Value,
+      tables: Seq[String] = Seq(),
+      format: Format.Value = Format.CSV): BackupCLIOptions = {
+    new BackupCLIOptions(
+      rootPath.toUri.toString,
+      Mode.LIST,
+      tables = tables,
+      listType = listType,
+      format = format)
+  }
 }