You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2021/06/10 19:50:00 UTC

[accumulo] branch main updated: Improve pattern matching and parameter validation (#2156)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new dad7e01  Improve pattern matching and parameter validation (#2156)
dad7e01 is described below

commit dad7e01ae7d450064cba5d60a1e0770311ebdb64
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Thu Jun 10 15:49:48 2021 -0400

    Improve pattern matching and parameter validation (#2156)
    
    Improved validation for table and namespace names
    
    * Distinguish between validating *existing* table or namespace names and
      *new* names. Existing names could exceed the max length, in which case
      we should issue a warning. New names, on the other hand, must be
      prevented from exceeding the max length and throw an exception
    * Be more rigorous in checking table name length by checking the
      namespace segment independently from the table segment
    * Check arguments in ClientContext consistently *after* ensureOpen()
    * Remove redundant checks for name validation when it is already
      validated another API method that is immediately called in the
      implementation
    * Update TableOperationsImpl and NamespaceOperationsImpl and
      ClientContext (impl for AccumuloClient) to validate namespace and
      table names
    * Incorporate null checks into the Validators so two checks aren't
      necessary
    * Remove validation from ConnectorImpl, because all those methods defer
      to ClientContext anyway, which already does validation
    
    Improvements to Validator guts
    
    * Consolidate Validators into a single class, and rewrite Validator
      class to more efficiently return the error string when there is an
      error, so that the reason for a validation failure are immediately
      known upon failure without executing the validation logic a second
      time to determine the error message
    * Cache the most recently validated object in each Validator so that
      validation can be short-circuited if unintentionally called multiple
      times in quick succession (like in an API that calls another API
      method and both do argument validation)
    * Move the details of the validation logic (the regular expressions,
      etc.) to private members of the Validators utility class, to keep
      validation implementation all in one place and remove any redundant
      implementations discovered in the code
    * Reuse as much code as possible, to ensure consistency in error
      messages and validation behavior
    * Add test cases for Validator implementations
    
    Other improvements
    
    * Avoid str.matches(pattern) in favor of precompiled patterns using
      Pattern.compile(pattern).matcher(str).matches() to improve performance
      in loops or other places where a pattern was used multiple times
    * Use Pattern.asMatchPredicate where it would make the code or loops
      more readable
    * Remove code from 8b54ced2777beb81fce04aa8300e192eb4534a36 added to
      version 1.6.0 to automatically map `!METADATA` to `accumulo.metadata`
      that never worked because the regex validation would have failed
      before that code was reached
    * Clean up redundant helper methods in FateServiceHandler
    
    This fixes #2151 (TableOperationsIT and NamespacesIT failures)
---
 .../client/summary/SummarizerConfiguration.java    |   7 +-
 .../accumulo/core/clientImpl/ClientContext.java    |  51 +----
 .../accumulo/core/clientImpl/ConnectorImpl.java    |  31 ----
 .../core/clientImpl/NamespaceOperationsImpl.java   |  66 ++-----
 .../accumulo/core/clientImpl/Namespaces.java       |  40 ----
 .../core/clientImpl/SecurityOperationsImpl.java    |  17 +-
 .../apache/accumulo/core/clientImpl/TableMap.java  |   3 +-
 .../core/clientImpl/TableOperationsHelper.java     |  42 +----
 .../core/clientImpl/TableOperationsImpl.java       | 203 +++++---------------
 .../apache/accumulo/core/clientImpl/Tables.java    |  14 +-
 .../accumulo/core/clientImpl/bulk/BulkImport.java  |   3 +-
 .../accumulo/core/metadata/MetadataTable.java      |   3 -
 .../org/apache/accumulo/core/util/Validator.java   | 116 ++++++------
 .../org/apache/accumulo/core/util/Validators.java  | 206 +++++++++++++++++++++
 .../apache/accumulo/core/util/ValidatorTest.java   |  77 ++++----
 .../apache/accumulo/core/util/ValidatorsTest.java  | 165 +++++++++++++++++
 .../accumulo/manager/FateServiceHandler.java       | 149 +++++----------
 .../accumulo/manager/util/TableValidators.java     | 133 -------------
 .../shell/commands/CreateTableCommand.java         |  12 +-
 .../apache/accumulo/shell/commands/DUCommand.java  |   9 +-
 .../shell/commands/ListCompactionsCommand.java     |   5 +-
 .../shell/commands/ListTabletsCommand.java         |   5 +-
 .../accumulo/shell/commands/TableOperation.java    |   9 +-
 .../start/classloader/AccumuloClassLoader.java     |   3 +-
 .../classloader/vfs/AccumuloVFSClassLoader.java    |   7 +-
 .../classloader/vfs/AccumuloClasspathTest.java     |  13 +-
 .../org/apache/accumulo/test/AuditMessageIT.java   |  15 +-
 .../org/apache/accumulo/test/ShellServerIT.java    |  11 +-
 .../accumulo/test/VerifySerialRecoveryIT.java      |   5 +-
 .../accumulo/test/functional/CloneTestIT.java      |   4 +-
 .../apache/accumulo/test/iterator/RegExTest.java   |  47 ++---
 31 files changed, 664 insertions(+), 807 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java
index 758891f..5f160ae 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/SummarizerConfiguration.java
@@ -26,6 +26,8 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.summary.SummarizerConfigurationUtil;
 
@@ -185,6 +187,7 @@ public class SummarizerConfiguration {
     private final String className;
     private final ImmutableMap.Builder<String,String> imBuilder = ImmutableMap.builder();
     private String configId = null;
+    private static final Predicate<String> ALPHANUM = Pattern.compile("\\w+").asMatchPredicate();
 
     private Builder(String className) {
       this.className = className;
@@ -201,7 +204,7 @@ public class SummarizerConfiguration {
      * @see SummarizerConfiguration#toTableProperties()
      */
     public Builder setPropertyId(String propId) {
-      Preconditions.checkArgument(propId.matches("\\w+"), "Config Id %s is not alphanum", propId);
+      Preconditions.checkArgument(ALPHANUM.test(propId), "Config Id %s is not alphanum", propId);
       this.configId = propId;
       return this;
     }
@@ -214,7 +217,7 @@ public class SummarizerConfiguration {
      * @see SummarizerConfiguration#getOptions()
      */
     public Builder addOption(String key, String value) {
-      Preconditions.checkArgument(key.matches("\\w+"), "Option Id %s is not alphanum", key);
+      Preconditions.checkArgument(ALPHANUM.test(key), "Option Id %s is not alphanum", key);
       imBuilder.put(key, value);
       return this;
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 60c0c5b..35623e7 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -22,6 +22,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
+import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
 
 import java.net.URL;
 import java.nio.file.Path;
@@ -98,7 +99,6 @@ public class ClientContext implements AccumuloClient {
   private final ClientInfo info;
   private String instanceId;
   private final ZooCache zooCache;
-  private static final String VALID_TABLENAME_REGEX = "^(\\w{1,1024}[.])?(\\w{1,1024})$";
 
   private Credentials creds;
   private BatchWriterConfig batchWriterConfig;
@@ -239,8 +239,8 @@ public class ClientContext implements AccumuloClient {
    * other auth token
    */
   public synchronized void setCredentials(Credentials newCredentials) {
-    checkArgument(newCredentials != null, "newCredentials is null");
     ensureOpen();
+    checkArgument(newCredentials != null, "newCredentials is null");
     creds = newCredentials;
     rpcCreds = null;
   }
@@ -509,12 +509,9 @@ public class ClientContext implements AccumuloClient {
     return zooCache;
   }
 
+  // this validates the table name for all callers
   TableId getTableId(String tableName) throws TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
-    TableId tableId = Tables.getTableId(this, tableName);
+    TableId tableId = Tables.getTableId(this, EXISTING_TABLE_NAME.validate(tableName));
     if (Tables.getTableState(this, tableId) == TableState.OFFLINE)
       throw new TableOfflineException(Tables.getTableOfflineMsg(this, tableId));
     return tableId;
@@ -523,13 +520,8 @@ public class ClientContext implements AccumuloClient {
   @Override
   public BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
       int numQueryThreads) throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(authorizations != null, "authorizations is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     ensureOpen();
+    checkArgument(authorizations != null, "authorizations is null");
     return new TabletServerBatchReader(this, getTableId(tableName), authorizations,
         numQueryThreads);
   }
@@ -537,10 +529,10 @@ public class ClientContext implements AccumuloClient {
   @Override
   public BatchScanner createBatchScanner(String tableName, Authorizations authorizations)
       throws TableNotFoundException {
+    ensureOpen();
     Integer numQueryThreads =
         ClientProperty.BATCH_SCANNER_NUM_QUERY_THREADS.getInteger(getProperties());
     Objects.requireNonNull(numQueryThreads);
-    ensureOpen();
     return createBatchScanner(tableName, authorizations, numQueryThreads);
   }
 
@@ -554,13 +546,8 @@ public class ClientContext implements AccumuloClient {
   @Override
   public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
       int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(authorizations != null, "authorizations is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     ensureOpen();
+    checkArgument(authorizations != null, "authorizations is null");
     return new TabletServerBatchDeleter(this, getTableId(tableName), authorizations,
         numQueryThreads, config.merge(getBatchWriterConfig()));
   }
@@ -575,11 +562,6 @@ public class ClientContext implements AccumuloClient {
   @Override
   public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
       throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     ensureOpen();
     // we used to allow null inputs for bw config
     if (config == null) {
@@ -590,10 +572,6 @@ public class ClientContext implements AccumuloClient {
 
   @Override
   public BatchWriter createBatchWriter(String tableName) throws TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     return createBatchWriter(tableName, new BatchWriterConfig());
   }
 
@@ -611,10 +589,6 @@ public class ClientContext implements AccumuloClient {
   @Override
   public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config)
       throws TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     ensureOpen();
     if (config == null) {
       config = new ConditionalWriterConfig();
@@ -632,13 +606,8 @@ public class ClientContext implements AccumuloClient {
   @Override
   public Scanner createScanner(String tableName, Authorizations authorizations)
       throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(authorizations != null, "authorizations is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     ensureOpen();
+    checkArgument(authorizations != null, "authorizations is null");
     Scanner scanner = new ScannerImpl(this, getTableId(tableName), authorizations);
     Integer batchSize = ClientProperty.SCANNER_BATCH_SIZE.getInteger(getProperties());
     if (batchSize != null) {
@@ -650,10 +619,6 @@ public class ClientContext implements AccumuloClient {
   @Override
   public Scanner createScanner(String tableName)
       throws TableNotFoundException, AccumuloSecurityException, AccumuloException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     Authorizations auths = securityOperations().getUserAuthorizations(getPrincipal());
     return createScanner(tableName, auths);
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
index 3ffe85f..31b5b3d 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
@@ -54,7 +54,6 @@ public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
   private static final String SYSTEM_TOKEN_NAME =
       "org.apache.accumulo.server.security.SystemCredentials$SystemToken";
   private final ClientContext context;
-  private static final String VALID_TABLENAME_REGEX = "^(\\w{1,1024}[.])?(\\w{1,1024})$";
 
   public ConnectorImpl(ClientContext context) throws AccumuloSecurityException, AccumuloException {
     this.context = context;
@@ -86,10 +85,6 @@ public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
   @Override
   public BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
       int numQueryThreads) throws TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     return context.createBatchScanner(tableName, authorizations, numQueryThreads);
   }
 
@@ -97,12 +92,7 @@ public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
   public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
       int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
       throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
     checkArgument(authorizations != null, "authorizations is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     return new TabletServerBatchDeleter(context, context.getTableId(tableName), authorizations,
         numQueryThreads, new BatchWriterConfig().setMaxMemory(maxMemory)
             .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
@@ -111,21 +101,12 @@ public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
   @Override
   public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
       int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     return context.createBatchDeleter(tableName, authorizations, numQueryThreads, config);
   }
 
   @Override
   public BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
       int maxWriteThreads) throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     return new BatchWriterImpl(context, context.getTableId(tableName),
         new BatchWriterConfig().setMaxMemory(maxMemory)
             .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
@@ -134,10 +115,6 @@ public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
   @Override
   public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
       throws TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     return context.createBatchWriter(tableName, config);
   }
 
@@ -156,20 +133,12 @@ public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
   @Override
   public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config)
       throws TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     return context.createConditionalWriter(tableName, config);
   }
 
   @Override
   public Scanner createScanner(String tableName, Authorizations authorizations)
       throws TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     return context.createScanner(tableName, authorizations);
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
index a33482c..c95cc87 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
@@ -20,6 +20,8 @@ package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.Validators.EXISTING_NAMESPACE_NAME;
+import static org.apache.accumulo.core.util.Validators.NEW_NAMESPACE_NAME;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
@@ -63,7 +65,6 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   private TableOperationsImpl tableOps;
 
   private static final Logger log = LoggerFactory.getLogger(TableOperations.class);
-  private static final String VALID_NAMESPACE_REGEX = "^\\w{0,1024}$";
 
   public NamespaceOperationsImpl(ClientContext context, TableOperationsImpl tableOps) {
     checkArgument(context != null, "context is null");
@@ -94,10 +95,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
 
   @Override
   public boolean exists(String namespace) {
-    checkArgument(namespace != null, "namespace is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
 
     OpTimer timer = null;
 
@@ -121,10 +119,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   @Override
   public void create(String namespace)
       throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
-    checkArgument(namespace != null, "namespace is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    NEW_NAMESPACE_NAME.validate(namespace);
 
     try {
       doNamespaceFateOperation(FateOperation.NAMESPACE_CREATE,
@@ -139,10 +134,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   @Override
   public void delete(String namespace) throws AccumuloException, AccumuloSecurityException,
       NamespaceNotFoundException, NamespaceNotEmptyException {
-    checkArgument(namespace != null, "namespace is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
 
     NamespaceId namespaceId = Namespaces.getNamespaceId(context, namespace);
     if (namespaceId.equals(Namespace.ACCUMULO.id()) || namespaceId.equals(Namespace.DEFAULT.id())) {
@@ -172,13 +164,8 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   public void rename(String oldNamespaceName, String newNamespaceName)
       throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
       NamespaceExistsException {
-    checkArgument(oldNamespaceName.matches(VALID_NAMESPACE_REGEX),
-        "oldNamespaceName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
-    checkArgument(newNamespaceName.matches(VALID_NAMESPACE_REGEX),
-        "newNamespaceName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_NAMESPACE_NAME.validate(oldNamespaceName);
+    NEW_NAMESPACE_NAME.validate(newNamespaceName);
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes(UTF_8)),
         ByteBuffer.wrap(newNamespaceName.getBytes(UTF_8)));
@@ -189,12 +176,9 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   @Override
   public void setProperty(final String namespace, final String property, final String value)
       throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    checkArgument(namespace != null, "namespace is null");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
     checkArgument(property != null, "property is null");
     checkArgument(value != null, "value is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     ManagerClient.executeNamespace(context,
         client -> client.setNamespaceProperty(TraceUtil.traceInfo(), context.rpcCreds(), namespace,
@@ -205,11 +189,8 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   @Override
   public void removeProperty(final String namespace, final String property)
       throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    checkArgument(namespace != null, "namespace is null");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
     checkArgument(property != null, "property is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     ManagerClient.executeNamespace(context, client -> client
         .removeNamespaceProperty(TraceUtil.traceInfo(), context.rpcCreds(), namespace, property));
@@ -219,10 +200,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   @Override
   public Map<String,String> getConfiguration(final String namespace)
       throws AccumuloException, NamespaceNotFoundException {
-    checkArgument(namespace != null, "namespace is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
 
     try {
       return ServerClient.executeRaw(context, client -> client
@@ -254,12 +232,9 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   public boolean testClassLoad(final String namespace, final String className,
       final String asTypeName)
       throws NamespaceNotFoundException, AccumuloException, AccumuloSecurityException {
-    checkArgument(namespace != null, "namespace is null");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
     checkArgument(className != null, "className is null");
     checkArgument(asTypeName != null, "asTypeName is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     try {
       return ServerClient.executeRaw(context,
@@ -285,10 +260,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   public void attachIterator(String namespace, IteratorSetting setting,
       EnumSet<IteratorScope> scopes)
       throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
+    // testClassLoad validates the namespace name
     testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
     super.attachIterator(namespace, setting, scopes);
   }
@@ -296,10 +268,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   @Override
   public int addConstraint(String namespace, String constraintClassName)
       throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
+    // testClassLoad validates the namespace name
     testClassLoad(namespace, constraintClassName, Constraint.class.getName());
     return super.addConstraint(namespace, constraintClassName);
   }
@@ -307,10 +276,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   private String doNamespaceFateOperation(FateOperation op, List<ByteBuffer> args,
       Map<String,String> opts, String namespace) throws AccumuloSecurityException,
       AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
+    // caller should validate the namespace name
     try {
       return tableOps.doFateOperation(op, args, opts, namespace);
     } catch (TableExistsException | TableNotFoundException e) {
@@ -321,9 +287,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
 
   private void checkLocalityGroups(String namespace, String propChanged)
       throws AccumuloException, NamespaceNotFoundException {
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
 
     if (LocalityGroupUtil.isLocalityGroupProperty(propChanged)) {
       Map<String,String> allProps = getConfiguration(namespace);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespaces.java
index 74d3411..c8a8a42 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespaces.java
@@ -32,7 +32,6 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.util.Validator;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,45 +39,6 @@ import org.slf4j.LoggerFactory;
 public class Namespaces {
   private static final Logger log = LoggerFactory.getLogger(Namespaces.class);
 
-  public static final String VALID_NAMESPACE_REGEX = "^\\w{0,1024}$";
-  public static final Validator<String> VALID_NAME = new Validator<>() {
-    @Override
-    public boolean test(String namespace) {
-      return namespace != null && namespace.matches(VALID_NAMESPACE_REGEX);
-    }
-
-    @Override
-    public String invalidMessage(String namespace) {
-      if (namespace == null)
-        return "Namespace cannot be null";
-      return "Namespaces must only contain word characters (letters, digits, and underscores): "
-          + namespace;
-    }
-  };
-  public static final Validator<String> NOT_DEFAULT = new Validator<>() {
-    @Override
-    public boolean test(String namespace) {
-      return !Namespace.DEFAULT.name().equals(namespace);
-    }
-
-    @Override
-    public String invalidMessage(String namespace) {
-      return "Namespace cannot be the reserved empty namespace";
-    }
-  };
-
-  public static final Validator<String> NOT_ACCUMULO = new Validator<>() {
-    @Override
-    public boolean test(String namespace) {
-      return !Namespace.ACCUMULO.name().equals(namespace);
-    }
-
-    @Override
-    public String invalidMessage(String namespace) {
-      return "Namespace cannot be the reserved namespace, " + Namespace.ACCUMULO.name();
-    }
-  };
-
   public static boolean exists(ClientContext context, NamespaceId namespaceId) {
     ZooCache zc = context.getZooCache();
     List<String> namespaceIds = zc.getChildren(context.getZooKeeperRoot() + Constants.ZNAMESPACES);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/SecurityOperationsImpl.java
index fabb446..8346058 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/SecurityOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/SecurityOperationsImpl.java
@@ -20,6 +20,7 @@ package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST;
+import static org.apache.accumulo.core.util.Validators.EXISTING_NAMESPACE_NAME;
 
 import java.nio.ByteBuffer;
 import java.util.Set;
@@ -49,7 +50,6 @@ import org.apache.accumulo.core.util.ByteBufferUtil;
 public class SecurityOperationsImpl implements SecurityOperations {
 
   private final ClientContext context;
-  private static final String VALID_NAMESPACE_REGEX = "^\\w{0,1024}$";
 
   private void executeVoid(ClientExec<ClientService.Client> exec)
       throws AccumuloException, AccumuloSecurityException {
@@ -202,11 +202,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
   public boolean hasNamespacePermission(final String principal, final String namespace,
       final NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
     checkArgument(principal != null, "principal is null");
-    checkArgument(namespace != null, "namespace is null");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
     checkArgument(permission != null, "permission is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     return execute(client -> client.hasNamespacePermission(TraceUtil.traceInfo(),
         context.rpcCreds(), principal, namespace, permission.getId()));
@@ -244,11 +241,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
   public void grantNamespacePermission(final String principal, final String namespace,
       final NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
     checkArgument(principal != null, "principal is null");
-    checkArgument(namespace != null, "namespace is null");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
     checkArgument(permission != null, "permission is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     executeVoid(client -> client.grantNamespacePermission(TraceUtil.traceInfo(), context.rpcCreds(),
         principal, namespace, permission.getId()));
@@ -286,11 +280,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
   public void revokeNamespacePermission(final String principal, final String namespace,
       final NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
     checkArgument(principal != null, "principal is null");
-    checkArgument(namespace != null, "namespace is null");
+    EXISTING_NAMESPACE_NAME.validate(namespace);
     checkArgument(permission != null, "permission is null");
-    checkArgument(namespace.matches(VALID_NAMESPACE_REGEX),
-        "namespace must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     executeVoid(client -> client.revokeNamespacePermission(TraceUtil.traceInfo(),
         context.rpcCreds(), principal, namespace, permission.getId()));
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableMap.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableMap.java
index 93fb618..4bee221 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableMap.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableMap.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.clientImpl.Tables.qualified;
 
 import java.util.HashMap;
 import java.util.List;
@@ -93,7 +92,7 @@ public class TableMap {
         }
       }
       if (tableName != null && namespaceName != null) {
-        String tableNameStr = qualified(new String(tableName, UTF_8), namespaceName);
+        String tableNameStr = Tables.qualified(new String(tableName, UTF_8), namespaceName);
         TableId tableId = TableId.of(tableIdStr);
         tableNameToIdBuilder.put(tableNameStr, tableId);
         tableIdToNameBuilder.put(tableId, tableNameStr);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java
index 9843346..65b19f0 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsHelper.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
 
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -37,15 +38,9 @@ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 
 public abstract class TableOperationsHelper implements TableOperations {
 
-  public static final String VALID_TABLENAME_REGEX = "^(\\w{1,1024}[.])?(\\w{1,1024})$";
-
   @Override
   public void attachIterator(String tableName, IteratorSetting setting)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     attachIterator(tableName, setting, EnumSet.allOf(IteratorScope.class));
   }
 
@@ -53,12 +48,9 @@ public abstract class TableOperationsHelper implements TableOperations {
   public void attachIterator(String tableName, IteratorSetting setting,
       EnumSet<IteratorScope> scopes)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(setting != null, "setting is null");
     checkArgument(scopes != null, "scopes is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
     checkIteratorConflicts(tableName, setting, scopes);
 
     for (IteratorScope scope : scopes) {
@@ -74,9 +66,7 @@ public abstract class TableOperationsHelper implements TableOperations {
   @Override
   public void removeIterator(String tableName, String name, EnumSet<IteratorScope> scopes)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     Map<String,String> copy = Map.copyOf(this.getConfiguration(tableName));
     for (IteratorScope scope : scopes) {
@@ -92,12 +82,9 @@ public abstract class TableOperationsHelper implements TableOperations {
   @Override
   public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope)
       throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(name != null, "name is null");
     checkArgument(scope != null, "scope is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     int priority = -1;
     String classname = null;
@@ -127,9 +114,7 @@ public abstract class TableOperationsHelper implements TableOperations {
   @Override
   public Map<String,EnumSet<IteratorScope>> listIterators(String tableName)
       throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     Map<String,EnumSet<IteratorScope>> result = new TreeMap<>();
     for (Entry<String,String> property : this.getProperties(tableName)) {
@@ -189,10 +174,7 @@ public abstract class TableOperationsHelper implements TableOperations {
   @Override
   public void checkIteratorConflicts(String tableName, IteratorSetting setting,
       EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     Map<String,String> iteratorProps = Map.copyOf(this.getConfiguration(tableName));
     checkIteratorConflicts(iteratorProps, setting, scopes);
@@ -201,9 +183,7 @@ public abstract class TableOperationsHelper implements TableOperations {
   @Override
   public int addConstraint(String tableName, String constraintClassName)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TreeSet<Integer> constraintNumbers = new TreeSet<>();
     TreeMap<String,Integer> constraintClasses = new TreeMap<>();
@@ -234,19 +214,13 @@ public abstract class TableOperationsHelper implements TableOperations {
   @Override
   public void removeConstraint(String tableName, int number)
       throws AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     this.removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
   }
 
   @Override
   public Map<String,Integer> listConstraints(String tableName)
       throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     Map<String,Integer> constraints = new TreeMap<>();
     for (Entry<String,String> property : this.getProperties(tableName)) {
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 52ae9e2..d189de8 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
@@ -26,6 +26,8 @@ import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toSet;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
+import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
+import static org.apache.accumulo.core.util.Validators.NEW_TABLE_NAME;
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.BufferedReader;
@@ -155,7 +157,6 @@ import com.google.common.base.Preconditions;
 public class TableOperationsImpl extends TableOperationsHelper {
 
   public static final String CLONE_EXCLUDE_PREFIX = "!";
-  public static final String VALID_TABLENAME_REGEX = "^(\\w{1,1024}[.])?(\\w{1,1024})$";
 
   private static final Logger log = LoggerFactory.getLogger(TableOperations.class);
   private final ClientContext context;
@@ -188,10 +189,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   @Override
   public boolean exists(String tableName) {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME))
       return true;
@@ -217,21 +215,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void create(String tableName)
       throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
     create(tableName, new NewTableConfiguration());
   }
 
   @Override
   public void create(String tableName, NewTableConfiguration ntc)
       throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    checkArgument(tableName != null, "tableName is null");
+    NEW_TABLE_NAME.validate(tableName);
     checkArgument(ntc != null, "ntc is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     List<ByteBuffer> args = new ArrayList<>();
     args.add(ByteBuffer.wrap(tableName.getBytes(UTF_8)));
@@ -346,9 +337,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   public String doBulkFateOperation(List<ByteBuffer> args, String tableName)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     try {
       return doFateOperation(FateOperation.TABLE_BULK_IMPORT2, args, Collections.emptyMap(),
@@ -485,9 +474,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void addSplits(String tableName, SortedSet<Text> partitionKeys)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = Tables.getTableId(context, tableName);
     List<Text> splits = new ArrayList<>(partitionKeys);
@@ -542,9 +529,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   private void addSplits(String tableName, SortedSet<Text> partitionKeys, TableId tableId)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
       AccumuloServerException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TabletLocator tabLocator = TabletLocator.getLocator(context, tableId);
     for (Text split : partitionKeys) {
@@ -632,10 +617,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void merge(String tableName, Text start, Text end)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     ByteBuffer EMPTY = ByteBuffer.allocate(0);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)),
@@ -654,10 +636,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void deleteRows(String tableName, Text start, Text end)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     ByteBuffer EMPTY = ByteBuffer.allocate(0);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)),
@@ -676,19 +655,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Collection<Text> listSplits(String tableName)
       throws TableNotFoundException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
+    // tableName is validated in _listSplits
     return _listSplits(tableName);
   }
 
   private List<Text> _listSplits(String tableName)
       throws TableNotFoundException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = Tables.getTableId(context, tableName);
     TreeMap<KeyExtent,String> tabletLocations = new TreeMap<>();
@@ -726,10 +699,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Collection<Text> listSplits(String tableName, int maxSplits)
       throws TableNotFoundException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
+    // tableName is validated in _listSplits
     List<Text> endRows = _listSplits(tableName);
     if (endRows.size() <= maxSplits)
       return endRows;
@@ -753,10 +723,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void delete(String tableName)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)));
     Map<String,String> opts = new HashMap<>();
@@ -784,16 +751,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void clone(String srcTableName, String newTableName, CloneConfiguration config)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
       TableExistsException {
-
-    checkArgument(srcTableName != null, "srcTableName is null");
-    checkArgument(newTableName != null, "newTableName is null");
-    checkArgument(srcTableName.matches(VALID_TABLENAME_REGEX),
-        "srcTableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
-    checkArgument(newTableName.matches(VALID_TABLENAME_REGEX),
-        "newTableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(srcTableName);
+    NEW_TABLE_NAME.validate(newTableName);
 
     TableId srcTableId = Tables.getTableId(context, srcTableName);
 
@@ -829,13 +788,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException,
       TableNotFoundException, AccumuloException, TableExistsException {
-    checkArgument(oldTableName.matches(VALID_TABLENAME_REGEX),
-        "oldTableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
-    checkArgument(newTableName.matches(VALID_TABLENAME_REGEX),
-        "newTableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(oldTableName);
+    NEW_TABLE_NAME.validate(newTableName);
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(UTF_8)),
         ByteBuffer.wrap(newTableName.getBytes(UTF_8)));
@@ -846,10 +800,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   @Override
   public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
+    // tableName is validated in the flush method being called below
     try {
       flush(tableName, null, null, false);
     } catch (TableNotFoundException e) {
@@ -860,11 +811,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void flush(String tableName, Text start, Text end, boolean wait)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
+    EXISTING_TABLE_NAME.validate(tableName);
     TableId tableId = Tables.getTableId(context, tableName);
     _flush(tableId, start, end, wait);
   }
@@ -886,10 +833,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void compact(String tableName, CompactionConfig config)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     // Ensure compaction iterators exist on a tabletserver
     final String skviName = SortedKeyValueIterator.class.getName();
@@ -957,9 +901,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void cancelCompaction(String tableName)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = Tables.getTableId(context, tableName);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.canonical().getBytes(UTF_8)));
@@ -1045,12 +987,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void setProperty(final String tableName, final String property, final String value)
       throws AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(property != null, "property is null");
     checkArgument(value != null, "value is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     try {
       setPropertyNoChecks(tableName, property, value);
@@ -1071,11 +1010,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void removeProperty(final String tableName, final String property)
       throws AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(property != null, "property is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     try {
       removePropertyNoChecks(tableName, property);
@@ -1112,10 +1048,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Map<String,String> getConfiguration(final String tableName)
       throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     try {
       return ServerClient.executeRaw(context, client -> client
@@ -1199,11 +1132,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(range != null, "range is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     if (maxSplits < 1)
       throw new IllegalArgumentException("maximum splits must be >= 1");
@@ -1293,12 +1223,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Deprecated(since = "2.0.0")
   public void importDirectory(String tableName, String dir, String failureDir, boolean setTime)
       throws IOException, AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(dir != null, "dir is null");
     checkArgument(failureDir != null, "failureDir is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     // check for table existence
     Tables.getTableId(context, tableName);
@@ -1423,10 +1350,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void offline(String tableName, boolean wait)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = Tables.getTableId(context, tableName);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.canonical().getBytes(UTF_8)));
@@ -1446,10 +1370,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   @Override
   public boolean isOnline(String tableName) throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = Tables.getTableId(context, tableName);
     TableState expectedState = Tables.getTableState(context, tableId, true);
@@ -1465,10 +1386,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void online(String tableName, boolean wait)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TableId tableId = Tables.getTableId(context, tableName);
     /**
@@ -1497,10 +1415,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   @Override
   public void clearLocatorCache(String tableName) throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "Table name must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     TabletLocator tabLocator =
         TabletLocator.getLocator(context, Tables.getTableId(context, tableName));
@@ -1518,10 +1433,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow,
       boolean startInclusive, Text endRow, boolean endInclusive) throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     Scanner scanner = context.createScanner(tableName, auths);
     return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
@@ -1645,11 +1557,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void importTable(String tableName, Set<String> importDirs)
       throws TableExistsException, AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(importDirs != null, "importDir is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     Set<String> checkedImportDirs = new HashSet<>();
     try {
@@ -1706,11 +1615,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void exportTable(String tableName, String exportDir)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(exportDir != null, "exportDir is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)),
         ByteBuffer.wrap(exportDir.getBytes(UTF_8)));
@@ -1729,12 +1635,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public boolean testClassLoad(final String tableName, final String className,
       final String asTypeName)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
+    EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(className != null, "className is null");
     checkArgument(asTypeName != null, "asTypeName is null");
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
 
     try {
       return ServerClient.executeRaw(context,
@@ -1799,9 +1702,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   private void clearSamplerOptions(String tableName)
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     String prefix = Property.TABLE_SAMPLER_OPTS.getKey();
     for (Entry<String,String> entry : getProperties(tableName)) {
@@ -1815,9 +1716,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void setSamplerConfiguration(String tableName, SamplerConfiguration samplerConfiguration)
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     clearSamplerOptions(tableName);
     List<Pair<String,String>> props =
@@ -1830,9 +1729,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void clearSamplerConfiguration(String tableName)
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     removeProperty(tableName, Property.TABLE_SAMPLER.getKey());
     clearSamplerOptions(tableName);
@@ -1841,9 +1738,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public SamplerConfiguration getSamplerConfiguration(String tableName)
       throws TableNotFoundException, AccumuloException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     AccumuloConfiguration conf = new ConfigurationCopy(this.getProperties(tableName));
     SamplerConfigurationImpl sci = SamplerConfigurationImpl.newSamplerConfig(conf);
@@ -1915,10 +1810,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Locations locate(String tableName, Collection<Range> ranges)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-    requireNonNull(tableName, "tableName must be non null");
+    EXISTING_TABLE_NAME.validate(tableName);
     requireNonNull(ranges, "ranges must be non null");
 
     TableId tableId = Tables.getTableId(context, tableName);
@@ -1962,9 +1854,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   @Override
   public SummaryRetriever summaries(String tableName) {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     return new SummaryRetriever() {
       private Text startRow = null;
@@ -2067,9 +1957,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void addSummarizers(String tableName, SummarizerConfiguration... newConfigs)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     HashSet<SummarizerConfiguration> currentConfigs =
         new HashSet<>(SummarizerConfiguration.fromTableProperties(getProperties(tableName)));
@@ -2095,9 +1983,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void removeSummarizers(String tableName, Predicate<SummarizerConfiguration> predicate)
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
+    EXISTING_TABLE_NAME.validate(tableName);
 
     Collection<SummarizerConfiguration> summarizerConfigs =
         SummarizerConfiguration.fromTableProperties(getProperties(tableName));
@@ -2115,10 +2001,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public List<SummarizerConfiguration> listSummarizers(String tableName)
       throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "tableName must only contain word characters (letters, digits, and underscores)"
-            + " and cannot exceed 1024 characters");
-
+    EXISTING_TABLE_NAME.validate(tableName);
     return new ArrayList<>(SummarizerConfiguration.fromTableProperties(getProperties(tableName)));
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Tables.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Tables.java
index e49efb8..2c3a64b 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Tables.java
@@ -20,6 +20,7 @@ package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
 
 import java.security.SecurityPermission;
 import java.util.List;
@@ -33,7 +34,6 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonService;
 import org.apache.accumulo.core.util.Pair;
@@ -45,10 +45,9 @@ import com.google.common.cache.CacheBuilder;
 
 public class Tables {
 
-  public static final String VALID_TABLENAME_REGEX = "^(\\w{1,1024}[.])?(\\w{1,1024})$";
-
   private static final SecurityPermission TABLES_PERMISSION =
       new SecurityPermission("tablesPermission");
+
   // Per instance cache will expire after 10 minutes in case we
   // encounter an instance not used frequently
   private static Cache<String,TableMap> instanceToMapCache =
@@ -275,7 +274,6 @@ public class Tables {
 
   public static String qualified(String tableName, String defaultNamespace) {
     Pair<String,String> qualifiedTableName = qualify(tableName, defaultNamespace);
-
     if (Namespace.DEFAULT.name().equals(qualifiedTableName.getFirst()))
       return qualifiedTableName.getSecond();
     else
@@ -286,12 +284,8 @@ public class Tables {
     return qualify(tableName, Namespace.DEFAULT.name());
   }
 
-  public static Pair<String,String> qualify(String tableName, String defaultNamespace) {
-    checkArgument(tableName.matches(VALID_TABLENAME_REGEX),
-        "Invalid table name '" + tableName + "'");
-
-    if (MetadataTable.OLD_NAME.equals(tableName))
-      tableName = MetadataTable.NAME;
+  private static Pair<String,String> qualify(String tableName, String defaultNamespace) {
+    EXISTING_TABLE_NAME.validate(tableName);
     if (tableName.contains(".")) {
       String[] s = tableName.split("\\.", 2);
       return new Pair<>(s[0], s[1]);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
index 995b73e..9a97e75 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
@@ -23,6 +23,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.stream.Collectors.groupingBy;
 import static org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.pathToCacheId;
+import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -249,7 +250,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
 
   @Override
   public ImportMappingOptions to(String tableName) {
-    this.tableName = Objects.requireNonNull(tableName);
+    this.tableName = EXISTING_TABLE_NAME.validate(tableName);
     return this;
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
index 850607b..d222d3f 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
@@ -22,9 +22,6 @@ import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.data.TableId;
 
 public class MetadataTable {
-
-  public static final String OLD_NAME = "!METADATA";
-
   public static final TableId ID = TableId.of("!0");
   public static final String NAME = Namespace.ACCUMULO.name() + ".metadata";
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Validator.java b/core/src/main/java/org/apache/accumulo/core/util/Validator.java
index 5e9fdd3..c34a919 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Validator.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Validator.java
@@ -18,16 +18,40 @@
  */
 package org.apache.accumulo.core.util;
 
-import java.util.function.Predicate;
+import static java.util.Objects.requireNonNull;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.function.Function;
 
 /**
- * A class that validates arguments of a particular type. Implementations must implement
- * {@link #test(Object)} and should override {@link #invalidMessage(Object)}.
+ * A class that validates arguments of a specified generic type. Given a validation function that
+ * emits an error message if and only if the validation fails, this object's validate method will
+ * return the original valid argument, or throw an IllegalArgumentException with the custom error
+ * message if it fails to validate.
  */
-public abstract class Validator<T> implements Predicate<T> {
+public class Validator<T> {
+
+  public static final Optional<String> OK = Optional.empty();
+
+  private final Function<T,Optional<String>> validateFunction;
+
+  private volatile T lastValidated = null;
 
   /**
-   * Validates an argument.
+   * Constructor to build a validator given the mapping function that validates. If the argument is
+   * valid, the mapping function should return an empty Optional. Otherwise, it should return an
+   * Optional containing the error message to be set in the IllegalArgumentException.
+   *
+   * @param validateFunction
+   *          the function that validates or returns an error message
+   */
+  public Validator(final Function<T,Optional<String>> validateFunction) {
+    this.validateFunction = requireNonNull(validateFunction);
+  }
+
+  /**
+   * Validates the provided argument.
    *
    * @param argument
    *          argument to validate
@@ -36,25 +60,25 @@ public abstract class Validator<T> implements Predicate<T> {
    *           if validation fails
    */
   public final T validate(final T argument) {
-    if (!test(argument))
-      throw new IllegalArgumentException(invalidMessage(argument));
+    // check if argument was recently validated, to short-circuit the check
+    // this especially helps if an API validates, then calls another API that validates the same
+    T lastValidatedSnapshot = lastValidated;
+    if (lastValidatedSnapshot != null && Objects.equals(argument, lastValidatedSnapshot))
+      return argument;
+
+    validateFunction.apply(argument).ifPresent(msg -> {
+      throw new IllegalArgumentException(msg);
+    });
+
+    // save most recently validated, to save time validating again
+    lastValidated = argument;
     return argument;
   }
 
   /**
-   * Formulates an exception message for invalid values.
-   *
-   * @param argument
-   *          argument that failed validation
-   * @return exception message
-   */
-  public String invalidMessage(final T argument) {
-    return String.format("Invalid argument %s", argument);
-  }
-
-  /**
    * Creates a new validator that is the conjunction of this one and the given one. An argument
-   * passed to the returned validator is valid only if it passes both validators.
+   * passed to the returned validator is valid if only if it passes both validators. If the other
+   * validator is null, the current validator is returned unchanged.
    *
    * @param other
    *          other validator
@@ -63,25 +87,14 @@ public abstract class Validator<T> implements Predicate<T> {
   public final Validator<T> and(final Validator<T> other) {
     if (other == null)
       return this;
-    final Validator<T> mine = this;
-    return new Validator<>() {
-
-      @Override
-      public boolean test(T argument) {
-        return mine.test(argument) && other.test(argument);
-      }
-
-      @Override
-      public String invalidMessage(T argument) {
-        return (mine.test(argument) ? other : mine).invalidMessage(argument);
-      }
-
-    };
+    return new Validator<>(
+        arg -> validateFunction.apply(arg).or(() -> other.validateFunction.apply(arg)));
   }
 
   /**
    * Creates a new validator that is the disjunction of this one and the given one. An argument
-   * passed to the returned validator is valid only if it passes at least one of the validators.
+   * passed to the returned validator is valid if and only if it passes at least one of the
+   * validators. If the other validator is null, the current validator is returned unchanged.
    *
    * @param other
    *          other validator
@@ -90,20 +103,8 @@ public abstract class Validator<T> implements Predicate<T> {
   public final Validator<T> or(final Validator<T> other) {
     if (other == null)
       return this;
-    final Validator<T> mine = this;
-    return new Validator<>() {
-
-      @Override
-      public boolean test(T argument) {
-        return mine.test(argument) || other.test(argument);
-      }
-
-      @Override
-      public String invalidMessage(T argument) {
-        return mine.invalidMessage(argument);
-      }
-
-    };
+    return new Validator<>(
+        arg -> validateFunction.apply(arg).isEmpty() ? OK : other.validateFunction.apply(arg));
   }
 
   /**
@@ -113,19 +114,10 @@ public abstract class Validator<T> implements Predicate<T> {
    * @return negated validator
    */
   public final Validator<T> not() {
-    final Validator<T> mine = this;
-    return new Validator<>() {
-
-      @Override
-      public boolean test(T argument) {
-        return !mine.test(argument);
-      }
-
-      @Override
-      public String invalidMessage(T argument) {
-        return "Validation should have failed with: " + mine.invalidMessage(argument);
-      }
-
-    };
+    return new Validator<>(arg -> {
+      return validateFunction.apply(arg).isPresent() ? OK
+          : Optional.of("Validation should have failed with: Invalid argument " + arg);
+    });
   }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Validators.java b/core/src/main/java/org/apache/accumulo/core/util/Validators.java
new file mode 100644
index 0000000..777e308
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/Validators.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+import org.apache.accumulo.core.clientImpl.Namespace;
+import org.apache.accumulo.core.clientImpl.Tables;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Joiner;
+
+public class Validators {
+  private static final Logger log = LoggerFactory.getLogger(Validators.class);
+
+  // do not instantiate
+  private Validators() {}
+
+  private static final int MAX_SEGMENT_LEN = 1024;
+  private static final Pattern SEGMENT_PATTERN = Pattern.compile("\\w{1," + MAX_SEGMENT_LEN + "}");
+  // before we added the length restriction; some existing tables may still be long
+  private static final Pattern EXISTING_SEGMENT_PATTERN = Pattern.compile("\\w+");
+
+  private enum NameSegment {
+    Table, Namespace;
+
+    Optional<String> isNull() {
+      return Optional.of(name() + " name must not be null");
+    }
+
+    Optional<String> isBlank() {
+      return Optional.of(name() + " name must not be blank");
+    }
+
+    Optional<String> tooLong(String s) {
+      return Optional
+          .of(name() + " name exceeds a maximum length of " + MAX_SEGMENT_LEN + ": " + s);
+    }
+
+    Optional<String> invalidChars(String s) {
+      return Optional.of(name() + " name '" + s + "' contains invalid (non-word) characters.");
+    }
+
+    void warnTooLong(String s) {
+      log.warn(name() + " name exceeds a length of {};"
+          + " Excessively long names are not supported and can result in unexpected behavior."
+          + " Please rename {}", MAX_SEGMENT_LEN, s);
+    }
+
+  }
+
+  // namespace name validators
+
+  // common implementation for EXISTING_NAMESPACE_NAME and NEW_NAMESPACE_NAME
+  private static Optional<String> _namespaceName(String ns, boolean existing) {
+    if (ns == null)
+      return NameSegment.Namespace.isNull();
+    // special case for default namespace, which always exists
+    if (ns.isEmpty())
+      return Validator.OK;
+    if (ns.length() > MAX_SEGMENT_LEN) {
+      if (existing)
+        NameSegment.Namespace.warnTooLong(ns);
+      else
+        return NameSegment.Namespace.tooLong(ns);
+    }
+    if ((existing ? EXISTING_SEGMENT_PATTERN : SEGMENT_PATTERN).matcher(ns).matches())
+      return Validator.OK;
+    return NameSegment.Namespace.invalidChars(ns);
+  }
+
+  public static final Validator<String> EXISTING_NAMESPACE_NAME =
+      new Validator<>(ns -> _namespaceName(ns, true));
+
+  public static final Validator<String> NEW_NAMESPACE_NAME =
+      new Validator<>(ns -> _namespaceName(ns, false));
+
+  public static final Validator<String> NOT_BUILTIN_NAMESPACE = new Validator<>(ns -> {
+    if (ns == null)
+      return NameSegment.Namespace.isNull();
+    if (Namespace.DEFAULT.name().equals(ns))
+      return Optional.of("Namespace must not be the reserved empty namespace");
+    if (Namespace.ACCUMULO.name().equals(ns))
+      return Optional.of("Namespace must not be the reserved namespace, " + ns);
+    return Validator.OK;
+  });
+
+  // table name validators
+
+  // common implementation for EXISTING_TABLE_NAME and NEW_TABLE_NAME
+  private static Optional<String> _tableName(String tableName, boolean existing) {
+    if (tableName == null)
+      return NameSegment.Table.isNull();
+    int dotPosition = tableName.indexOf('.');
+    if (dotPosition == 0)
+      return Optional.of("Table name must include a namespace prior to a dot(.) character");
+    String tablePart = tableName;
+    if (dotPosition > 0) {
+      String namespacePart = tableName.substring(0, dotPosition);
+      if (!EXISTING_SEGMENT_PATTERN.matcher(namespacePart).matches())
+        return NameSegment.Namespace.invalidChars(namespacePart);
+      tablePart = tableName.substring(dotPosition + 1);
+    }
+    if (tablePart.isBlank()) {
+      return NameSegment.Table.isBlank();
+    }
+    if (tablePart.length() > MAX_SEGMENT_LEN) {
+      if (existing)
+        NameSegment.Table.warnTooLong(tablePart);
+      else
+        return NameSegment.Table.tooLong(tablePart);
+    }
+    if (!(existing ? EXISTING_SEGMENT_PATTERN : SEGMENT_PATTERN).matcher(tablePart).matches())
+      return NameSegment.Table.invalidChars(tablePart);
+    return Validator.OK;
+  }
+
+  public static final Validator<String> EXISTING_TABLE_NAME =
+      new Validator<>(tableName -> _tableName(tableName, true));
+
+  public static final Validator<String> NEW_TABLE_NAME =
+      new Validator<>(tableName -> _tableName(tableName, false));
+
+  private static final List<String> metadataTables = List.of(RootTable.NAME, MetadataTable.NAME);
+  public static final Validator<String> NOT_METADATA_TABLE = new Validator<>(t -> {
+    if (t == null)
+      return NameSegment.Table.isNull();
+    if (metadataTables.contains(t))
+      return Optional.of("Table must not be any of {" + Joiner.on(",").join(metadataTables) + "}");
+    return Validator.OK;
+  });
+
+  public static final Validator<String> NOT_BUILTIN_TABLE = new Validator<>(t -> {
+    if (Namespace.ACCUMULO.name().equals(Tables.qualify(t).getFirst()))
+      return Optional.of("Table must not be in the '" + Namespace.ACCUMULO.name() + "' namespace");
+    return Validator.OK;
+  });
+
+  public static Validator<String> sameNamespaceAs(String oldTableName) {
+    final String oldNamespace = Tables.qualify(oldTableName).getFirst();
+    return new Validator<>(newName -> {
+      if (!oldNamespace.equals(Tables.qualify(newName).getFirst()))
+        return Optional
+            .of("Unable to move tables to a new namespace by renaming. The namespace for " + newName
+                + " does not match " + oldTableName);
+      return Validator.OK;
+    });
+  }
+
+  // table id validators
+
+  private static final Pattern VALID_ID_PATTERN = Pattern.compile("[a-z0-9]+"); // BigDecimal base36
+  public static final Validator<TableId> VALID_TABLE_ID = new Validator<>(id -> {
+    if (id == null)
+      return Optional.of("Table id must not be null");
+    if (RootTable.ID.equals(id) || MetadataTable.ID.equals(id) || ReplicationTable.ID.equals(id)
+        || VALID_ID_PATTERN.matcher(id.canonical()).matches())
+      return Validator.OK;
+    return Optional
+        .of("Table IDs are base-36 numbers, represented with lowercase alphanumeric digits: " + id);
+  });
+
+  public static final Validator<TableId> CAN_CLONE_TABLE = new Validator<>(id -> {
+    if (id == null)
+      return Optional.of("Table id must not be null");
+    if (id.equals(MetadataTable.ID))
+      return Optional.of("Cloning " + MetadataTable.NAME + " is dangerous and no longer supported,"
+          + " see https://github.com/apache/accumulo/issues/1309.");
+    if (id.equals(RootTable.ID))
+      return Optional.of("Unable to clone " + RootTable.NAME);
+    return Validator.OK;
+  });
+
+  public static final Validator<TableId> NOT_ROOT_TABLE_ID = new Validator<>(id -> {
+    if (id == null)
+      return Optional.of("Table id must not be null");
+    if (RootTable.ID.equals(id))
+      return Optional
+          .of("Table must not be the " + RootTable.NAME + "(Id: " + RootTable.ID + ") table");
+    return Validator.OK;
+  });
+
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/util/ValidatorTest.java b/core/src/test/java/org/apache/accumulo/core/util/ValidatorTest.java
index c7ab9a4..c77aa81 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/ValidatorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/ValidatorTest.java
@@ -19,46 +19,27 @@
 package org.apache.accumulo.core.util;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertThrows;
 
+import java.util.Optional;
+
+import org.apache.hadoop.shaded.com.google.re2j.Pattern;
 import org.junit.Before;
 import org.junit.Test;
 
 public class ValidatorTest {
-  private static class TestValidator extends Validator<String> {
-    private final String s;
-
-    TestValidator(String s) {
-      this.s = s;
-    }
-
-    @Override
-    public boolean test(String argument) {
-      return s.equals(argument);
-    }
-  }
-
-  private static class Test2Validator extends Validator<String> {
-    private final String ps;
-
-    Test2Validator(String s) {
-      ps = s;
-    }
-
-    @Override
-    public boolean test(String argument) {
-      return (argument != null && argument.matches(ps));
-    }
-  }
 
   private Validator<String> v, v2, v3;
+  private static final Pattern STARTSWITH_C = Pattern.compile("c.*");
 
   @Before
   public void setUp() {
-    v = new TestValidator("correct");
-    v2 = new TestValidator("righto");
-    v3 = new Test2Validator("c.*");
+    v = new Validator<>(
+        arg -> "correct".equals(arg) ? Validator.OK : Optional.of("Invalid argument " + arg));
+    v2 = new Validator<>(arg -> "righto".equals(arg) ? Validator.OK
+        : Optional.of("Not a correct argument : " + arg + " : done"));
+    v3 = new Validator<>(s -> s != null && STARTSWITH_C.matcher(s).matches() ? Validator.OK
+        : Optional.of("Invalid argument " + s));
   }
 
   @Test
@@ -66,37 +47,41 @@ public class ValidatorTest {
     assertEquals("correct", v.validate("correct"));
   }
 
-  @Test(expected = IllegalArgumentException.class)
-  public void testValidate_Failure() {
-    v.validate("incorrect");
-  }
-
   @Test
-  public void testInvalidMessage() {
-    assertEquals("Invalid argument incorrect", v.invalidMessage("incorrect"));
+  public void testValidate_Failure() {
+    // check default message maker
+    var e = assertThrows(IllegalArgumentException.class, () -> v.validate("incorrect"));
+    assertEquals("Invalid argument incorrect", e.getMessage());
+    // check custom message maker
+    e = assertThrows(IllegalArgumentException.class, () -> v2.validate("somethingwrong"));
+    assertEquals("Not a correct argument : somethingwrong : done", e.getMessage());
   }
 
   @Test
   public void testAnd() {
     Validator<String> vand = v3.and(v);
-    assertTrue(vand.test("correct"));
-    assertFalse(vand.test("righto"));
-    assertFalse(vand.test("coriander"));
+    assertEquals("correct", vand.validate("correct"));
+    assertThrows(IllegalArgumentException.class, () -> vand.validate("righto"));
+    assertThrows(IllegalArgumentException.class, () -> vand.validate("coriander"));
   }
 
   @Test
   public void testOr() {
     Validator<String> vor = v.or(v2);
-    assertTrue(vor.test("correct"));
-    assertTrue(vor.test("righto"));
-    assertFalse(vor.test("coriander"));
+    assertEquals("correct", vor.validate("correct"));
+    assertEquals("righto", vor.validate("righto"));
+    assertThrows(IllegalArgumentException.class, () -> vor.validate("coriander"));
   }
 
   @Test
   public void testNot() {
     Validator<String> vnot = v3.not();
-    assertFalse(vnot.test("correct"));
-    assertFalse(vnot.test("coriander"));
-    assertTrue(vnot.test("righto"));
+    var e = assertThrows(IllegalArgumentException.class, () -> vnot.validate("correct"));
+    assertEquals("Validation should have failed with: Invalid argument correct", e.getMessage());
+    e = assertThrows(IllegalArgumentException.class, () -> vnot.validate("coriander"));
+    assertEquals("Validation should have failed with: Invalid argument coriander", e.getMessage());
+    assertEquals("righto", vnot.validate("righto"));
+    assertEquals("anythingNotStartingWithLowercaseC",
+        vnot.validate("anythingNotStartingWithLowercaseC"));
   }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/util/ValidatorsTest.java b/core/src/test/java/org/apache/accumulo/core/util/ValidatorsTest.java
new file mode 100644
index 0000000..fdbf328
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/ValidatorsTest.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+import java.util.function.Consumer;
+
+import org.apache.accumulo.core.clientImpl.Namespace;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+
+public class ValidatorsTest {
+
+  private static <T> void checkNull(Consumer<T> nullConsumer) {
+    var e = assertThrows(IllegalArgumentException.class, () -> nullConsumer.accept(null));
+    assertTrue(e.getMessage().endsWith("must not be null"));
+  }
+
+  private static <T> void assertAllValidate(Validator<T> v, List<T> items) {
+    assertFalse("nothing to check", items.isEmpty());
+    items.forEach(item -> assertSame(item, v.validate(item)));
+  }
+
+  private static <T> void assertAllThrow(Validator<T> v, List<T> items) {
+    assertFalse("nothing to check", items.isEmpty());
+    items.forEach(item -> assertThrows(String.valueOf(item), IllegalArgumentException.class,
+        () -> v.validate(item)));
+  }
+
+  @Test
+  public void test_CAN_CLONE_TABLE() {
+    Validator<TableId> v = Validators.CAN_CLONE_TABLE;
+    checkNull(v::validate);
+    assertAllValidate(v, List.of(ReplicationTable.ID, TableId.of("id1")));
+    assertAllThrow(v, List.of(RootTable.ID, MetadataTable.ID));
+  }
+
+  @Test
+  public void test_EXISTING_NAMESPACE_NAME() {
+    Validator<String> v = Validators.EXISTING_NAMESPACE_NAME;
+    checkNull(v::validate);
+    assertAllValidate(v, List.of(Namespace.DEFAULT.name(), Namespace.ACCUMULO.name(), "normalNs",
+        "withNumber2", "has_underscore", "_underscoreStart", StringUtils.repeat("a", 1025)));
+    assertAllThrow(v, List.of("has.dot", "has-dash", " hasSpace", ".", "has$dollar"));
+  }
+
+  @Test
+  public void test_EXISTING_TABLE_NAME() {
+    Validator<String> v = Validators.EXISTING_TABLE_NAME;
+    checkNull(v::validate);
+    assertAllValidate(v,
+        List.of(RootTable.NAME, MetadataTable.NAME, "normalTable", "withNumber2", "has_underscore",
+            "_underscoreStart", StringUtils.repeat("a", 1025),
+            StringUtils.repeat("a", 1025) + "." + StringUtils.repeat("a", 1025)));
+    assertAllThrow(v, List.of("has-dash", "has-dash.inNamespace", "has.dash-inTable", " hasSpace",
+        ".", "has$dollar", "two.dots.here", ".startsDot"));
+  }
+
+  @Test
+  public void test_NEW_NAMESPACE_NAME() {
+    Validator<String> v = Validators.NEW_NAMESPACE_NAME;
+    checkNull(v::validate);
+    assertAllValidate(v, List.of(Namespace.DEFAULT.name(), Namespace.ACCUMULO.name(), "normalNs",
+        "withNumber2", "has_underscore", "_underscoreStart", StringUtils.repeat("a", 1024)));
+    assertAllThrow(v, List.of("has.dot", "has-dash", " hasSpace", ".", "has$dollar",
+        StringUtils.repeat("a", 1025)));
+  }
+
+  @Test
+  public void test_NEW_TABLE_NAME() {
+    Validator<String> v = Validators.NEW_TABLE_NAME;
+    checkNull(v::validate);
+    assertAllValidate(v,
+        List.of(RootTable.NAME, MetadataTable.NAME, "normalTable", "withNumber2", "has_underscore",
+            "_underscoreStart", StringUtils.repeat("a", 1024),
+            StringUtils.repeat("a", 1025) + "." + StringUtils.repeat("a", 1024)));
+    assertAllThrow(v,
+        List.of("has-dash", "has-dash.inNamespace", "has.dash-inTable", " hasSpace", ".",
+            "has$dollar", "two.dots.here", ".startsDot", StringUtils.repeat("a", 1025),
+            StringUtils.repeat("a", 1025) + "." + StringUtils.repeat("a", 1025)));
+  }
+
+  @Test
+  public void test_NOT_BUILTIN_NAMESPACE() {
+    Validator<String> v = Validators.NOT_BUILTIN_NAMESPACE;
+    checkNull(v::validate);
+    assertAllValidate(v, List.of("root", "metadata", " .#!)(*$&^", "  ")); // doesn't validate name
+    assertAllThrow(v, List.of(Namespace.DEFAULT.name(), Namespace.ACCUMULO.name()));
+  }
+
+  @Test
+  public void test_NOT_BUILTIN_TABLE() {
+    Validator<String> v = Validators.NOT_BUILTIN_TABLE;
+    checkNull(v::validate);
+    assertAllValidate(v, List.of("root", "metadata", "user", "ns1.table2"));
+    assertAllThrow(v, List.of(RootTable.NAME, MetadataTable.NAME, ReplicationTable.NAME));
+  }
+
+  @Test
+  public void test_NOT_METADATA_TABLE() {
+    Validator<String> v = Validators.NOT_METADATA_TABLE;
+    checkNull(v::validate);
+    assertAllValidate(v, List.of("root", "metadata", "user", "ns1.table2", ReplicationTable.NAME));
+    assertAllThrow(v, List.of(RootTable.NAME, MetadataTable.NAME));
+  }
+
+  @Test
+  public void test_NOT_ROOT_TABLE_ID() {
+    Validator<TableId> v = Validators.NOT_ROOT_TABLE_ID;
+    checkNull(v::validate);
+    assertAllValidate(v,
+        List.of(TableId.of(""), MetadataTable.ID, ReplicationTable.ID, TableId.of(" #0(U!$. ")));
+    assertAllThrow(v, List.of(RootTable.ID));
+  }
+
+  @Test
+  public void test_VALID_TABLE_ID() {
+    Validator<TableId> v = Validators.VALID_TABLE_ID;
+    checkNull(v::validate);
+    assertAllValidate(v, List.of(RootTable.ID, MetadataTable.ID, ReplicationTable.ID,
+        TableId.of("111"), TableId.of("aaaa"), TableId.of("r2d2")));
+    assertAllThrow(v, List.of(TableId.of(""), TableId.of("#0(U!$"), TableId.of(" #0(U!$. "),
+        TableId.of("."), TableId.of(" "), TableId.of("C3P0")));
+  }
+
+  @Test
+  public void test_sameNamespaceAs() {
+    checkNull(Validators::sameNamespaceAs);
+    Validator<String> inDefaultNS = Validators.sameNamespaceAs("tableInDefaultNamespace");
+    checkNull(inDefaultNS::validate);
+    assertAllValidate(inDefaultNS, List.of("t1"));
+    assertAllThrow(inDefaultNS, List.of("accumulo.other", "other.t2", ".", "other.", ".malformed"));
+
+    Validator<String> inOtherNS = Validators.sameNamespaceAs("other.tableInOtherNamespace");
+    checkNull(inOtherNS::validate);
+    assertAllValidate(inOtherNS, List.of("other.t1", "other.t2"));
+    assertAllThrow(inOtherNS, List.of("other.", "other", "else.t3"));
+  }
+
+}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
index 8cf1f12..02c1b85 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
@@ -19,12 +19,17 @@
 package org.apache.accumulo.manager;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.manager.util.TableValidators.CAN_CLONE;
-import static org.apache.accumulo.manager.util.TableValidators.NOT_METADATA;
-import static org.apache.accumulo.manager.util.TableValidators.NOT_ROOT_ID;
-import static org.apache.accumulo.manager.util.TableValidators.NOT_SYSTEM;
-import static org.apache.accumulo.manager.util.TableValidators.VALID_ID;
-import static org.apache.accumulo.manager.util.TableValidators.VALID_NAME;
+import static org.apache.accumulo.core.util.Validators.CAN_CLONE_TABLE;
+import static org.apache.accumulo.core.util.Validators.EXISTING_NAMESPACE_NAME;
+import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
+import static org.apache.accumulo.core.util.Validators.NEW_NAMESPACE_NAME;
+import static org.apache.accumulo.core.util.Validators.NEW_TABLE_NAME;
+import static org.apache.accumulo.core.util.Validators.NOT_BUILTIN_NAMESPACE;
+import static org.apache.accumulo.core.util.Validators.NOT_BUILTIN_TABLE;
+import static org.apache.accumulo.core.util.Validators.NOT_METADATA_TABLE;
+import static org.apache.accumulo.core.util.Validators.NOT_ROOT_TABLE_ID;
+import static org.apache.accumulo.core.util.Validators.VALID_TABLE_ID;
+import static org.apache.accumulo.core.util.Validators.sameNamespaceAs;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -91,8 +96,6 @@ class FateServiceHandler implements FateService.Iface {
 
   protected final Manager manager;
   protected static final Logger log = Manager.log;
-  protected static final int MAX_TABLE_NAME_LEN = 1024;
-  protected static final int MAX_NAMESPACE_LEN = 1024;
 
   public FateServiceHandler(Manager manager) {
     this.manager = manager;
@@ -115,7 +118,7 @@ class FateServiceHandler implements FateService.Iface {
       case NAMESPACE_CREATE: {
         TableOperation tableOp = TableOperation.CREATE;
         validateArgumentCount(arguments, tableOp, 1);
-        String namespace = validateNewNamespaceArgument(arguments.get(0), tableOp, null);
+        String namespace = validateName(arguments.get(0), tableOp, NEW_NAMESPACE_NAME);
 
         if (!manager.security.canCreateNamespace(c))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -128,9 +131,9 @@ class FateServiceHandler implements FateService.Iface {
       case NAMESPACE_RENAME: {
         TableOperation tableOp = TableOperation.RENAME;
         validateArgumentCount(arguments, tableOp, 2);
-        String oldName = validateNamespaceArgument(arguments.get(0), tableOp,
-            Namespaces.NOT_DEFAULT.and(Namespaces.NOT_ACCUMULO));
-        String newName = validateNewNamespaceArgument(arguments.get(1), tableOp, null);
+        String oldName = validateName(arguments.get(0), tableOp,
+            EXISTING_NAMESPACE_NAME.and(NOT_BUILTIN_NAMESPACE));
+        String newName = validateName(arguments.get(1), tableOp, NEW_NAMESPACE_NAME);
 
         NamespaceId namespaceId =
             ClientServiceHandler.checkNamespaceId(manager.getContext(), oldName, tableOp);
@@ -144,8 +147,8 @@ class FateServiceHandler implements FateService.Iface {
       case NAMESPACE_DELETE: {
         TableOperation tableOp = TableOperation.DELETE;
         validateArgumentCount(arguments, tableOp, 1);
-        String namespace = validateNamespaceArgument(arguments.get(0), tableOp,
-            Namespaces.NOT_DEFAULT.and(Namespaces.NOT_ACCUMULO));
+        String namespace = validateName(arguments.get(0), tableOp,
+            EXISTING_NAMESPACE_NAME.and(NOT_BUILTIN_NAMESPACE));
 
         NamespaceId namespaceId =
             ClientServiceHandler.checkNamespaceId(manager.getContext(), namespace, tableOp);
@@ -164,7 +167,8 @@ class FateServiceHandler implements FateService.Iface {
               TableOperationExceptionType.OTHER,
               "Expected at least " + SPLIT_OFFSET + " arguments, saw :" + arguments.size());
         }
-        String tableName = validateNewTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
+        String tableName =
+            validateName(arguments.get(0), tableOp, NEW_TABLE_NAME.and(NOT_BUILTIN_TABLE));
         TimeType timeType = TimeType.valueOf(ByteBufferUtil.toString(arguments.get(1)));
         InitialTableState initialTableState =
             InitialTableState.valueOf(ByteBufferUtil.toString(arguments.get(2)));
@@ -207,26 +211,10 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_RENAME: {
         TableOperation tableOp = TableOperation.RENAME;
         validateArgumentCount(arguments, tableOp, 2);
-        final String oldTableName =
-            validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
-        String newTableName =
-
-            validateNewTableNameArgument(arguments.get(1), tableOp, new Validator<>() {
-
-              @Override
-              public boolean test(String argument) {
-                // verify they are in the same namespace
-                String oldNamespace = Tables.qualify(oldTableName).getFirst();
-                return oldNamespace.equals(Tables.qualify(argument).getFirst());
-              }
-
-              @Override
-              public String invalidMessage(String argument) {
-                return "Cannot move tables to a new namespace by renaming. The namespace for "
-                    + oldTableName + " does not match " + argument;
-              }
-
-            });
+        String oldTableName =
+            validateName(arguments.get(0), tableOp, EXISTING_TABLE_NAME.and(NOT_BUILTIN_TABLE));
+        String newTableName = validateName(arguments.get(1), tableOp,
+            NEW_TABLE_NAME.and(sameNamespaceAs(oldTableName)));
 
         TableId tableId =
             ClientServiceHandler.checkTableId(manager.getContext(), oldTableName, tableOp);
@@ -258,8 +246,9 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_CLONE: {
         TableOperation tableOp = TableOperation.CLONE;
         validateArgumentCount(arguments, tableOp, 3);
-        TableId srcTableId = validateTableIdArgument(arguments.get(0), tableOp, CAN_CLONE);
-        String tableName = validateNewTableNameArgument(arguments.get(1), tableOp, NOT_SYSTEM);
+        TableId srcTableId = validateTableIdArgument(arguments.get(0), tableOp, CAN_CLONE_TABLE);
+        String tableName =
+            validateName(arguments.get(1), tableOp, NEW_TABLE_NAME.and(NOT_BUILTIN_TABLE));
         boolean keepOffline = false;
         if (arguments.get(2) != null) {
           keepOffline = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(2)));
@@ -315,7 +304,8 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_DELETE: {
         TableOperation tableOp = TableOperation.DELETE;
         validateArgumentCount(arguments, tableOp, 1);
-        String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
+        String tableName =
+            validateName(arguments.get(0), tableOp, EXISTING_TABLE_NAME.and(NOT_BUILTIN_TABLE));
 
         final TableId tableId =
             ClientServiceHandler.checkTableId(manager.getContext(), tableName, tableOp);
@@ -338,7 +328,7 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_ONLINE: {
         TableOperation tableOp = TableOperation.ONLINE;
         validateArgumentCount(arguments, tableOp, 1);
-        final TableId tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
+        final var tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_TABLE_ID);
         NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canOnlineOfflineTable;
@@ -360,7 +350,7 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_OFFLINE: {
         TableOperation tableOp = TableOperation.OFFLINE;
         validateArgumentCount(arguments, tableOp, 1);
-        final TableId tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
+        final var tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_TABLE_ID);
         NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canOnlineOfflineTable;
@@ -382,7 +372,7 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_MERGE: {
         TableOperation tableOp = TableOperation.MERGE;
         validateArgumentCount(arguments, tableOp, 3);
-        String tableName = validateTableNameArgument(arguments.get(0), tableOp, null);
+        String tableName = validateName(arguments.get(0), tableOp, EXISTING_TABLE_NAME);
         Text startRow = ByteBufferUtil.toText(arguments.get(1));
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
@@ -410,7 +400,8 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_DELETE_RANGE: {
         TableOperation tableOp = TableOperation.DELETE_RANGE;
         validateArgumentCount(arguments, tableOp, 3);
-        String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_METADATA);
+        String tableName =
+            validateName(arguments.get(0), tableOp, EXISTING_TABLE_NAME.and(NOT_METADATA_TABLE));
         Text startRow = ByteBufferUtil.toText(arguments.get(1));
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
@@ -438,7 +429,8 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_BULK_IMPORT: {
         TableOperation tableOp = TableOperation.BULK_IMPORT;
         validateArgumentCount(arguments, tableOp, 4);
-        String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
+        String tableName =
+            validateName(arguments.get(0), tableOp, EXISTING_TABLE_NAME.and(NOT_BUILTIN_TABLE));
         String dir = ByteBufferUtil.toString(arguments.get(1));
         String failDir = ByteBufferUtil.toString(arguments.get(2));
         boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(3)));
@@ -518,7 +510,8 @@ class FateServiceHandler implements FateService.Iface {
               TableOperationExceptionType.OTHER,
               "Expected at least " + IMPORT_DIR_OFFSET + "arguments, sar :" + arguments.size());
         }
-        String tableName = validateNewTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
+        String tableName =
+            validateName(arguments.get(0), tableOp, NEW_TABLE_NAME.and(NOT_BUILTIN_TABLE));
         List<ByteBuffer> exportDirArgs = arguments.stream().skip(1).collect(Collectors.toList());
         Set<String> exportDirs = ByteBufferUtil.toStringSet(exportDirArgs);
         NamespaceId namespaceId;
@@ -549,7 +542,8 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_EXPORT: {
         TableOperation tableOp = TableOperation.EXPORT;
         validateArgumentCount(arguments, tableOp, 2);
-        String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
+        String tableName =
+            validateName(arguments.get(0), tableOp, EXISTING_TABLE_NAME.and(NOT_BUILTIN_TABLE));
         String exportDir = ByteBufferUtil.toString(arguments.get(1));
 
         TableId tableId =
@@ -575,7 +569,7 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_BULK_IMPORT2:
         TableOperation tableOp = TableOperation.BULK_IMPORT;
         validateArgumentCount(arguments, tableOp, 3);
-        TableId tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
+        final var tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_TABLE_ID);
         String dir = ByteBufferUtil.toString(arguments.get(1));
 
         boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(2)));
@@ -694,7 +688,7 @@ class FateServiceHandler implements FateService.Iface {
       Validator<TableId> userValidator) throws ThriftTableOperationException {
     TableId tableId = tableIdArg == null ? null : ByteBufferUtil.toTableId(tableIdArg);
     try {
-      return VALID_ID.and(userValidator).validate(tableId);
+      return VALID_TABLE_ID.and(userValidator).validate(tableId);
     } catch (IllegalArgumentException e) {
       String why = e.getMessage();
       // Information provided by a client should generate a user-level exception, not a system-level
@@ -705,32 +699,6 @@ class FateServiceHandler implements FateService.Iface {
     }
   }
 
-  // Verify existing table's name argument is valid, and match any additional restrictions
-  private String validateTableNameArgument(ByteBuffer tableNameArg, TableOperation op,
-      Validator<String> userValidator) throws ThriftTableOperationException {
-    String tableName = tableNameArg == null ? null : ByteBufferUtil.toString(tableNameArg);
-    if ((tableName != null) && (tableName.length() > MAX_TABLE_NAME_LEN)) {
-      log.warn("Table names greater than " + MAX_TABLE_NAME_LEN
-          + " characters should be renamed to conform to a " + MAX_TABLE_NAME_LEN
-          + " character limit. Longer table names are no longer supported and may result in "
-          + " unexpected behavior.");
-    }
-    return _validateArgument(tableName, op, VALID_NAME.and(userValidator));
-  }
-
-  // Verify table name arguments are valid, and match any additional restrictions
-  private String validateNewTableNameArgument(ByteBuffer tableNameArg, TableOperation op,
-      Validator<String> userValidator) throws ThriftTableOperationException {
-    String tableName = tableNameArg == null ? null : ByteBufferUtil.toString(tableNameArg);
-    if ((tableName != null) && (tableName.length() > MAX_TABLE_NAME_LEN)) {
-      throw new ThriftTableOperationException(null, tableName, op,
-          TableOperationExceptionType.INVALID_NAME,
-          "Table names must be less than or equal to " + MAX_TABLE_NAME_LEN + " characters. " + "'"
-              + tableName + "' is " + tableName.length() + " characters long.");
-    }
-    return _validateArgument(tableName, op, VALID_NAME.and(userValidator));
-  }
-
   private void validateArgumentCount(List<ByteBuffer> arguments, TableOperation op, int expected)
       throws ThriftTableOperationException {
     if (arguments.size() != expected) {
@@ -739,41 +707,16 @@ class FateServiceHandler implements FateService.Iface {
     }
   }
 
-  // Verify namespace arguments are valid, and match any additional restrictions
-  private String validateNamespaceArgument(ByteBuffer namespaceArg, TableOperation op,
-      Validator<String> userValidator) throws ThriftTableOperationException {
-    String namespace = namespaceArg == null ? null : ByteBufferUtil.toString(namespaceArg);
-    if ((namespace != null) && (namespace.length() > MAX_NAMESPACE_LEN)) {
-      log.warn("Namespaces greater than " + MAX_NAMESPACE_LEN
-          + " characters should be renamed to conform to a " + MAX_NAMESPACE_LEN
-          + " character limit. "
-          + "Longer namespaces are no longer supported and may result in unexpected behavior.");
-    }
-    return _validateArgument(namespace, op, Namespaces.VALID_NAME.and(userValidator));
-  }
-
-  // Verify namespace arguments are valid, and match any additional restrictions
-  private String validateNewNamespaceArgument(ByteBuffer namespaceArg, TableOperation op,
-      Validator<String> userValidator) throws ThriftTableOperationException {
-    String namespace = namespaceArg == null ? null : ByteBufferUtil.toString(namespaceArg);
-    if ((namespace != null) && (namespace.length() > MAX_NAMESPACE_LEN)) {
-      throw new ThriftTableOperationException(null, namespace, op,
-          TableOperationExceptionType.INVALID_NAME,
-          "Namespaces must be less than or equal to " + MAX_NAMESPACE_LEN + " characters. " + "'"
-              + namespace + "' is " + namespace.length() + " characters long.");
-    }
-    return _validateArgument(namespace, op, Namespaces.VALID_NAME.and(userValidator));
-  }
-
-  // helper to handle the exception
-  private <T> T _validateArgument(T arg, TableOperation op, Validator<T> validator)
+  // Verify namespace or table name arguments are valid, and match any additional restrictions
+  private String validateName(ByteBuffer argument, TableOperation op, Validator<String> validator)
       throws ThriftTableOperationException {
+    String arg = argument == null ? null : ByteBufferUtil.toString(argument);
     try {
       return validator.validate(arg);
     } catch (IllegalArgumentException e) {
       String why = e.getMessage();
-      // Information provided by a client should generate a user-level exception, not a system-level
-      // warning.
+      // Information provided by a client should generate a user-level exception,
+      // not a system-level warning, so use debug here.
       log.debug(why);
       throw new ThriftTableOperationException(null, String.valueOf(arg), op,
           TableOperationExceptionType.INVALID_NAME, why);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/util/TableValidators.java b/server/manager/src/main/java/org/apache/accumulo/manager/util/TableValidators.java
deleted file mode 100644
index 4eefa80..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/util/TableValidators.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.manager.util;
-
-import static org.apache.accumulo.core.clientImpl.Tables.VALID_TABLENAME_REGEX;
-import static org.apache.accumulo.core.clientImpl.Tables.qualify;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.util.Validator;
-
-import com.google.common.base.Joiner;
-
-public class TableValidators {
-  public static final String VALID_ID_REGEX = "^([a-z0-9]+)$"; // BigDecimal base36
-
-  public static final Validator<String> VALID_NAME = new Validator<>() {
-    @Override
-    public boolean test(String tableName) {
-      return tableName != null && tableName.matches(VALID_TABLENAME_REGEX);
-    }
-
-    @Override
-    public String invalidMessage(String tableName) {
-      if (tableName == null)
-        return "Table name cannot be null";
-      return "Table names must only contain word characters (letters, digits, and underscores): "
-          + tableName;
-    }
-  };
-
-  public static final Validator<TableId> VALID_ID = new Validator<>() {
-    @Override
-    public boolean test(TableId tableId) {
-      return tableId != null && (RootTable.ID.equals(tableId) || MetadataTable.ID.equals(tableId)
-          || ReplicationTable.ID.equals(tableId) || tableId.canonical().matches(VALID_ID_REGEX));
-    }
-
-    @Override
-    public String invalidMessage(TableId tableId) {
-      if (tableId == null)
-        return "Table id cannot be null";
-      return "Table IDs are base-36 numbers, represented with lowercase alphanumeric digits: "
-          + tableId;
-    }
-  };
-
-  public static final Validator<String> NOT_METADATA = new Validator<>() {
-
-    private List<String> metadataTables = Arrays.asList(RootTable.NAME, MetadataTable.NAME);
-
-    @Override
-    public boolean test(String tableName) {
-      return !metadataTables.contains(tableName);
-    }
-
-    @Override
-    public String invalidMessage(String tableName) {
-      return "Table cannot be one of {" + Joiner.on(",").join(metadataTables) + "}";
-    }
-  };
-
-  public static final Validator<TableId> CAN_CLONE = new Validator<>() {
-
-    private List<TableId> metaIDs = Arrays.asList(RootTable.ID, MetadataTable.ID);
-
-    @Override
-    public boolean test(TableId tableId) {
-      return !metaIDs.contains(tableId);
-    }
-
-    @Override
-    public String invalidMessage(TableId tableId) {
-      String msg;
-      if (tableId.equals(MetadataTable.ID)) {
-        msg = " Cloning " + MetadataTable.NAME
-            + " is dangerous and no longer supported, see https://github.com/apache/accumulo/issues/1309.";
-      } else {
-        msg = "Can not clone " + RootTable.NAME;
-      }
-      return msg;
-    }
-  };
-
-  public static final Validator<String> NOT_SYSTEM = new Validator<>() {
-
-    @Override
-    public boolean test(String tableName) {
-      return !Namespace.ACCUMULO.name().equals(qualify(tableName).getFirst());
-    }
-
-    @Override
-    public String invalidMessage(String tableName) {
-      return "Table cannot be in the " + Namespace.ACCUMULO.name() + " namespace";
-    }
-  };
-
-  public static final Validator<TableId> NOT_ROOT_ID = new Validator<>() {
-
-    @Override
-    public boolean test(TableId tableId) {
-      return !RootTable.ID.equals(tableId);
-    }
-
-    @Override
-    public String invalidMessage(TableId tableId) {
-      return "Table cannot be the " + RootTable.NAME + "(Id: " + RootTable.ID + ") table";
-    }
-  };
-
-}
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
index 93a933c..caa41a9 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CreateTableCommand.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.shell.commands;
 
+import static org.apache.accumulo.core.util.Validators.NEW_TABLE_NAME;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.EnumSet;
@@ -36,7 +38,6 @@ import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.conf.IterConfigUtil;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.constraints.VisibilityConstraint;
@@ -71,16 +72,11 @@ public class CreateTableCommand extends Command {
       throws AccumuloException, AccumuloSecurityException, TableExistsException,
       TableNotFoundException, IOException {
 
-    final String testTableName = cl.getArgs()[0];
+    final String tableName = cl.getArgs()[0];
     NewTableConfiguration ntc = new NewTableConfiguration();
 
-    if (!testTableName.matches(Tables.VALID_TABLENAME_REGEX)) {
-      shellState.getWriter()
-          .println("Only letters, numbers and underscores are allowed for use in table names.");
-      throw new IllegalArgumentException();
-    }
+    NEW_TABLE_NAME.validate(tableName);
 
-    final String tableName = cl.getArgs()[0];
     if (shellState.getAccumuloClient().tableOperations().exists(tableName)) {
       throw new TableExistsException(null, tableName, null);
     }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java
index fbadef1..c4ac589 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/DUCommand.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -60,11 +61,9 @@ public class DUCommand extends Command {
 
     // Add any patterns
     if (cl.hasOption(optTablePattern.getOpt())) {
-      for (String table : shellState.getAccumuloClient().tableOperations().list()) {
-        if (table.matches(cl.getOptionValue(optTablePattern.getOpt()))) {
-          tables.add(table);
-        }
-      }
+      shellState.getAccumuloClient().tableOperations().list().stream()
+          .filter(Pattern.compile(cl.getOptionValue(optTablePattern.getOpt())).asMatchPredicate())
+          .forEach(tables::add);
     }
 
     // If we didn't get any tables, and we have a table selected, add the current table
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ListCompactionsCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ListCompactionsCommand.java
index f09edc2..78162b8 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ListCompactionsCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ListCompactionsCommand.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.shell.commands;
 
+import java.util.regex.Pattern;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.admin.InstanceOperations;
@@ -62,8 +63,8 @@ public class ListCompactionsCommand extends Command {
     }
 
     if (filterText != null) {
-      final String finalFilterText = filterText;
-      activeCompactionStream = activeCompactionStream.filter(t -> t.matches(finalFilterText));
+      activeCompactionStream =
+          activeCompactionStream.filter(Pattern.compile(filterText).asMatchPredicate());
     }
 
     activeCompactionStream = ActiveCompactionHelper.appendHeader(activeCompactionStream);
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ListTabletsCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ListTabletsCommand.java
index 64b8ba8..9ef4cbb 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ListTabletsCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ListTabletsCommand.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Objects;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperations;
@@ -129,9 +130,9 @@ public class ListTabletsCommand extends Command {
     Set<TableInfo> tableSet = new TreeSet<>();
 
     if (cl.hasOption(optTablePattern.getOpt())) {
-      String tablePattern = cl.getOptionValue(optTablePattern.getOpt());
+      Pattern tablePattern = Pattern.compile(cl.getOptionValue(optTablePattern.getOpt()));
       for (String table : tableOps.list()) {
-        if (table.matches(tablePattern)) {
+        if (tablePattern.matcher(table).matches()) {
           TableId id = TableId.of(tableIdMap.get(table));
           tableSet.add(new TableInfo(table, id));
         }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java b/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java
index c2b3487..94bba6e 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/TableOperation.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.Namespaces;
@@ -50,11 +51,9 @@ public abstract class TableOperation extends Command {
     // populate the tableSet set with the tables you want to operate on
     final SortedSet<String> tableSet = new TreeSet<>();
     if (cl.hasOption(optTablePattern.getOpt())) {
-      String tablePattern = cl.getOptionValue(optTablePattern.getOpt());
-      for (String table : shellState.getAccumuloClient().tableOperations().list())
-        if (table.matches(tablePattern)) {
-          tableSet.add(table);
-        }
+      shellState.getAccumuloClient().tableOperations().list().stream()
+          .filter(Pattern.compile(cl.getOptionValue(optTablePattern.getOpt())).asMatchPredicate())
+          .forEach(tableSet::add);
       pruneTables(tableSet);
     } else if (cl.hasOption(optTableName.getOpt())) {
       tableSet.add(cl.getOptionValue(optTableName.getOpt()));
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java
index 77b4b0c..e77ff5a 100644
--- a/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java
+++ b/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java
@@ -153,8 +153,9 @@ public class AccumuloClassLoader {
         urls.add(extDir.toURI().toURL());
       else {
         if (extDir.getParentFile() != null) {
+          var pattern = Pattern.compile(extDir.getName());
           File[] extJars =
-              extDir.getParentFile().listFiles((dir, name) -> name.matches("^" + extDir.getName()));
+              extDir.getParentFile().listFiles((dir, name) -> pattern.matcher(name).matches());
           if (extJars != null && extJars.length > 0) {
             for (File jar : extJars)
               urls.add(jar.toURI().toURL());
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
index 00c6ec1..34a761a 100644
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
+++ b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.function.Supplier;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.start.classloader.AccumuloClassLoader;
 import org.apache.commons.io.FileUtils;
@@ -148,8 +149,8 @@ public class AccumuloVFSClassLoader {
           pathsToMonitor.add(fo);
           break;
         case IMAGINARY:
-          // assume its a pattern
-          String pattern = fo.getName().getBaseName();
+          // assume it's a pattern
+          var pattern = Pattern.compile(fo.getName().getBaseName());
           if (fo.getParent() != null) {
             // still monitor the parent
             pathsToMonitor.add(fo.getParent());
@@ -157,7 +158,7 @@ public class AccumuloVFSClassLoader {
               FileObject[] children = fo.getParent().getChildren();
               for (FileObject child : children) {
                 if (child.getType() == FileType.FILE
-                    && child.getName().getBaseName().matches(pattern)) {
+                    && pattern.matcher(child.getName().getBaseName()).matches()) {
                   classpath.add(child);
                 }
               }
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloClasspathTest.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloClasspathTest.java
index c1e5e2f..f2ff3ec 100644
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloClasspathTest.java
+++ b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloClasspathTest.java
@@ -21,29 +21,32 @@ package org.apache.accumulo.start.classloader.vfs;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.util.regex.Pattern;
+
 import org.junit.Test;
 
 public class AccumuloClasspathTest {
 
-  private static void assertPattern(String output, String pattern, boolean shouldMatch) {
+  private static void assertPattern(String output, Pattern pattern, boolean shouldMatch) {
     if (shouldMatch) {
       assertTrue("Pattern " + pattern + " did not match output: " + output,
-          output.matches(pattern));
+          pattern.matcher(output).matches());
     } else {
       assertFalse("Pattern " + pattern + " should not match output: " + output,
-          output.matches(pattern));
+          pattern.matcher(output).matches());
     }
   }
 
   @Test
   public void basic() {
-    assertPattern(getClassPath(true), "(?s).*\\s+.*\\n$", true);
+    var pattern = Pattern.compile("(?s).*\\s+.*\\n$");
+    assertPattern(getClassPath(true), pattern, true);
     assertTrue(getClassPath(true).contains("app"));
     assertTrue(getClassPath(true).contains("Level"));
 
     assertTrue(getClassPath(true).length() > getClassPath(false).length());
 
-    assertPattern(getClassPath(false), "(?s).*\\s+.*\\n$", false);
+    assertPattern(getClassPath(false), pattern, false);
     assertFalse(getClassPath(false).contains("app"));
     assertFalse(getClassPath(false).contains("Level"));
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java b/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
index 134f770..eb614f6 100644
--- a/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
@@ -95,7 +95,7 @@ public class AuditMessageIT extends ConfigurableMacBase {
   private AccumuloClient client;
 
   private static long findAuditMessage(ArrayList<String> input, String pattern) {
-    return input.stream().filter(s -> s.matches(".*" + pattern + ".*")).count();
+    return input.stream().filter(Pattern.compile(".*" + pattern + ".*").asMatchPredicate()).count();
   }
 
   /**
@@ -124,13 +124,13 @@ public class AuditMessageIT extends ConfigurableMacBase {
       // We want to grab the files called .out
       if (file.getName().contains(".out") && file.isFile() && file.canRead()) {
         try (java.util.Scanner it = new java.util.Scanner(file, UTF_8)) {
+          // strip off prefix, because log4j.properties does
+          final var pattern = Pattern.compile(".* \\["
+              + AuditedSecurityOperation.AUDITLOG.replace("org.apache.", "").replace(".", "[.]")
+              + "\\] .*");
           while (it.hasNext()) {
             String line = it.nextLine();
-            // strip off prefix, because log4j.properties does
-            String pattern = ".* \\["
-                + AuditedSecurityOperation.AUDITLOG.replace("org.apache.", "").replace(".", "[.]")
-                + "\\] .*";
-            if (line.matches(pattern)) {
+            if (pattern.matcher(line).matches()) {
               // Only include the message if startTimestamp is null. or the message occurred after
               // the startTimestamp value
               if ((lastAuditTimestamp == null)
@@ -330,9 +330,10 @@ public class AuditMessageIT extends ConfigurableMacBase {
     String filePrefix = "file:";
 
     try (java.util.Scanner it = new java.util.Scanner(distCpTxt, UTF_8)) {
+      var pattern = Pattern.compile(".*\\.rf");
       while (it.hasNext() && importFile == null) {
         String line = it.nextLine();
-        if (line.matches(".*\\.rf")) {
+        if (pattern.matcher(line).matches()) {
           importFile = new File(line.replaceFirst(filePrefix, ""));
         }
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index 4a8d4cd..012dc4b 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -57,6 +57,7 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
@@ -518,8 +519,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
     ts.shell.execCommand("du -h", false, false);
     String o = ts.output.get();
     // for some reason, there's a bit of fluctuation
-    assertTrue("Output did not match regex: '" + o + "'",
-        o.matches(".*[1-9][0-9][0-9]\\s\\[" + table + "]\\n"));
+    assertMatches(o, ".*[1-9][0-9][0-9]\\s\\[" + table + "]\\n");
     ts.exec("deletetable -f " + table);
   }
 
@@ -1725,10 +1725,10 @@ public class ShellServerIT extends SharedMiniClusterBase {
         // TODO: any way to tell if the client address is accurate? could be local IP, host,
         // loopback...?
         String hostPortPattern = ".+:\\d+";
-        assertTrue(tserver.matches(hostPortPattern));
+        assertMatches(tserver, hostPortPattern);
         assertTrue(accumuloClient.instanceOperations().getTabletServers().contains(tserver));
         String client = parts[1].trim();
-        assertTrue(client + " does not match " + hostPortPattern, client.matches(hostPortPattern));
+        assertMatches(client, hostPortPattern);
         // Scan ID should be a long (throwing an exception if it fails to parse)
         Long r = Long.parseLong(parts[11].trim());
         assertNotNull(r);
@@ -2225,7 +2225,8 @@ public class ShellServerIT extends SharedMiniClusterBase {
   }
 
   private static void assertMatches(String output, String pattern) {
-    assertTrue("Pattern " + pattern + " did not match output : " + output, output.matches(pattern));
+    var p = Pattern.compile(pattern).asMatchPredicate();
+    assertTrue("Pattern " + pattern + " did not match output : " + output, p.test(output));
   }
 
   private static void assertNotContains(String output, String subsequence) {
diff --git a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
index d4511ef..ac7ce09 100644
--- a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
@@ -26,6 +26,7 @@ import java.security.SecureRandom;
 import java.util.Random;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -120,6 +121,8 @@ public class VerifySerialRecoveryIT extends ConfigurableMacBase {
       // time
       boolean started = false;
       int recoveries = 0;
+      var pattern =
+          Pattern.compile(".*recovered \\d+ mutations creating \\d+ entries from \\d+ walogs.*");
       for (String line : result.split("\n")) {
         // ignore metadata tables
         if (line.contains("!0") || line.contains("+r"))
@@ -129,7 +132,7 @@ public class VerifySerialRecoveryIT extends ConfigurableMacBase {
           started = true;
           recoveries++;
         }
-        if (line.matches(".*recovered \\d+ mutations creating \\d+ entries from \\d+ walogs.*")) {
+        if (pattern.matcher(line).matches()) {
           assertTrue(started);
           started = false;
         }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
index acd4abb..89f04b1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
@@ -33,6 +33,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.core.client.Accumulo;
@@ -147,6 +148,7 @@ public class CloneTestIT extends AccumuloClusterHarness {
       Key k;
       Text cf = new Text(), cq = new Text();
       int itemsInspected = 0;
+      var pattern = Pattern.compile("[tc]-[0-9a-z]+");
       for (Entry<Key,Value> entry : s) {
         itemsInspected++;
         k = entry.getKey();
@@ -163,7 +165,7 @@ public class CloneTestIT extends AccumuloClusterHarness {
 
           String dirName = entry.getValue().toString();
 
-          assertTrue("Bad dir name " + dirName, dirName.matches("[tc]-[0-9a-z]+"));
+          assertTrue("Bad dir name " + dirName, pattern.matcher(dirName).matches());
         } else {
           fail("Got unexpected key-value: " + entry);
           throw new RuntimeException();
diff --git a/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java b/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
index c7dcf05..b4e0dd0 100644
--- a/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
@@ -18,9 +18,13 @@
  */
 package org.apache.accumulo.test.iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.util.ArrayList;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -62,18 +66,8 @@ public class RegExTest {
     }
   }
 
-  private void check(String regex, String val) throws Exception {
-    if (regex != null && !val.matches(regex)) {
-      throw new Exception(" " + val + " does not match " + regex);
-    }
-  }
-
-  private void check(String regex, Text val) throws Exception {
-    check(regex, val.toString());
-  }
-
-  private void check(String regex, Value val) throws Exception {
-    check(regex, val.toString());
+  private void assertMatches(Pattern regex, Object val) throws Exception {
+    assertTrue(" " + val + " does not match " + regex, regex.matcher(val.toString()).matches());
   }
 
   @Test
@@ -121,30 +115,27 @@ public class RegExTest {
     RegExFilter iter = new RegExFilter();
     iter.init(source, is.getOptions(), null);
     iter.seek(range, es, false);
-    runTest(iter, rowRegEx, cfRegEx, cqRegEx, valRegEx, expected);
-  }
-
-  private void runTest(RegExFilter scanner, String rowRegEx, String cfRegEx, String cqRegEx,
-      String valRegEx, int expected) throws Exception {
 
     int counter = 0;
 
-    while (scanner.hasTop()) {
-      Key k = scanner.getTopKey();
+    var rowPattern = Pattern.compile(rowRegEx == null ? ".*" : rowRegEx);
+    var cfPattern = Pattern.compile(cfRegEx == null ? ".*" : cfRegEx);
+    var cqPattern = Pattern.compile(cqRegEx == null ? ".*" : cqRegEx);
+    var valPattern = Pattern.compile(valRegEx == null ? ".*" : valRegEx);
 
-      check(rowRegEx, k.getRow());
-      check(cfRegEx, k.getColumnFamily());
-      check(cqRegEx, k.getColumnQualifier());
-      check(valRegEx, scanner.getTopValue());
+    while (iter.hasTop()) {
+      Key k = iter.getTopKey();
 
-      scanner.next();
+      assertMatches(rowPattern, k.getRow());
+      assertMatches(cfPattern, k.getColumnFamily());
+      assertMatches(cqPattern, k.getColumnQualifier());
+      assertMatches(valPattern, iter.getTopValue());
+
+      iter.next();
 
       counter++;
     }
 
-    if (counter != expected) {
-      throw new Exception(
-          "scan did not return the expected number of entries " + counter + " " + expected);
-    }
+    assertEquals("scan did not return the expected number of entries", expected, counter);
   }
 }