You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by jm...@apache.org on 2023/06/12 17:06:49 UTC

[accumulo] branch elasticity updated: Add API method for getting tablet hosting goals (#3451)

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

jmark99 pushed a commit to branch elasticity
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/elasticity by this push:
     new 18e9a3453b Add API method for getting tablet hosting goals (#3451)
18e9a3453b is described below

commit 18e9a3453bf3a16bc81f68edb1839206045ddf71
Author: Mark Owens <jm...@apache.org>
AuthorDate: Mon Jun 12 13:06:42 2023 -0400

    Add API method for getting tablet hosting goals (#3451)
    
    * Add API method for getting tablet hosting goals
    
    Add an API method for obtaining hosting goals for a tablet. This PR creates the new API method and also the corresponding shell command.
    
    In order to simplify setting and retrieving hosting goals the previously created shell command, 'goal', which sets hosting goals was renamed to 'setgoal'. The shell command to retrieve hosting goals is named 'getgoal'.
    
    Change summary:
    
    * A class, HostingGoalForTablet, was created to hold the hosting goal   informtion from a tablet query.
    * TableOperations class was updated to contain the   'getTabletHostingGoal' API method.
    * TabletOperationsImpl class was updated to contain the API   implementation details.
    * A simple typo was corrected in the KeyExtent class.
    * Shell class modifications include updating the TableCommands section   with the new GetTabletHostingGoalsCommand and renaming the   TabletHostingGoalCommand to SetTabletHostingGoalCommand.
    * Corrected a minor issue in the renamed SetTabletHostingGoalCommand usage  statement. The statement was incorrectly using 'range' rather than   'row' in its output.
    * Created a new class, GetTabletHostingGoalCommand.
    * Wrote integration tests for both the API and Shell commands.
    * Added @since tag to class
    
    Co-authored-by: Keith Turner <kt...@apache.org>
    
    Closes #3302
---
 .../core/client/admin/HostingGoalForTablet.java    |  65 +++++
 .../core/client/admin/TableOperations.java         |  14 +
 .../core/clientImpl/TableOperationsImpl.java       |  28 ++
 .../apache/accumulo/core/dataImpl/KeyExtent.java   |   4 +-
 .../main/java/org/apache/accumulo/shell/Shell.java |  10 +-
 ...mmand.java => GetTabletHostingGoalCommand.java} |  48 ++--
 ...mmand.java => SetTabletHostingGoalCommand.java} |  14 +-
 .../apache/accumulo/test/TableOperationsIT.java    | 308 +++++++++++++++++++++
 .../apache/accumulo/test/shell/ShellServerIT.java  | 131 ++++++++-
 9 files changed, 576 insertions(+), 46 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/HostingGoalForTablet.java b/core/src/main/java/org/apache/accumulo/core/client/admin/HostingGoalForTablet.java
new file mode 100644
index 0000000000..10718ff01f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/HostingGoalForTablet.java
@@ -0,0 +1,65 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.client.admin;
+
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.TabletId;
+
+/**
+ * This class contains information that defines the tablet hosting data for a table. The class
+ * contains the TabletId and associated goal for each tablet in a table or a subset of tablets if a
+ * range is provided.
+ *
+ * @since 4.0.0
+ */
+public class HostingGoalForTablet {
+  private final TabletId tabletId;
+  private final TabletHostingGoal hostingGoal;
+
+  public HostingGoalForTablet(TabletId tabletId, TabletHostingGoal hostingGoal) {
+    this.tabletId = tabletId;
+    this.hostingGoal = hostingGoal;
+  }
+
+  public TabletHostingGoal getHostingGoal() {
+    return hostingGoal;
+  }
+
+  public TabletId getTabletId() {
+    return tabletId;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    HostingGoalForTablet that = (HostingGoalForTablet) o;
+    return Objects.equals(tabletId, that.tabletId) && hostingGoal == that.hostingGoal;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(tabletId, hostingGoal);
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index fe92ec568c..0eb7504d15 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -29,6 +29,7 @@ import java.util.SortedSet;
 import java.util.concurrent.Executor;
 import java.util.function.Consumer;
 import java.util.function.Predicate;
+import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -1005,4 +1006,17 @@ public interface TableOperations {
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     throw new UnsupportedOperationException();
   }
+
+  /**
+   * Retrieve the hosting goal for a range of tablets in the specified table.
+   *
+   * @param tableName table name
+   * @param range tablet range
+   * @since 4.0.0
+   */
+  default Stream<HostingGoalForTablet> getTabletHostingGoal(final String tableName,
+      final Range range) throws TableNotFoundException {
+    throw new UnsupportedOperationException();
+  }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index a063cff241..c837d23a77 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -25,6 +25,7 @@ import static java.util.Objects.requireNonNull;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toSet;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.HOSTING_GOAL;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.HOSTING_REQUESTED;
@@ -69,6 +70,7 @@ import java.util.function.Consumer;
 import java.util.function.Predicate;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipInputStream;
 
@@ -87,6 +89,7 @@ import org.apache.accumulo.core.client.admin.CloneConfiguration;
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.FindMax;
+import org.apache.accumulo.core.client.admin.HostingGoalForTablet;
 import org.apache.accumulo.core.client.admin.ImportConfiguration;
 import org.apache.accumulo.core.client.admin.Locations;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
@@ -112,6 +115,8 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.TabletId;
@@ -2145,4 +2150,27 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
+  @Override
+  public Stream<HostingGoalForTablet> getTabletHostingGoal(final String tableName,
+      final Range range) throws TableNotFoundException {
+    EXISTING_TABLE_NAME.validate(tableName);
+
+    final Text scanRangeStart = (range.getStartKey() == null) ? null : range.getStartKey().getRow();
+    TableId tableId = context.getTableId(tableName);
+
+    TabletsMetadata tabletsMetadata =
+        context.getAmple().readTablets().forTable(tableId).overlapping(scanRangeStart, true, null)
+            .fetch(HOSTING_GOAL, PREV_ROW).checkConsistency().build();
+
+    return tabletsMetadata.stream().peek(tm -> {
+      if (scanRangeStart != null && tm.getEndRow() != null
+          && tm.getEndRow().compareTo(scanRangeStart) < 0) {
+        log.debug(">>>> tablet {} is before scan start range: {}", tm.getExtent(), scanRangeStart);
+        throw new RuntimeException("Bug in ample or this code.");
+      }
+    }).takeWhile(tm -> tm.getPrevEndRow() == null
+        || !range.afterEndKey(new Key(tm.getPrevEndRow()).followingKey(PartialKey.ROW)))
+        .map(tm -> new HostingGoalForTablet(new TabletIdImpl(tm.getExtent()), tm.getHostingGoal()))
+        .onClose(tabletsMetadata::close);
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
index ddf19ff3e9..c131535b5b 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
@@ -380,8 +380,8 @@ public class KeyExtent implements Comparable<KeyExtent> {
    * <p>
    * For example, if this extent represented a range of data from <code>A</code> to <code>Z</code>
    * for a user table, <code>T</code>, this would compute the range to scan
-   * <code>accumulo.metadata</code> that would include all the the metadata for <code>T</code>'s
-   * tablets that contain data in the range <code>(A,Z]</code>.
+   * <code>accumulo.metadata</code> that would include all the metadata for <code>T</code>'s tablets
+   * that contain data in the range <code>(A,Z]</code>.
    */
   public Range toMetaRange() {
     Text metadataPrevRow =
diff --git a/shell/src/main/java/org/apache/accumulo/shell/Shell.java b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
index 88405f197c..61e658236a 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
@@ -110,6 +110,7 @@ import org.apache.accumulo.shell.commands.FormatterCommand;
 import org.apache.accumulo.shell.commands.GetAuthsCommand;
 import org.apache.accumulo.shell.commands.GetGroupsCommand;
 import org.apache.accumulo.shell.commands.GetSplitsCommand;
+import org.apache.accumulo.shell.commands.GetTabletHostingGoalCommand;
 import org.apache.accumulo.shell.commands.GrantCommand;
 import org.apache.accumulo.shell.commands.GrepCommand;
 import org.apache.accumulo.shell.commands.HelpCommand;
@@ -146,13 +147,13 @@ import org.apache.accumulo.shell.commands.SetAuthsCommand;
 import org.apache.accumulo.shell.commands.SetGroupsCommand;
 import org.apache.accumulo.shell.commands.SetIterCommand;
 import org.apache.accumulo.shell.commands.SetShellIterCommand;
+import org.apache.accumulo.shell.commands.SetTabletHostingGoalCommand;
 import org.apache.accumulo.shell.commands.SleepCommand;
 import org.apache.accumulo.shell.commands.SummariesCommand;
 import org.apache.accumulo.shell.commands.SystemPermissionsCommand;
 import org.apache.accumulo.shell.commands.TableCommand;
 import org.apache.accumulo.shell.commands.TablePermissionsCommand;
 import org.apache.accumulo.shell.commands.TablesCommand;
-import org.apache.accumulo.shell.commands.TabletHostingGoalCommand;
 import org.apache.accumulo.shell.commands.TraceCommand;
 import org.apache.accumulo.shell.commands.UserCommand;
 import org.apache.accumulo.shell.commands.UserPermissionsCommand;
@@ -410,9 +411,10 @@ public class Shell extends ShellOptions implements KeywordExecutable {
     Command[] tableCommands = {new CloneTableCommand(), new ConfigCommand(),
         new CreateTableCommand(), new DeleteTableCommand(), new DropTableCommand(), new DUCommand(),
         new ExportTableCommand(), new ImportTableCommand(), new OfflineCommand(),
-        new TabletHostingGoalCommand(), new OnlineCommand(), new RenameTableCommand(),
-        new TablesCommand(), new NamespacesCommand(), new CreateNamespaceCommand(),
-        new DeleteNamespaceCommand(), new RenameNamespaceCommand(), new SummariesCommand()};
+        new SetTabletHostingGoalCommand(), new GetTabletHostingGoalCommand(), new OnlineCommand(),
+        new RenameTableCommand(), new TablesCommand(), new NamespacesCommand(),
+        new CreateNamespaceCommand(), new DeleteNamespaceCommand(), new RenameNamespaceCommand(),
+        new SummariesCommand()};
     Command[] tableControlCommands = {new AddSplitsCommand(), new CompactCommand(),
         new ConstraintCommand(), new FlushCommand(), new GetGroupsCommand(), new GetSplitsCommand(),
         new MergeCommand(), new SetGroupsCommand()};
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/TabletHostingGoalCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/GetTabletHostingGoalCommand.java
similarity index 68%
copy from shell/src/main/java/org/apache/accumulo/shell/commands/TabletHostingGoalCommand.java
copy to shell/src/main/java/org/apache/accumulo/shell/commands/GetTabletHostingGoalCommand.java
index fef60c6282..c79f929d37 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/TabletHostingGoalCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/GetTabletHostingGoalCommand.java
@@ -18,12 +18,10 @@
  */
 package org.apache.accumulo.shell.commands;
 
-import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
 
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.TabletHostingGoal;
+import org.apache.accumulo.core.client.admin.HostingGoalForTablet;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.shell.Shell;
 import org.apache.commons.cli.CommandLine;
@@ -31,37 +29,36 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.io.Text;
 
-public class TabletHostingGoalCommand extends TableOperation {
+public class GetTabletHostingGoalCommand extends TableOperation {
 
   private Option optRow;
   private Option optStartRowExclusive;
   private Option optEndRowExclusive;
-  private Option goalOpt;
-
   private Range range;
-  private TabletHostingGoal goal;
 
   @Override
   public String getName() {
-    return "goal";
+    return "getgoal";
   }
 
   @Override
   public String description() {
-    return "Sets the hosting goal (ALWAYS, ONDEMAND, NEVER) for a range of tablets";
+    return "Retrieves the hosting goal (ALWAYS, ONDEMAND, NEVER) for a range of tablets";
   }
 
   @Override
-  protected void doTableOp(final Shell shellState, final String tableName)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, IOException {
-    shellState.getAccumuloClient().tableOperations().setTabletHostingGoal(tableName, range, goal);
-    Shell.log.debug("Set goal state: {} on table: {}, range: {}", goal, tableName, range);
+  protected void doTableOp(Shell shellState, String tableName) throws Exception {
+    List<HostingGoalForTablet> tabletHostingGoal = shellState.getAccumuloClient().tableOperations()
+        .getTabletHostingGoal(tableName, range).collect(Collectors.toList());
+    shellState.getWriter().println("TABLE: " + tableName);
+    shellState.getWriter().println("TABLET ID    HOSTING GOAL");
+    tabletHostingGoal.forEach(p -> shellState.getWriter()
+        .println(String.format("%-10s   %s", p.getTabletId(), p.getHostingGoal())));
   }
 
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
       throws Exception {
-
     if ((cl.hasOption(OptUtil.START_ROW_OPT) || cl.hasOption(OptUtil.END_ROW_OPT))
         && cl.hasOption(optRow.getOpt())) {
       // did not see a way to make commons cli do this check... it has mutually exclusive options
@@ -79,35 +76,30 @@ public class TabletHostingGoalCommand extends TableOperation {
       final boolean endInclusive = !cl.hasOption(optEndRowExclusive.getOpt());
       this.range = new Range(startRow, startInclusive, endRow, endInclusive);
     }
-
-    this.goal = TabletHostingGoal.valueOf(cl.getOptionValue(goalOpt).toUpperCase());
     return super.execute(fullCommand, cl, shellState);
   }
 
   @Override
   public Options getOptions() {
+    Option optStartRowInclusive =
+        new Option(OptUtil.START_ROW_OPT, "begin-row", true, "begin row (inclusive)");
     optStartRowExclusive = new Option("be", "begin-exclusive", false,
-        "make start row exclusive (by default it's inclusive)");
+        "make start row exclusive (by default it's inclusive");
     optStartRowExclusive.setArgName("begin-exclusive");
     optEndRowExclusive = new Option("ee", "end-exclusive", false,
         "make end row exclusive (by default it's inclusive)");
     optEndRowExclusive.setArgName("end-exclusive");
-    optRow = new Option("r", "range", true, "tablet range to modify");
-    optRow.setArgName("range");
-    goalOpt = new Option("g", "goal", true, "tablet hosting goal");
-    goalOpt.setArgName("goal");
-    goalOpt.setArgs(1);
-    goalOpt.setRequired(true);
+    optRow = new Option("r", "row", true, "tablet row to read");
+    optRow.setArgName("row");
 
     final Options opts = super.getOptions();
-    opts.addOption(OptUtil.startRowOpt());
+    opts.addOption(optStartRowInclusive);
     opts.addOption(optStartRowExclusive);
     opts.addOption(OptUtil.endRowOpt());
-    opts.addOption(optStartRowExclusive);
     opts.addOption(optEndRowExclusive);
     opts.addOption(optRow);
-    opts.addOption(goalOpt);
 
     return opts;
   }
+
 }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/TabletHostingGoalCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/SetTabletHostingGoalCommand.java
similarity index 91%
rename from shell/src/main/java/org/apache/accumulo/shell/commands/TabletHostingGoalCommand.java
rename to shell/src/main/java/org/apache/accumulo/shell/commands/SetTabletHostingGoalCommand.java
index fef60c6282..414d3f60a7 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/TabletHostingGoalCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/SetTabletHostingGoalCommand.java
@@ -31,7 +31,7 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.io.Text;
 
-public class TabletHostingGoalCommand extends TableOperation {
+public class SetTabletHostingGoalCommand extends TableOperation {
 
   private Option optRow;
   private Option optStartRowExclusive;
@@ -43,7 +43,7 @@ public class TabletHostingGoalCommand extends TableOperation {
 
   @Override
   public String getName() {
-    return "goal";
+    return "setgoal";
   }
 
   @Override
@@ -86,24 +86,26 @@ public class TabletHostingGoalCommand extends TableOperation {
 
   @Override
   public Options getOptions() {
+    Option optStartRowInclusive =
+        new Option(OptUtil.START_ROW_OPT, "begin-row", true, "begin row (inclusive)");
+    optStartRowInclusive.setArgName("begin-row");
     optStartRowExclusive = new Option("be", "begin-exclusive", false,
         "make start row exclusive (by default it's inclusive)");
     optStartRowExclusive.setArgName("begin-exclusive");
     optEndRowExclusive = new Option("ee", "end-exclusive", false,
         "make end row exclusive (by default it's inclusive)");
     optEndRowExclusive.setArgName("end-exclusive");
-    optRow = new Option("r", "range", true, "tablet range to modify");
-    optRow.setArgName("range");
+    optRow = new Option("r", "row", true, "tablet row to modify");
+    optRow.setArgName("row");
     goalOpt = new Option("g", "goal", true, "tablet hosting goal");
     goalOpt.setArgName("goal");
     goalOpt.setArgs(1);
     goalOpt.setRequired(true);
 
     final Options opts = super.getOptions();
-    opts.addOption(OptUtil.startRowOpt());
+    opts.addOption(optStartRowInclusive);
     opts.addOption(optStartRowExclusive);
     opts.addOption(OptUtil.endRowOpt());
-    opts.addOption(optStartRowExclusive);
     opts.addOption(optEndRowExclusive);
     opts.addOption(optRow);
     opts.addOption(goalOpt);
diff --git a/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java b/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
index 526db10bc6..94d41f82b4 100644
--- a/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
@@ -39,6 +39,7 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -50,15 +51,21 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.DiskUsage;
+import org.apache.accumulo.core.client.admin.HostingGoalForTablet;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.admin.TabletHostingGoal;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.constraints.DefaultKeySizeConstraint;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Authorizations;
@@ -368,4 +375,305 @@ public class TableOperationsIT extends AccumuloClusterHarness {
         "specified table does not exist");
   }
 
+  // This test will create a total of six tables.
+  // This test will create three tables with no additional parameters, i.e., no initial splits, etc.
+  // For each of the first three tablets, set ONDEMAND, ALWAYS, and NEVER as the HostingGoals,
+  // respectively.
+  // Retrieving the HostingGoals should return the above goals back in a single tablet.
+  //
+  // The other three tables will be created with initial splits and then queried for HostingGoals.
+  // For each table a list of tablets will be returned with the corresponding HostingGoal verified
+  // for correctness.
+  // The last three tables will also be queried for ranges within the table and only expect to see
+  // tablets with those ranges returned.
+  @Test
+  public void testGetHostingGoals_DefaultTableCreation() throws AccumuloException,
+      TableExistsException, AccumuloSecurityException, TableNotFoundException {
+
+    final String[] tableNames = getUniqueNames(6);
+    final String tableOnDemand = tableNames[0];
+    final String tableAlways = tableNames[1];
+    final String tableNever = tableNames[2];
+    final String tableOnDemandWithSplits = tableNames[3];
+    final String tableAlwaysWithSplits = tableNames[4];
+    final String tableNeverWithSplits = tableNames[5];
+
+    SortedSet<Text> splits =
+        Sets.newTreeSet(Arrays.asList(new Text("d"), new Text("m"), new Text("s")));
+    NewTableConfiguration ntc = new NewTableConfiguration();
+
+    try {
+      // create all the tables with initial hosting goals and splits
+      ntc = ntc.withInitialHostingGoal(TabletHostingGoal.ONDEMAND);
+      accumuloClient.tableOperations().create(tableOnDemand, ntc);
+
+      ntc = ntc.withInitialHostingGoal(TabletHostingGoal.ALWAYS);
+      accumuloClient.tableOperations().create(tableAlways, ntc);
+
+      ntc = ntc.withInitialHostingGoal(TabletHostingGoal.NEVER);
+      accumuloClient.tableOperations().create(tableNever, ntc);
+
+      ntc = ntc.withSplits(splits).withInitialHostingGoal(TabletHostingGoal.ONDEMAND);
+      accumuloClient.tableOperations().create(tableOnDemandWithSplits, ntc);
+
+      ntc = ntc.withSplits(splits).withInitialHostingGoal(TabletHostingGoal.ALWAYS);
+      accumuloClient.tableOperations().create(tableAlwaysWithSplits, ntc);
+
+      ntc = ntc.withSplits(splits).withInitialHostingGoal(TabletHostingGoal.NEVER);
+      accumuloClient.tableOperations().create(tableNeverWithSplits, ntc);
+
+      Map<String,String> idMap = accumuloClient.tableOperations().tableIdMap();
+
+      List<HostingGoalForTablet> expectedGoals = new ArrayList<>();
+      setExpectedGoal(expectedGoals, idMap.get(tableOnDemand), null, null,
+          TabletHostingGoal.ONDEMAND);
+      verifyTabletGoals(tableOnDemand, null, null, expectedGoals);
+
+      expectedGoals.clear();
+      setExpectedGoal(expectedGoals, idMap.get(tableAlways), null, null, TabletHostingGoal.ALWAYS);
+      verifyTabletGoals(tableAlways, null, null, expectedGoals);
+
+      expectedGoals.clear();
+      setExpectedGoal(expectedGoals, idMap.get(tableNever), null, null, TabletHostingGoal.NEVER);
+      verifyTabletGoals(tableNever, null, null, expectedGoals);
+
+      verifyTablesWithSplits(tableOnDemandWithSplits, idMap, splits, TabletHostingGoal.ONDEMAND);
+      verifyTablesWithSplits(tableAlwaysWithSplits, idMap, splits, TabletHostingGoal.ALWAYS);
+      verifyTablesWithSplits(tableNeverWithSplits, idMap, splits, TabletHostingGoal.NEVER);
+
+    } finally {
+      accumuloClient.tableOperations().delete(tableOnDemand);
+      accumuloClient.tableOperations().delete(tableAlways);
+      accumuloClient.tableOperations().delete(tableNever);
+      accumuloClient.tableOperations().delete(tableOnDemandWithSplits);
+      accumuloClient.tableOperations().delete(tableAlwaysWithSplits);
+      accumuloClient.tableOperations().delete(tableNeverWithSplits);
+    }
+  }
+
+  // This test creates a table with splits at creation time
+  // Once created, the four tablets are provided separate hosting goals.
+  // The test verifies that each tablet is assigned the correct hosting goal.
+  @Test
+  public void testGetHostingGoals_MixedGoals() throws AccumuloException, TableExistsException,
+      AccumuloSecurityException, TableNotFoundException {
+
+    String tableName = getUniqueNames(1)[0];
+    List<HostingGoalForTablet> expectedGoals;
+    SortedSet<Text> splits =
+        Sets.newTreeSet(Arrays.asList(new Text("d"), new Text("m"), new Text("s")));
+
+    try {
+      // create table with initial splits at creation time
+      NewTableConfiguration ntc = new NewTableConfiguration().withSplits(splits);
+      accumuloClient.tableOperations().create(tableName, ntc);
+
+      // set each tablet with a different goal and query to see if they are set accordingly
+      Range range = new Range(null, false, new Text("d"), true);
+      accumuloClient.tableOperations().setTabletHostingGoal(tableName, range,
+          TabletHostingGoal.NEVER);
+      range = new Range(new Text("m"), false, new Text("s"), true);
+      accumuloClient.tableOperations().setTabletHostingGoal(tableName, range,
+          TabletHostingGoal.ALWAYS);
+      range = new Range(new Text("s"), false, null, true);
+      accumuloClient.tableOperations().setTabletHostingGoal(tableName, range,
+          TabletHostingGoal.NEVER);
+
+      List<HostingGoalForTablet> hostingInfo = accumuloClient.tableOperations()
+          .getTabletHostingGoal(tableName, new Range()).collect(Collectors.toList());
+
+      Map<String,String> idMap = accumuloClient.tableOperations().tableIdMap();
+      expectedGoals = new ArrayList<>();
+      String tableId = idMap.get(tableName);
+      setExpectedGoal(expectedGoals, tableId, "d", null, TabletHostingGoal.NEVER);
+      // this range was intentionally not set above, checking that the tablet has the default
+      // hosting goal
+      setExpectedGoal(expectedGoals, tableId, "m", "d", TabletHostingGoal.ONDEMAND);
+      setExpectedGoal(expectedGoals, tableId, "s", "m", TabletHostingGoal.ALWAYS);
+      setExpectedGoal(expectedGoals, tableId, null, "s", TabletHostingGoal.NEVER);
+      assertEquals(4, hostingInfo.size());
+      hostingInfo.forEach(p -> assertTrue(expectedGoals.contains(p)));
+    } finally {
+      accumuloClient.tableOperations().delete(tableName);
+    }
+  }
+
+  // This tests creates a tables with initial splits and then queries getgoal using ranges that
+  // are not on split point boundaries
+  @Test
+  public void testGetHostingGoals_NonSplitBoundaries() throws AccumuloException,
+      TableExistsException, AccumuloSecurityException, TableNotFoundException {
+
+    String tableName = getUniqueNames(1)[0];
+    SortedSet<Text> splits =
+        Sets.newTreeSet(Arrays.asList(new Text("d"), new Text("m"), new Text("s")));
+    List<HostingGoalForTablet> expectedGoals = new ArrayList<>();
+    List<HostingGoalForTablet> hostingInfo;
+    Map<String,String> idMap;
+    String tableId;
+
+    try {
+      // create table with initial splits at creation time
+      NewTableConfiguration ntc = new NewTableConfiguration().withSplits(splits);
+      accumuloClient.tableOperations().create(tableName, ntc);
+
+      // set each different goal for each tablet and query to see if they are set accordingly
+      accumuloClient.tableOperations().setTabletHostingGoal(tableName, new Range(new Text("d")),
+          TabletHostingGoal.ALWAYS);
+      accumuloClient.tableOperations().setTabletHostingGoal(tableName, new Range(new Text("m")),
+          TabletHostingGoal.NEVER);
+      accumuloClient.tableOperations().setTabletHostingGoal(tableName, new Range(new Text("s")),
+          TabletHostingGoal.ALWAYS);
+
+      idMap = accumuloClient.tableOperations().tableIdMap();
+      tableId = idMap.get(tableName);
+
+      setExpectedGoal(expectedGoals, tableId, "d", null, TabletHostingGoal.ALWAYS);
+      // test using row as range constructor
+      hostingInfo = accumuloClient.tableOperations().getTabletHostingGoal(tableName, new Range("a"))
+          .collect(Collectors.toList());
+      assertEquals(1, hostingInfo.size());
+      hostingInfo.forEach(p -> assertTrue(expectedGoals.contains(p)));
+
+      // test using startRowInclusive set to true
+      Range range = new Range(new Text("c"), true, new Text("c"), true);
+      hostingInfo = accumuloClient.tableOperations().getTabletHostingGoal(tableName, range)
+          .collect(Collectors.toList());
+      assertEquals(1, hostingInfo.size());
+      hostingInfo.forEach(p -> assertTrue(expectedGoals.contains(p)));
+
+      expectedGoals.clear();
+      setExpectedGoal(expectedGoals, tableId, "m", "d", TabletHostingGoal.NEVER);
+      setExpectedGoal(expectedGoals, tableId, "s", "m", TabletHostingGoal.ALWAYS);
+
+      range = new Range(new Text("m"), new Text("p"));
+      hostingInfo = accumuloClient.tableOperations().getTabletHostingGoal(tableName, range)
+          .collect(Collectors.toList());
+      assertEquals(expectedGoals, hostingInfo);
+
+      expectedGoals.clear();
+      setExpectedGoal(expectedGoals, tableId, "d", null, TabletHostingGoal.ALWAYS);
+      setExpectedGoal(expectedGoals, tableId, "m", "d", TabletHostingGoal.NEVER);
+      setExpectedGoal(expectedGoals, tableId, "s", "m", TabletHostingGoal.ALWAYS);
+      setExpectedGoal(expectedGoals, tableId, null, "s", TabletHostingGoal.ONDEMAND);
+
+      range = new Range("b", false, "t", true);
+      hostingInfo = accumuloClient.tableOperations().getTabletHostingGoal(tableName, range)
+          .collect(Collectors.toList());
+      assertEquals(4, hostingInfo.size());
+      hostingInfo.forEach(p -> assertTrue(expectedGoals.contains(p)));
+
+    } finally {
+      accumuloClient.tableOperations().delete(tableName);
+    }
+  }
+
+  // This test creates a table with no initial splits. The splits are added after table creation.
+  // This test verifies that the existing hosting goal is properly propagated to the metadata table
+  // for
+  // each tablet.
+  @Test
+  public void testGetHostingGoals_DelayedSplits() throws AccumuloException, TableExistsException,
+      AccumuloSecurityException, TableNotFoundException {
+
+    String tableName = getUniqueNames(1)[0];
+
+    try {
+      accumuloClient.tableOperations().create(tableName);
+      Map<String,String> idMap = accumuloClient.tableOperations().tableIdMap();
+
+      // set goals to ALWAYS
+      accumuloClient.tableOperations().setTabletHostingGoal(tableName, new Range(),
+          TabletHostingGoal.ALWAYS);
+
+      List<HostingGoalForTablet> expectedGoals = new ArrayList<>();
+      String tableId = idMap.get(tableName);
+      setExpectedGoal(expectedGoals, tableId, null, null, TabletHostingGoal.ALWAYS);
+      List<HostingGoalForTablet> hostingInfo = accumuloClient.tableOperations()
+          .getTabletHostingGoal(tableName, new Range()).collect(Collectors.toList());
+      assertEquals(1, hostingInfo.size());
+      hostingInfo.forEach(p -> assertTrue(expectedGoals.contains(p)));
+
+      // Add splits after the fact
+      SortedSet<Text> splits =
+          Sets.newTreeSet(Arrays.asList(new Text("g"), new Text("n"), new Text("r")));
+      accumuloClient.tableOperations().addSplits(tableName, splits);
+
+      expectedGoals.clear();
+      hostingInfo.clear();
+      setExpectedGoal(expectedGoals, tableId, "g", null, TabletHostingGoal.ALWAYS);
+      setExpectedGoal(expectedGoals, tableId, "n", "g", TabletHostingGoal.ALWAYS);
+      setExpectedGoal(expectedGoals, tableId, "r", "n", TabletHostingGoal.ALWAYS);
+      setExpectedGoal(expectedGoals, tableId, null, "r", TabletHostingGoal.ALWAYS);
+
+      // Retrieve goals for table
+      hostingInfo.clear();
+      hostingInfo = accumuloClient.tableOperations().getTabletHostingGoal(tableName, new Range())
+          .collect(Collectors.toList());
+      assertEquals(4, hostingInfo.size());
+      hostingInfo.forEach(p -> assertTrue(expectedGoals.contains(p)));
+    } finally {
+      accumuloClient.tableOperations().delete(tableName);
+    }
+  }
+
+  private void verifyTablesWithSplits(String tableName, Map<String,String> idMap,
+      SortedSet<Text> splits, TabletHostingGoal goal) throws TableNotFoundException {
+
+    List<HostingGoalForTablet> expectedGoals = new ArrayList<>();
+    List<HostingGoalForTablet> hostingInfo;
+    String tableId = idMap.get(tableName);
+    String[] splitPts = splits.stream().map(Text::toString).toArray(String[]::new);
+
+    // retrieve all tablets for a table
+    setExpectedGoal(expectedGoals, tableId, splitPts[0], null, goal);
+    setExpectedGoal(expectedGoals, tableId, splitPts[1], splitPts[0], goal);
+    setExpectedGoal(expectedGoals, tableId, splitPts[2], splitPts[1], goal);
+    setExpectedGoal(expectedGoals, tableId, null, splitPts[2], goal);
+    // Retrieve goals for table
+    hostingInfo = accumuloClient.tableOperations().getTabletHostingGoal(tableName, new Range())
+        .collect(Collectors.toList());
+    assertEquals(4, hostingInfo.size());
+    hostingInfo.forEach(p -> assertTrue(expectedGoals.contains(p)));
+
+    // verify individual tablets can be retrieved
+    expectedGoals.clear();
+    setExpectedGoal(expectedGoals, tableId, splitPts[0], null, goal);
+    verifyTabletGoals(tableName, new Text(splitPts[0]), null, expectedGoals);
+
+    expectedGoals.clear();
+    setExpectedGoal(expectedGoals, tableId, splitPts[1], splitPts[0], goal);
+    verifyTabletGoals(tableName, new Text(splitPts[1]), new Text(splitPts[0]), expectedGoals);
+
+    expectedGoals.clear();
+    setExpectedGoal(expectedGoals, tableId, splitPts[2], splitPts[1], goal);
+    verifyTabletGoals(tableName, new Text(splitPts[2]), new Text(splitPts[1]), expectedGoals);
+
+    expectedGoals.clear();
+    setExpectedGoal(expectedGoals, tableId, null, splitPts[2], goal);
+    verifyTabletGoals(tableName, null, new Text(splitPts[2]), expectedGoals);
+
+    expectedGoals.clear();
+    setExpectedGoal(expectedGoals, tableId, splitPts[1], splitPts[0], goal);
+    setExpectedGoal(expectedGoals, tableId, splitPts[2], splitPts[1], goal);
+    verifyTabletGoals(tableName, new Text(splitPts[2]), new Text(splitPts[0]), expectedGoals);
+  }
+
+  private void verifyTabletGoals(String tableName, Text endRow, Text prevEndRow,
+      List<HostingGoalForTablet> expectedGoals) throws TableNotFoundException {
+
+    List<HostingGoalForTablet> hostingInfo;
+    Range range = new Range(prevEndRow, false, endRow, true);
+    hostingInfo = accumuloClient.tableOperations().getTabletHostingGoal(tableName, range)
+        .collect(Collectors.toList());
+    assertEquals(expectedGoals, hostingInfo);
+  }
+
+  private void setExpectedGoal(List<HostingGoalForTablet> expected, String id, String endRow,
+      String prevEndRow, TabletHostingGoal goal) {
+    KeyExtent ke = new KeyExtent(TableId.of(id), endRow == null ? null : new Text(endRow),
+        prevEndRow == null ? null : new Text(prevEndRow));
+    expected.add(new HostingGoalForTablet(new TabletIdImpl(ke), goal));
+  }
+
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
index 0c2a61d138..6ccd6b7f15 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
@@ -32,10 +32,13 @@ import static org.junit.jupiter.api.Assumptions.assumeFalse;
 import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
 import java.io.BufferedReader;
+import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -44,6 +47,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.SortedSet;
 import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.Constants;
@@ -53,6 +57,7 @@ import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.IteratorSetting.Column;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TabletHostingGoal;
 import org.apache.accumulo.core.client.sample.RowColumnSampler;
 import org.apache.accumulo.core.client.sample.RowSampler;
@@ -100,6 +105,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -1225,14 +1231,14 @@ public class ShellServerIT extends SharedMiniClusterBase {
   }
 
   @Test
-  public void goal() throws Exception {
+  public void testSetGoalCommand() throws Exception {
     final String table = getUniqueNames(1)[0];
     ts.exec("createtable " + table);
     ts.exec("addsplits -t " + table + " a c e g");
-    String result = ts.exec("goal -?");
+    String result = ts.exec("setgoal -?");
     assertTrue(result.contains("Sets the hosting goal"));
-    ts.exec("goal -t " + table + " -b a -e a -g never");
-    ts.exec("goal -t " + table + " -b c -e e -ee -g always");
+    ts.exec("setgoal -t " + table + " -b a -e a -g never");
+    ts.exec("setgoal -t " + table + " -b c -e e -ee -g always");
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build();
         Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
       String tableId = client.tableOperations().tableIdMap().get(table);
@@ -1242,11 +1248,13 @@ public class ShellServerIT extends SharedMiniClusterBase {
       for (Entry<Key,Value> e : s) {
         switch (e.getKey().getRow().toString()) {
           case "1;c":
-            assertEquals(TabletHostingGoal.NEVER.name(), e.getValue().toString());
-            break;
           case "1;e":
             assertEquals(TabletHostingGoal.ALWAYS.name(), e.getValue().toString());
             break;
+          case "1;a":
+            assertEquals(TabletHostingGoal.NEVER.name(), e.getValue().toString());
+            break;
+          case "1;g":
           case "1<":
             // this tablet was loaded ondemand when we executed
             // the addsplits command
@@ -1259,6 +1267,105 @@ public class ShellServerIT extends SharedMiniClusterBase {
     }
   }
 
+  @Test
+  public void testGetGoalCommand() throws IOException, TableNotFoundException {
+
+    SortedSet<Text> splits =
+        Sets.newTreeSet(Arrays.asList(new Text("d"), new Text("m"), new Text("s")));
+
+    final String tableName = getUniqueNames(1)[0];
+    java.nio.file.Path splitsFilePath = null;
+
+    try {
+      splitsFilePath = createSplitsFile("splitsFile", splits);
+
+      ts.exec("createtable " + tableName + " -sf " + splitsFilePath.toAbsolutePath(), true);
+      String result = ts.exec("getgoal -?");
+      assertTrue(result.contains("usage: getgoal"));
+
+      result = ts.exec("getgoal");
+      assertTrue(result.contains("TABLE: " + tableName));
+      assertTrue(result.contains("TABLET ID    HOSTING GOAL"));
+      assertTrue(result.contains("1;d<         ONDEMAND"));
+      assertTrue(result.contains("1;m;d        ONDEMAND"));
+      assertTrue(result.contains("1;s;m        ONDEMAND"));
+      assertTrue(result.contains("1<;s         ONDEMAND"));
+
+      ts.exec("setgoal -g ALWAYS -r p");
+      result = ts.exec("getgoal -r p");
+      assertFalse(result.contains("1;d<         ONDEMAND"));
+      assertFalse(result.contains("1;m;d        ONDEMAND"));
+      assertTrue(result.contains("1;s;m        ALWAYS"));
+      assertFalse(result.contains("1<;s         ONDEMAND"));
+
+      result = ts.exec("getgoal");
+      assertTrue(result.contains("1;d<         ONDEMAND"));
+      assertTrue(result.contains("1;m;d        ONDEMAND"));
+      assertTrue(result.contains("1;s;m        ALWAYS"));
+      assertTrue(result.contains("1<;s         ONDEMAND"));
+
+      result = ts.exec("getgoal -b f -e p");
+      assertFalse(result.contains("1;d<         ONDEMAND"));
+      assertTrue(result.contains("1;m;d        ONDEMAND"));
+      assertTrue(result.contains("1;s;m        ALWAYS"));
+      assertFalse(result.contains("1<;s         ONDEMAND"));
+
+    } finally {
+      if (splitsFilePath != null) {
+        Files.delete(splitsFilePath);
+      }
+    }
+  }
+
+  // Verify that when splits are added after table creation, hosting goals are set properly
+  @Test
+  public void testGetGoalCommand_DelayedSplits() throws IOException {
+
+    final String[] tableName = getUniqueNames(2);
+
+    ts.exec("createtable " + tableName[0], true);
+    String result = ts.exec("getgoal");
+    assertTrue(result.contains("TABLE: " + tableName[0]));
+    assertTrue(result.contains("TABLET ID    HOSTING GOAL"));
+    assertTrue(result.contains("1<<          ONDEMAND"));
+
+    // add the splits and check goals again
+    ts.exec("addsplits d m s", true);
+    result = ts.exec("getgoal");
+    assertTrue(result.contains("1;d<         ONDEMAND"));
+    assertTrue(result.contains("1;m;d        ONDEMAND"));
+    assertTrue(result.contains("1;s;m        ONDEMAND"));
+    assertTrue(result.contains("1<;s         ONDEMAND"));
+
+    // scan metadata table to be sure the hosting goals were properly set
+    result = ts.exec("scan -t accumulo.metadata -c hosting:goal", true);
+    assertTrue(result.contains("1;d hosting:goal []\tONDEMAND"));
+    assertTrue(result.contains("1;m hosting:goal []\tONDEMAND"));
+    assertTrue(result.contains("1;s hosting:goal []\tONDEMAND"));
+    assertTrue(result.contains("1< hosting:goal []\tONDEMAND"));
+
+    ts.exec("createtable " + tableName[1] + " -g always", true);
+    result = ts.exec("getgoal");
+    assertTrue(result.contains("TABLE: " + tableName[1]));
+    assertTrue(result.contains("TABLET ID    HOSTING GOAL"));
+    assertTrue(result.contains("2<<          ALWAYS"));
+
+    ts.exec("addsplits d m s", true);
+    result = ts.exec("getgoal");
+    assertTrue(result.contains("2;d<         ALWAYS"));
+    assertTrue(result.contains("2;m;d        ALWAYS"));
+    assertTrue(result.contains("2;s;m        ALWAYS"));
+    assertTrue(result.contains("2<;s         ALWAYS"));
+
+    // scan metadata table to be sure the hosting goals were properly set
+    result = ts.exec("scan -t accumulo.metadata -c hosting:goal -b 2", true);
+    assertTrue(result.contains("2;d hosting:goal []\tALWAYS"));
+    assertTrue(result.contains("2;m hosting:goal []\tALWAYS"));
+    assertTrue(result.contains("2;s hosting:goal []\tALWAYS"));
+    assertTrue(result.contains("2< hosting:goal []\tALWAYS"));
+
+  }
+
   @Test
   public void grep() throws Exception {
     final String table = getUniqueNames(1)[0];
@@ -2151,4 +2258,16 @@ public class ShellServerIT extends SharedMiniClusterBase {
     assertMatches(output, "(?sm).*^.*total[:]2[,]\\s+missing[:]0[,]\\s+extra[:]0.*$.*");
   }
 
+  private java.nio.file.Path createSplitsFile(final String splitsFile, final SortedSet<Text> splits)
+      throws IOException {
+    String fullSplitsFile = System.getProperty("user.dir") + "/target/" + splitsFile;
+    java.nio.file.Path path = Paths.get(fullSplitsFile);
+    try (BufferedWriter writer = Files.newBufferedWriter(path, UTF_8)) {
+      for (Text text : splits) {
+        writer.write(text.toString() + '\n');
+      }
+    }
+    return path;
+  }
+
 }