You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2023/05/02 22:15:30 UTC

[accumulo] branch elasticity updated: updates tablet locs using conditional mutations (#3317)

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

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


The following commit(s) were added to refs/heads/elasticity by this push:
     new 950c806e20 updates tablet locs using conditional mutations (#3317)
950c806e20 is described below

commit 950c806e20abf91d1fc8fc2ececfcf0f935a4d15
Author: Keith Turner <kt...@apache.org>
AuthorDate: Tue May 2 18:15:24 2023 -0400

    updates tablet locs using conditional mutations (#3317)
    
    Updates tablet locations using conditional mutations. Two supporting changes
    were made and one bug was fixes while making this change.
    
    The first supporting change was streamlining handling of conditional mutations
    with a result of UNKNOWN.  An UNKNOWN result on a conditional mutation occurs
    when the RPC for the conditional mutation has an error.  In this case the
    conditional mutation may or may not have gone through. The tablet must be read
    to know what happened.  This update adds support for automatically reading the
    tablet and checking it via a lambda. This makes it easy to write code for
    handling the unknown case.
    
    The second supporting change was combining code that was mostly the same in
    ZooTabletStateStore and MetaDataStateStore by making both extend
    AbstractStateStore and use common code.  This change allowed the updates to use
    conditional mutations to be made in one place instead of two.
    
    The bug was with the new conditional writer code, it only supported writing
    tablets of the same table.  The code was changed to only require that tablets
    be on the same datalevel.  This change allowed ITs that create multiple tables
    to run.
    
    Some places in the code that set locations were not changed to use conditional
    mutations.  Comments were placed in the code for these.
    
    fixes #3284
---
 .../accumulo/core/metadata/schema/Ample.java       |  43 ++++-
 .../manager/state/AbstractTabletStateStore.java    | 174 +++++++++++++++++++++
 .../server/manager/state/MetaDataStateStore.java   | 115 +++-----------
 .../server/manager/state/ZooTabletStateStore.java  |  74 +++------
 .../metadata/ConditionalTabletMutatorImpl.java     |  22 ++-
 .../metadata/ConditionalTabletsMutatorImpl.java    | 101 ++++++++++--
 .../server/metadata/RootConditionalWriter.java     |   3 +-
 .../accumulo/server/util/ManagerMetadataUtil.java  |  11 +-
 .../manager/state/RootTabletStateStoreTest.java    |  55 ++++---
 .../ConditionalTabletsMutatorImplTest.java         | 140 +++++++++++++++++
 10 files changed, 558 insertions(+), 180 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
index d7a7177da2..c19729ddef 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.UUID;
+import java.util.function.Predicate;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.ConditionalWriter;
@@ -246,6 +247,30 @@ public interface Ample {
     void close();
   }
 
+  public interface ConditionalResult {
+
+    /**
+     * Returns the status of the conditional mutation. If the status was
+     * {@link org.apache.accumulo.core.client.ConditionalWriter.Status#UNKNOWN} and
+     * Ample#UknownValidator indicates things are ok then this will return
+     * {@link org.apache.accumulo.core.client.ConditionalWriter.Status#ACCEPTED}
+     */
+    ConditionalWriter.Status getStatus();
+
+    KeyExtent getExtent();
+
+    /**
+     * This can only be called when {@link #getStatus()} returns something other than
+     * {@link org.apache.accumulo.core.client.ConditionalWriter.Status#ACCEPTED}. It reads that
+     * tablets metadata for a failed conditional mutation. This can used used to see why it failed.
+     * In the case where {@link #getStatus()} returns
+     * {@link org.apache.accumulo.core.client.ConditionalWriter.Status#UNKNOWN} it can be used to
+     * see if the mutation actually succeeded or not.
+     *
+     */
+    TabletMetadata readMetadata();
+  }
+
   public interface ConditionalTabletsMutator extends AutoCloseable {
 
     /**
@@ -260,7 +285,7 @@ public interface Ample {
      *
      * @return The result from the {@link ConditionalWriter} of processing each tablet.
      */
-    Map<KeyExtent,ConditionalWriter.Result> process();
+    Map<KeyExtent,ConditionalResult> process();
 
     @Override
     void close();
@@ -371,6 +396,11 @@ public interface Ample {
     ConditionalTabletMutator requireAbsentTablet();
   }
 
+  /**
+   * Convenience interface for handling conditional mutations with a status of UNKNOWN.
+   */
+  interface UnknownValidator extends Predicate<TabletMetadata> {}
+
   interface ConditionalTabletMutator extends TabletUpdates<ConditionalTabletMutator> {
 
     /**
@@ -401,7 +431,16 @@ public interface Ample {
     /**
      * Submits or queues a conditional mutation for processing.
      */
-    ConditionalTabletsMutator submit();
+    void submit();
+
+    /**
+     * @param unknownCheck if the conditional mutation comes back with a status of
+     *        {@link org.apache.accumulo.core.client.ConditionalWriter.Status#UNKNOWN} then read the
+     *        tablets metadata and apply this check to see if it should be considered as
+     *        {@link org.apache.accumulo.core.client.ConditionalWriter.Status#ACCEPTED} in the
+     *        return of {@link ConditionalTabletsMutator#process()}
+     */
+    void submit(UnknownValidator unknownCheck);
   }
 
   /**
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java
new file mode 100644
index 0000000000..f1636ea882
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.manager.state;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.TabletLocationState;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.server.util.ManagerMetadataUtil;
+import org.apache.hadoop.fs.Path;
+
+import com.google.common.base.Preconditions;
+
+public abstract class AbstractTabletStateStore implements TabletStateStore {
+
+  private final ClientContext context;
+  private final Ample ample;
+
+  protected AbstractTabletStateStore(ClientContext context) {
+    this.context = context;
+    this.ample = context.getAmple();
+  }
+
+  @Override
+  public void setLocations(Collection<Assignment> assignments) throws DistributedStoreException {
+    try (var tabletsMutator = ample.conditionallyMutateTablets()) {
+      for (Assignment assignment : assignments) {
+        var conditionalMutator = tabletsMutator.mutateTablet(assignment.tablet)
+            .requireAbsentOperation()
+            .requireLocation(TabletMetadata.Location.future(assignment.server))
+            .putLocation(TabletMetadata.Location.current(assignment.server))
+            .deleteLocation(TabletMetadata.Location.future(assignment.server)).deleteSuspension();
+
+        ManagerMetadataUtil.updateLastForAssignmentMode(context, ample, conditionalMutator,
+            assignment.tablet, assignment.server);
+
+        conditionalMutator.submit(tabletMetadata -> {
+          Preconditions.checkArgument(tabletMetadata.getExtent().equals(assignment.tablet));
+          // see if we are the current location, if so then the unknown mutation actually
+          // succeeded
+          return tabletMetadata.getLocation() != null && tabletMetadata.getLocation()
+              .equals(TabletMetadata.Location.current(assignment.server));
+        });
+      }
+
+      if (tabletsMutator.process().values().stream()
+          .anyMatch(result -> result.getStatus() != ConditionalWriter.Status.ACCEPTED)) {
+        // TODO should this look at why?
+        throw new DistributedStoreException(
+            "failed to set tablet location, conditional mutation failed");
+      }
+    } catch (RuntimeException ex) {
+      throw new DistributedStoreException(ex);
+    }
+  }
+
+  @Override
+  public void setFutureLocations(Collection<Assignment> assignments)
+      throws DistributedStoreException {
+    try (var tabletsMutator = ample.conditionallyMutateTablets()) {
+      for (Assignment assignment : assignments) {
+        tabletsMutator.mutateTablet(assignment.tablet).requireAbsentOperation()
+            .requireAbsentLocation().deleteSuspension()
+            .putLocation(TabletMetadata.Location.future(assignment.server))
+            .submit(tabletMetadata -> {
+              Preconditions.checkArgument(tabletMetadata.getExtent().equals(assignment.tablet));
+              // see if we are the future location, if so then the unknown mutation actually
+              // succeeded
+              return tabletMetadata.getLocation() != null && tabletMetadata.getLocation()
+                  .equals(TabletMetadata.Location.future(assignment.server));
+            });
+      }
+
+      if (tabletsMutator.process().values().stream()
+          .anyMatch(result -> result.getStatus() != ConditionalWriter.Status.ACCEPTED)) {
+        // TODO should this look at why?
+        throw new DistributedStoreException(
+            "failed to set tablet location, conditional mutation failed");
+      }
+
+    } catch (RuntimeException ex) {
+      throw new DistributedStoreException(ex);
+    }
+  }
+
+  @Override
+  public void unassign(Collection<TabletLocationState> tablets,
+      Map<TServerInstance,List<Path>> logsForDeadServers) throws DistributedStoreException {
+    unassign(tablets, logsForDeadServers, -1);
+  }
+
+  @Override
+  public void suspend(Collection<TabletLocationState> tablets,
+      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
+      throws DistributedStoreException {
+    unassign(tablets, logsForDeadServers, suspensionTimestamp);
+  }
+
+  protected abstract void processSuspension(Ample.ConditionalTabletMutator tabletMutator,
+      TabletLocationState tls, long suspensionTimestamp);
+
+  private void unassign(Collection<TabletLocationState> tablets,
+      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
+      throws DistributedStoreException {
+    try (var tabletsMutator = ample.conditionallyMutateTablets()) {
+      for (TabletLocationState tls : tablets) {
+        var tabletMutator = tabletsMutator.mutateTablet(tls.extent).requireAbsentOperation();
+
+        if (tls.hasCurrent()) {
+          tabletMutator.requireLocation(tls.current);
+
+          ManagerMetadataUtil.updateLastForAssignmentMode(context, ample, tabletMutator, tls.extent,
+              tls.current.getServerInstance());
+          tabletMutator.deleteLocation(tls.current);
+          if (logsForDeadServers != null) {
+            List<Path> logs = logsForDeadServers.get(tls.current.getServerInstance());
+            if (logs != null) {
+              for (Path log : logs) {
+                LogEntry entry = new LogEntry(tls.extent, 0, log.toString());
+                tabletMutator.putWal(entry);
+              }
+            }
+          }
+        }
+
+        if (tls.hasFuture()) {
+          tabletMutator.requireLocation(tls.future);
+          tabletMutator.deleteLocation(tls.future);
+        }
+
+        processSuspension(tabletMutator, tls, suspensionTimestamp);
+
+        tabletMutator.submit(tabletMetadata -> {
+          // The status of the conditional update is unknown, so check and see if things are ok
+          return tabletMetadata.getLocation() == null;
+        });
+      }
+
+      Map<KeyExtent,Ample.ConditionalResult> results = tabletsMutator.process();
+
+      if (results.values().stream().anyMatch(conditionalResult -> conditionalResult.getStatus()
+          != ConditionalWriter.Status.ACCEPTED)) {
+        throw new DistributedStoreException("Some unassignments did not satisfy conditions.");
+      }
+
+    } catch (RuntimeException ex) {
+      throw new DistributedStoreException(ex);
+    }
+  }
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
index 680f4d1db9..86cd47552e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
@@ -19,22 +19,15 @@
 package org.apache.accumulo.server.manager.state;
 
 import java.util.Collection;
-import java.util.List;
-import java.util.Map;
 
+import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample;
-import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.server.util.ManagerMetadataUtil;
-import org.apache.hadoop.fs.Path;
 
-class MetaDataStateStore implements TabletStateStore {
+class MetaDataStateStore extends AbstractTabletStateStore implements TabletStateStore {
 
   protected final ClientContext context;
   protected final CurrentState state;
@@ -42,6 +35,7 @@ class MetaDataStateStore implements TabletStateStore {
   private final Ample ample;
 
   protected MetaDataStateStore(ClientContext context, CurrentState state, String targetTableName) {
+    super(context);
     this.context = context;
     this.state = state;
     this.ample = context.getAmple();
@@ -58,29 +52,22 @@ class MetaDataStateStore implements TabletStateStore {
   }
 
   @Override
-  public void setLocations(Collection<Assignment> assignments) throws DistributedStoreException {
-    try (var tabletsMutator = ample.mutateTablets()) {
-      for (Assignment assignment : assignments) {
-        TabletMutator tabletMutator = tabletsMutator.mutateTablet(assignment.tablet);
-        tabletMutator.putLocation(Location.current(assignment.server));
-        ManagerMetadataUtil.updateLastForAssignmentMode(context, ample, tabletMutator,
-            assignment.tablet, assignment.server);
-        tabletMutator.deleteLocation(Location.future(assignment.server));
-        tabletMutator.deleteSuspension();
-        tabletMutator.mutate();
+  public void unsuspend(Collection<TabletLocationState> tablets) throws DistributedStoreException {
+    try (var tabletsMutator = ample.conditionallyMutateTablets()) {
+      for (TabletLocationState tls : tablets) {
+        if (tls.suspend != null) {
+          continue;
+        }
+
+        // ELASTICITY_TODO pending #3314, add conditional mutation check that tls.suspend exists
+        tabletsMutator.mutateTablet(tls.extent).requireAbsentOperation().deleteSuspension()
+            .submit(tabletMetadata -> tabletMetadata.getSuspend() == null);
       }
-    } catch (RuntimeException ex) {
-      throw new DistributedStoreException(ex);
-    }
-  }
 
-  @Override
-  public void setFutureLocations(Collection<Assignment> assignments)
-      throws DistributedStoreException {
-    try (var tabletsMutator = ample.mutateTablets()) {
-      for (Assignment assignment : assignments) {
-        tabletsMutator.mutateTablet(assignment.tablet).deleteSuspension()
-            .putLocation(Location.future(assignment.server)).mutate();
+      boolean unacceptedConditions = tabletsMutator.process().values().stream().anyMatch(
+          conditionalResult -> conditionalResult.getStatus() != ConditionalWriter.Status.ACCEPTED);
+      if (unacceptedConditions) {
+        throw new DistributedStoreException("Some mutations failed to satisfy conditions");
       }
     } catch (RuntimeException ex) {
       throw new DistributedStoreException(ex);
@@ -88,71 +75,21 @@ class MetaDataStateStore implements TabletStateStore {
   }
 
   @Override
-  public void unassign(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers) throws DistributedStoreException {
-    unassign(tablets, logsForDeadServers, -1);
+  public String name() {
+    return "Normal Tablets";
   }
 
   @Override
-  public void suspend(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
-      throws DistributedStoreException {
-    unassign(tablets, logsForDeadServers, suspensionTimestamp);
-  }
-
-  private void unassign(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
-      throws DistributedStoreException {
-    try (var tabletsMutator = ample.mutateTablets()) {
-      for (TabletLocationState tls : tablets) {
-        TabletMutator tabletMutator = tabletsMutator.mutateTablet(tls.extent);
-        if (tls.current != null) {
-          ManagerMetadataUtil.updateLastForAssignmentMode(context, ample, tabletMutator, tls.extent,
-              tls.current.getServerInstance());
-          tabletMutator.deleteLocation(tls.current);
-          if (logsForDeadServers != null) {
-            List<Path> logs = logsForDeadServers.get(tls.current.getServerInstance());
-            if (logs != null) {
-              for (Path log : logs) {
-                LogEntry entry = new LogEntry(tls.extent, 0, log.toString());
-                tabletMutator.putWal(entry);
-              }
-            }
-          }
-          if (suspensionTimestamp >= 0) {
-            tabletMutator.putSuspension(tls.current.getServerInstance(), suspensionTimestamp);
-          }
-        }
-        if (tls.suspend != null && suspensionTimestamp < 0) {
-          tabletMutator.deleteSuspension();
-        }
-        if (tls.hasFuture()) {
-          tabletMutator.deleteLocation(tls.future);
-        }
-        tabletMutator.mutate();
+  protected void processSuspension(Ample.ConditionalTabletMutator tabletMutator,
+      TabletLocationState tls, long suspensionTimestamp) {
+    if (tls.current != null) {
+      if (suspensionTimestamp >= 0) {
+        tabletMutator.putSuspension(tls.current.getServerInstance(), suspensionTimestamp);
       }
-    } catch (RuntimeException ex) {
-      throw new DistributedStoreException(ex);
     }
-  }
 
-  @Override
-  public void unsuspend(Collection<TabletLocationState> tablets) throws DistributedStoreException {
-    try (var tabletsMutator = ample.mutateTablets()) {
-      for (TabletLocationState tls : tablets) {
-        if (tls.suspend != null) {
-          continue;
-        }
-        tabletsMutator.mutateTablet(tls.extent).deleteSuspension().mutate();
-      }
-    } catch (RuntimeException ex) {
-      throw new DistributedStoreException(ex);
+    if (tls.suspend != null && suspensionTimestamp < 0) {
+      tabletMutator.deleteSuspension();
     }
   }
-
-  @Override
-  public String name() {
-    return "Normal Tablets";
-  }
-
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
index dd4f89a438..e5e5cc4869 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
@@ -31,23 +31,21 @@ import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.ReadConsistency;
-import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-class ZooTabletStateStore implements TabletStateStore {
+class ZooTabletStateStore extends AbstractTabletStateStore implements TabletStateStore {
 
   private static final Logger log = LoggerFactory.getLogger(ZooTabletStateStore.class);
   private final Ample ample;
   private final ClientContext context;
 
   ZooTabletStateStore(ClientContext context) {
+    super(context);
     this.context = context;
     this.ample = context.getAmple();
   }
@@ -111,9 +109,7 @@ class ZooTabletStateStore implements TabletStateStore {
     };
   }
 
-  @Override
-  public void setFutureLocations(Collection<Assignment> assignments)
-      throws DistributedStoreException {
+  private static void validateAssignments(Collection<Assignment> assignments) {
     if (assignments.size() != 1) {
       throw new IllegalArgumentException("There is only one root tablet");
     }
@@ -121,34 +117,22 @@ class ZooTabletStateStore implements TabletStateStore {
     if (assignment.tablet.compareTo(RootTable.EXTENT) != 0) {
       throw new IllegalArgumentException("You can only store the root tablet location");
     }
+  }
 
-    TabletMutator tabletMutator = ample.mutateTablet(assignment.tablet);
-    tabletMutator.putLocation(Location.future(assignment.server));
-    tabletMutator.mutate();
+  @Override
+  public void setFutureLocations(Collection<Assignment> assignments)
+      throws DistributedStoreException {
+    validateAssignments(assignments);
+    super.setFutureLocations(assignments);
   }
 
   @Override
   public void setLocations(Collection<Assignment> assignments) throws DistributedStoreException {
-    if (assignments.size() != 1) {
-      throw new IllegalArgumentException("There is only one root tablet");
-    }
-    Assignment assignment = assignments.iterator().next();
-    if (assignment.tablet.compareTo(RootTable.EXTENT) != 0) {
-      throw new IllegalArgumentException("You can only store the root tablet location");
-    }
-
-    TabletMutator tabletMutator = ample.mutateTablet(assignment.tablet);
-    tabletMutator.putLocation(Location.current(assignment.server));
-    ManagerMetadataUtil.updateLastForAssignmentMode(context, ample, tabletMutator,
-        assignment.tablet, assignment.server);
-    tabletMutator.deleteLocation(Location.future(assignment.server));
-
-    tabletMutator.mutate();
+    validateAssignments(assignments);
+    super.setLocations(assignments);
   }
 
-  @Override
-  public void unassign(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers) throws DistributedStoreException {
+  private static void validateTablets(Collection<TabletLocationState> tablets) {
     if (tablets.size() != 1) {
       throw new IllegalArgumentException("There is only one root tablet");
     }
@@ -156,26 +140,14 @@ class ZooTabletStateStore implements TabletStateStore {
     if (tls.extent.compareTo(RootTable.EXTENT) != 0) {
       throw new IllegalArgumentException("You can only store the root tablet location");
     }
+  }
 
-    TabletMutator tabletMutator = ample.mutateTablet(tls.extent);
-    final TServerInstance futureOrCurrent = tls.futureOrCurrent().getServerInstance();
-
-    tabletMutator.deleteLocation(Location.future(futureOrCurrent));
-    tabletMutator.deleteLocation(Location.current(futureOrCurrent));
-    ManagerMetadataUtil.updateLastForAssignmentMode(context, ample, tabletMutator, tls.extent,
-        futureOrCurrent);
-    if (logsForDeadServers != null) {
-      List<Path> logs = logsForDeadServers.get(futureOrCurrent);
-      if (logs != null) {
-        for (Path entry : logs) {
-          LogEntry logEntry =
-              new LogEntry(RootTable.EXTENT, System.currentTimeMillis(), entry.toString());
-          tabletMutator.putWal(logEntry);
-        }
-      }
-    }
+  @Override
+  public void unassign(Collection<TabletLocationState> tablets,
+      Map<TServerInstance,List<Path>> logsForDeadServers) throws DistributedStoreException {
+    validateTablets(tablets);
 
-    tabletMutator.mutate();
+    super.unassign(tablets, logsForDeadServers);
 
     log.debug("unassign root tablet location");
   }
@@ -184,8 +156,14 @@ class ZooTabletStateStore implements TabletStateStore {
   public void suspend(Collection<TabletLocationState> tablets,
       Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
       throws DistributedStoreException {
-    // No support for suspending root tablet.
-    unassign(tablets, logsForDeadServers);
+    validateTablets(tablets);
+    super.suspend(tablets, logsForDeadServers, suspensionTimestamp);
+  }
+
+  @Override
+  protected void processSuspension(Ample.ConditionalTabletMutator tabletMutator,
+      TabletLocationState tls, long suspensionTimestamp) {
+    // No support for suspending root tablet, so this is a NOOP
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java
index bd9effe468..6cfd3add13 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java
@@ -23,6 +23,7 @@ import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSec
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.encodePrevEndRow;
 
+import java.util.function.BiConsumer;
 import java.util.function.Consumer;
 
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -54,14 +55,21 @@ public class ConditionalTabletMutatorImpl extends TabletMutatorBase<Ample.Condit
   private final Consumer<ConditionalMutation> mutationConsumer;
   private final Ample.ConditionalTabletsMutator parent;
 
+  private final BiConsumer<KeyExtent,Ample.UnknownValidator> unknownValidators;
+
+  private final KeyExtent extent;
+
   private boolean sawOperationRequirement = false;
 
   protected ConditionalTabletMutatorImpl(Ample.ConditionalTabletsMutator parent,
-      ServerContext context, KeyExtent extent, Consumer<ConditionalMutation> mutationConsumer) {
+      ServerContext context, KeyExtent extent, Consumer<ConditionalMutation> mutationConsumer,
+      BiConsumer<KeyExtent,Ample.UnknownValidator> unknownValidators) {
     super(context, new ConditionalMutation(extent.toMetaRow()));
     this.mutation = (ConditionalMutation) super.mutation;
     this.mutationConsumer = mutationConsumer;
     this.parent = parent;
+    this.unknownValidators = unknownValidators;
+    this.extent = extent;
   }
 
   @Override
@@ -142,11 +150,19 @@ public class ConditionalTabletMutatorImpl extends TabletMutatorBase<Ample.Condit
   }
 
   @Override
-  public Ample.ConditionalTabletsMutator submit() {
+  public void submit() {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
+    Preconditions.checkState(sawOperationRequirement, "No operation requirements were seen");
+    getMutation();
+    mutationConsumer.accept(mutation);
+  }
+
+  @Override
+  public void submit(Ample.UnknownValidator unknownCheck) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
     Preconditions.checkState(sawOperationRequirement, "No operation requirements were seen");
     getMutation();
     mutationConsumer.accept(mutation);
-    return parent;
+    unknownValidators.accept(extent, unknownCheck);
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletsMutatorImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletsMutatorImpl.java
index 7aaf149211..f8da9b26c3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletsMutatorImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletsMutatorImpl.java
@@ -23,22 +23,27 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.ConditionalMutation;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Maps;
 
 public class ConditionalTabletsMutatorImpl implements Ample.ConditionalTabletsMutator {
 
   private final ServerContext context;
-  private TableId currentTableId = null;
+  private Ample.DataLevel dataLevel = null;
 
   private List<ConditionalMutation> mutations = new ArrayList<>();
 
@@ -46,6 +51,8 @@ public class ConditionalTabletsMutatorImpl implements Ample.ConditionalTabletsMu
 
   private boolean active = true;
 
+  Map<KeyExtent,Ample.UnknownValidator> unknownValidators = new HashMap<>();
+
   public ConditionalTabletsMutatorImpl(ServerContext context) {
     this.context = context;
   }
@@ -53,19 +60,23 @@ public class ConditionalTabletsMutatorImpl implements Ample.ConditionalTabletsMu
   @Override
   public Ample.OperationRequirements mutateTablet(KeyExtent extent) {
     Preconditions.checkState(active);
-    if (currentTableId == null) {
-      currentTableId = extent.tableId();
-    } else if (!currentTableId.equals(extent.tableId())) {
+
+    var dataLevel = Ample.DataLevel.of(extent.tableId());
+
+    if (this.dataLevel == null) {
+      this.dataLevel = dataLevel;
+    } else if (!this.dataLevel.equals(dataLevel)) {
       throw new IllegalArgumentException(
-          "Can not mix tables ids " + currentTableId + " " + extent.tableId());
+          "Can not mix data levels " + this.dataLevel + " " + dataLevel);
     }
 
     Preconditions.checkState(extents.putIfAbsent(extent.toMetaRow(), extent) == null,
         "Duplicate extents not handled");
-    return new ConditionalTabletMutatorImpl(this, context, extent, mutations::add);
+    return new ConditionalTabletMutatorImpl(this, context, extent, mutations::add,
+        unknownValidators::put);
   }
 
-  private ConditionalWriter createConditionalWriter(Ample.DataLevel dataLevel)
+  protected ConditionalWriter createConditionalWriter(Ample.DataLevel dataLevel)
       throws TableNotFoundException {
     if (dataLevel == Ample.DataLevel.ROOT) {
       return new RootConditionalWriter(context);
@@ -74,11 +85,39 @@ public class ConditionalTabletsMutatorImpl implements Ample.ConditionalTabletsMu
     }
   }
 
+  protected Map<KeyExtent,TabletMetadata> readTablets(List<KeyExtent> extents) {
+    Map<KeyExtent,TabletMetadata> failedTablets = new HashMap<>();
+
+    try (var tabletsMeta = context.getAmple().readTablets().forTablets(extents).build()) {
+      tabletsMeta
+          .forEach(tabletMetadata -> failedTablets.put(tabletMetadata.getExtent(), tabletMetadata));
+    }
+
+    return failedTablets;
+  }
+
+  private Map<KeyExtent,TabletMetadata>
+      readFailedTablets(Map<KeyExtent,ConditionalWriter.Result> results) {
+
+    var extents = results.entrySet().stream().filter(e -> {
+      try {
+        return e.getValue().getStatus() != ConditionalWriter.Status.ACCEPTED;
+      } catch (AccumuloException | AccumuloSecurityException ex) {
+        throw new RuntimeException(ex);
+      }
+    }).map(Map.Entry::getKey).collect(Collectors.toList());
+
+    if (extents.isEmpty()) {
+      return Map.of();
+    }
+
+    return readTablets(extents);
+  }
+
   @Override
-  public Map<KeyExtent,ConditionalWriter.Result> process() {
+  public Map<KeyExtent,Ample.ConditionalResult> process() {
     Preconditions.checkState(active);
-    if (currentTableId != null) {
-      var dataLevel = Ample.DataLevel.of(currentTableId);
+    if (dataLevel != null) {
       try (ConditionalWriter conditionalWriter = createConditionalWriter(dataLevel)) {
         var results = conditionalWriter.write(mutations.iterator());
 
@@ -94,7 +133,45 @@ public class ConditionalTabletsMutatorImpl implements Ample.ConditionalTabletsMu
           throw new AssertionError("Not all extents were seen, this is unexpected");
         }
 
-        return resultsMap;
+        // only fetch the metadata for failures when requested and when it is requested fetch all
+        // of the failed extents at once to avoid fetching them one by one.
+        var failedMetadata = Suppliers.memoize(() -> readFailedTablets(resultsMap));
+
+        return Maps.transformEntries(resultsMap, (extent, result) -> new Ample.ConditionalResult() {
+
+          private ConditionalWriter.Status _getStatus() {
+            try {
+              return result.getStatus();
+            } catch (AccumuloException | AccumuloSecurityException e) {
+              throw new RuntimeException(e);
+            }
+          }
+
+          @Override
+          public ConditionalWriter.Status getStatus() {
+            var status = _getStatus();
+            if (status == ConditionalWriter.Status.UNKNOWN
+                && unknownValidators.containsKey(extent)) {
+              var tabletMetadata = readMetadata();
+              if (tabletMetadata != null && unknownValidators.get(extent).test(tabletMetadata)) {
+                return ConditionalWriter.Status.ACCEPTED;
+              }
+            }
+
+            return status;
+          }
+
+          @Override
+          public KeyExtent getExtent() {
+            return extent;
+          }
+
+          @Override
+          public TabletMetadata readMetadata() {
+            Preconditions.checkState(_getStatus() != ConditionalWriter.Status.ACCEPTED);
+            return failedMetadata.get().get(getExtent());
+          }
+        });
       } catch (TableNotFoundException e) {
         throw new RuntimeException(e);
       } finally {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootConditionalWriter.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootConditionalWriter.java
index d83f56273d..cbb33d7238 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootConditionalWriter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootConditionalWriter.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.clientImpl.ConditionalWriterImpl;
 import org.apache.accumulo.core.data.ConditionalMutation;
 import org.apache.accumulo.core.dataImpl.thrift.TCMResult;
 import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
+import org.apache.accumulo.core.iteratorsImpl.system.ColumnFamilySkippingIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SortedMapIterator;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
@@ -99,7 +100,7 @@ public class RootConditionalWriter implements ConditionalWriter {
         String currJson = new String(currVal, UTF_8);
         var rtm = new RootTabletMetadata(currJson);
 
-        var iter = new SortedMapIterator(rtm.toKeyValues());
+        var iter = new ColumnFamilySkippingIterator(new SortedMapIterator(rtm.toKeyValues()));
 
         // This could be called multiple times so clear before calling
         okMutations.clear();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
index e7b16e8e95..2739b74286 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
@@ -73,6 +73,10 @@ public class ManagerMetadataUtil {
       Map<Long,? extends Collection<TabletFile>> bulkLoadedFiles, MetadataTime time,
       long lastFlushID, long lastCompactID, ServiceLock zooLock) {
 
+    // ELASTICITY_TODO intentionally not using conditional mutations for this code because its only
+    // called when tablets split. Tablet splitting will drastically change, so there is no need to
+    // update this to use conditional mutations ATM.
+
     TabletMutator tablet = context.getAmple().mutateTablet(extent);
     tablet.putPrevEndRow(extent.prevEndRow());
     tablet.putZooLock(zooLock);
@@ -259,7 +263,7 @@ public class ManagerMetadataUtil {
    * @param location The new location
    */
   public static void updateLastForAssignmentMode(ClientContext context, Ample ample,
-      Ample.TabletMutator tabletMutator, KeyExtent extent, TServerInstance location) {
+      Ample.TabletUpdates<?> tabletMutator, KeyExtent extent, TServerInstance location) {
     // if the location mode is assignment, then preserve the current location in the last
     // location value
     if ("assignment".equals(context.getConfiguration().get(Property.TSERV_LAST_LOCATION_MODE))) {
@@ -296,8 +300,9 @@ public class ManagerMetadataUtil {
    * @param previousLocation The location (may be null)
    * @param newLocation The new location
    */
-  private static void updateLocation(TabletMutator tabletMutator, Location previousLocation,
-      Location newLocation) {
+  private static void updateLocation(Ample.TabletUpdates<?> tabletMutator,
+      Location previousLocation, Location newLocation) {
+    // ELASTICITY_TODO pending #3301, update this code to use conditional mutations
     if (previousLocation != null) {
       if (!previousLocation.equals(newLocation)) {
         tabletMutator.deleteLocation(previousLocation);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/RootTabletStateStoreTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/RootTabletStateStoreTest.java
index 98d19cba4a..4b408e87a0 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/RootTabletStateStoreTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/RootTabletStateStoreTest.java
@@ -27,10 +27,13 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.fail;
 
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TabletHostingGoal;
-import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.ConditionalMutation;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -45,11 +48,12 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.server.MockServerContext;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.metadata.TabletMutatorBase;
+import org.apache.accumulo.server.metadata.ConditionalTabletsMutatorImpl;
 import org.easymock.EasyMock;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
 import com.google.common.net.HostAndPort;
 
 public class RootTabletStateStoreTest {
@@ -71,25 +75,32 @@ public class RootTabletStateStoreTest {
       throw new UnsupportedOperationException("This method should be implemented in subclasses");
     }
 
-    private class TestTabletMutator extends TabletMutatorBase<TabletMutator>
-        implements TabletMutator {
-      public TestTabletMutator(ServerContext context, KeyExtent extent) {
-        super(context, extent);
-      }
-
-      public void mutate() {
-        Mutation m = getMutation();
-
-        var rtm = new RootTabletMetadata(json);
-        rtm.update(m);
-        json = rtm.toJson();
-      }
-    }
-
-    @Override
-    public TabletMutator mutateTablet(KeyExtent extent) {
-      Preconditions.checkArgument(extent.equals(RootTable.EXTENT));
-      return new TestTabletMutator(null, RootTable.EXTENT);
+    public ConditionalTabletsMutator conditionallyMutateTablets() {
+      return new ConditionalTabletsMutatorImpl(null) {
+        protected ConditionalWriter createConditionalWriter(Ample.DataLevel dataLevel)
+            throws TableNotFoundException {
+          Preconditions.checkArgument(dataLevel == DataLevel.ROOT);
+          return new ConditionalWriter() {
+            @Override
+            public Iterator<Result> write(Iterator<ConditionalMutation> mutations) {
+              return Iterators.transform(mutations, this::write);
+            }
+
+            @Override
+            public Result write(ConditionalMutation mutation) {
+              var rtm = new RootTabletMetadata(json);
+              rtm.update(mutation);
+              json = rtm.toJson();
+              return new Result(Status.ACCEPTED, mutation, "server");
+            }
+
+            @Override
+            public void close() {
+
+            }
+          };
+        }
+      };
     }
 
   }
@@ -125,7 +136,7 @@ public class RootTabletStateStoreTest {
     assertEquals(count, 1);
     TabletLocationState assigned = null;
     try {
-      assigned = new TabletLocationState(root, Location.future(server), null, null, null, null,
+      assigned = new TabletLocationState(root, null, Location.current(server), null, null, null,
           false, TabletHostingGoal.ALWAYS, false);
     } catch (BadLocationStateException e) {
       fail("Unexpected error " + e);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/metadata/ConditionalTabletsMutatorImplTest.java b/server/base/src/test/java/org/apache/accumulo/server/metadata/ConditionalTabletsMutatorImplTest.java
new file mode 100644
index 0000000000..569b6190a4
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/metadata/ConditionalTabletsMutatorImplTest.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.metadata;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriter.Status;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.ConditionalMutation;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.hadoop.io.Text;
+import org.easymock.EasyMock;
+import org.junit.jupiter.api.Test;
+
+public class ConditionalTabletsMutatorImplTest {
+
+  // a conditional tablet mutator that always returns a supplied status
+  static class TestConditionalTabletsMutator extends ConditionalTabletsMutatorImpl {
+
+    private final Map<KeyExtent,TabletMetadata> failedExtents;
+    private final Function<Text,Status> statuses;
+
+    public TestConditionalTabletsMutator(Function<Text,Status> statuses,
+        Map<KeyExtent,TabletMetadata> failedExtents) {
+      super(null);
+      this.statuses = statuses;
+      this.failedExtents = failedExtents;
+    }
+
+    protected Map<KeyExtent,TabletMetadata> readTablets(List<KeyExtent> extents) {
+      return failedExtents;
+    }
+
+    protected ConditionalWriter createConditionalWriter(Ample.DataLevel dataLevel)
+        throws TableNotFoundException {
+      return new ConditionalWriter() {
+        @Override
+        public Iterator<Result> write(Iterator<ConditionalMutation> mutations) {
+          Iterable<ConditionalMutation> iterable = () -> mutations;
+          return StreamSupport.stream(iterable.spliterator(), false)
+              .map(cm -> new Result(statuses.apply(new Text(cm.getRow())), cm, "server"))
+              .iterator();
+        }
+
+        @Override
+        public Result write(ConditionalMutation mutation) {
+          return write(List.of(mutation).iterator()).next();
+        }
+
+        @Override
+        public void close() {
+
+        }
+      };
+    }
+  }
+
+  @Test
+  public void testUnknownValidation() {
+
+    // this test checks the handling of conditional mutations that return a status of unknown
+
+    var ke1 = new KeyExtent(TableId.of("1"), null, null);
+
+    TabletMetadata tm1 = EasyMock.createMock(TabletMetadata.class);
+    EasyMock.expect(tm1.getDirName()).andReturn("dir1").anyTimes();
+    EasyMock.replay(tm1);
+
+    var ke2 = new KeyExtent(TableId.of("a"), null, null);
+
+    TabletMetadata tm2 = EasyMock.createMock(TabletMetadata.class);
+    EasyMock.expect(tm2.getDirName()).andReturn("dir2").anyTimes();
+    EasyMock.replay(tm2);
+
+    var ke3 = new KeyExtent(TableId.of("b"), null, null);
+    var ke4 = new KeyExtent(TableId.of("c"), null, null);
+
+    var failedExtents = Map.of(ke1, tm1, ke2, tm2);
+    var statuses = Map.of(ke1.toMetaRow(), Status.UNKNOWN, ke2.toMetaRow(), Status.UNKNOWN,
+        ke3.toMetaRow(), Status.REJECTED, ke4.toMetaRow(), Status.ACCEPTED);
+
+    try (var mutator = new TestConditionalTabletsMutator(statuses::get, failedExtents)) {
+      // passed in unknown handler should determine the mutations status should be accepted
+      mutator.mutateTablet(ke1).requireAbsentOperation().putDirName("dir1")
+          .submit(tmeta -> tmeta.getDirName().equals("dir1"));
+
+      // passed in unknown handler should determine the mutations status should continue to be
+      // UNKNOWN
+      mutator.mutateTablet(ke2).requireAbsentOperation().putDirName("dir3")
+          .submit(tmeta -> tmeta.getDirName().equals("dir3"));
+
+      // ensure the unknown handler is only called when the status is unknown, this mutation will
+      // have a status of REJECTED
+      mutator.mutateTablet(ke3).requireAbsentOperation().putDirName("dir3").submit(tmeta -> {
+        throw new IllegalStateException();
+      });
+
+      // ensure the unknown handler is only called when the status is unknown, this mutations will
+      // have a status of ACCEPTED
+      mutator.mutateTablet(ke4).requireAbsentOperation().putDirName("dir3").submit(tmeta -> {
+        throw new IllegalStateException();
+      });
+
+      Map<KeyExtent,Ample.ConditionalResult> results = mutator.process();
+
+      assertEquals(Set.of(ke1, ke2, ke3, ke4), results.keySet());
+      assertEquals(Status.ACCEPTED, results.get(ke1).getStatus());
+      assertEquals(Status.UNKNOWN, results.get(ke2).getStatus());
+      assertEquals(Status.REJECTED, results.get(ke3).getStatus());
+      assertEquals(Status.ACCEPTED, results.get(ke4).getStatus());
+    }
+  }
+}