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 0fc605342c Adds support for conditional mutations to Ample  (#3251)
0fc605342c is described below

commit 0fc605342cb615843c73d1c1b2e65399d056ae61
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Thu Apr 13 14:32:14 2023 -0400

    Adds support for conditional mutations to Ample  (#3251)
    
    In addition to adding conditional support to Ample, this change also adds 
new
    operation id metadata column.  The operation id is intended to support 
running
    mutually exclusive tablet operations.  The operation id is central to the 
goal
    of using conditional mutations to allow multiple processes to safely modify 
a
    tablet and that is why it is being introduced at the same time as general
    conditional mutation support.
---
 .../core/clientImpl/ConditionalWriterImpl.java     |  19 +-
 .../accumulo/core/metadata/TabletOperationId.java  |  33 +-
 .../accumulo/core/metadata/schema/Ample.java       | 145 +++++--
 .../core/metadata/schema/MetadataSchema.java       |  25 ++
 .../core/metadata/schema/RootTabletMetadata.java   |  18 +-
 .../core/metadata/schema/TabletMetadata.java       |  20 +
 .../core/metadata/schema/TabletOperation.java      |  33 +-
 .../server/constraints/MetadataConstraints.java    |   3 +-
 .../server}/data/ServerConditionalMutation.java    |   3 +-
 .../metadata/ConditionalTabletMutatorImpl.java     | 154 ++++++++
 .../metadata/ConditionalTabletsMutatorImpl.java    | 117 ++++++
 .../server/metadata/RootConditionalWriter.java     | 154 ++++++++
 .../server/metadata/RootTabletMutatorImpl.java     |   6 +-
 .../accumulo/server/metadata/ServerAmpleImpl.java  |   5 +
 .../server/metadata/TabletMutatorBase.java         | 140 ++++---
 .../server/metadata/TabletMutatorImpl.java         |   4 +-
 .../metadata/iterators/LocationExistsIterator.java |  68 ++++
 .../metadata/iterators/PresentIterator.java}       |  31 +-
 .../TabletExistsIterator.java}                     |  42 +-
 .../server/tablets}/ConditionCheckerContext.java   |   6 +-
 .../manager/state/RootTabletStateStoreTest.java    |  27 +-
 .../apache/accumulo/manager/TabletOperations.java} |  30 +-
 .../accumulo/tserver/ConditionalMutationSet.java   |   2 +-
 .../java/org/apache/accumulo/tserver/RowLocks.java |   2 +-
 .../accumulo/tserver/TabletClientHandler.java      |   5 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   2 +-
 .../test/functional/AmpleConditionalWriterIT.java  | 428 +++++++++++++++++++++
 27 files changed, 1283 insertions(+), 239 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
index d029d717c3..9c25135bab 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
@@ -89,7 +89,7 @@ import org.apache.thrift.transport.TTransportException;
 
 import com.google.common.net.HostAndPort;
 
-class ConditionalWriterImpl implements ConditionalWriter {
+public class ConditionalWriterImpl implements ConditionalWriter {
 
   private static final int MAX_SLEEP = 30000;
 
@@ -724,7 +724,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
     }
   }
 
-  private Status fromThrift(TCMStatus status) {
+  public static Status fromThrift(TCMStatus status) {
     switch (status) {
       case ACCEPTED:
         return Status.ACCEPTED;
@@ -745,12 +745,11 @@ class ConditionalWriterImpl implements ConditionalWriter {
       var tcondMutaions = new ArrayList<TConditionalMutation>();
 
       for (var cm : mutationList) {
-        TMutation tm = cm.toThrift();
+        var id = cmid.longValue();
 
-        List<TCondition> conditions = convertConditions(cm, compressedIters);
+        TConditionalMutation tcm = convertConditionalMutation(compressedIters, 
cm, id);
 
         cmidToCm.put(cmid.longValue(), new CMK(keyExtent, cm));
-        TConditionalMutation tcm = new TConditionalMutation(conditions, tm, 
cmid.longValue());
         cmid.increment();
         tcondMutaions.add(tcm);
       }
@@ -759,6 +758,14 @@ class ConditionalWriterImpl implements ConditionalWriter {
     });
   }
 
+  public static TConditionalMutation 
convertConditionalMutation(CompressedIterators compressedIters,
+      ConditionalMutation cm, long id) {
+    TMutation tm = cm.toThrift();
+    List<TCondition> conditions = convertConditions(cm, compressedIters);
+    TConditionalMutation tcm = new TConditionalMutation(conditions, tm, id);
+    return tcm;
+  }
+
   private static final Comparator<Long> TIMESTAMP_COMPARATOR =
       Comparator.nullsFirst(Comparator.reverseOrder());
 
@@ -767,7 +774,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
           .thenComparing(Condition::getVisibility)
           .thenComparing(Condition::getTimestamp, TIMESTAMP_COMPARATOR);
 
-  private List<TCondition> convertConditions(ConditionalMutation cm,
+  private static List<TCondition> convertConditions(ConditionalMutation cm,
       CompressedIterators compressedIters) {
     List<TCondition> conditions = new ArrayList<>(cm.getConditions().size());
 
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
 b/core/src/main/java/org/apache/accumulo/core/metadata/TabletOperationId.java
similarity index 55%
copy from 
server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
copy to 
core/src/main/java/org/apache/accumulo/core/metadata/TabletOperationId.java
index ee0db927a1..322c7e0f1f 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/TabletOperationId.java
@@ -16,32 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.data;
+package org.apache.accumulo.core.metadata;
 
-import java.util.List;
+import org.apache.accumulo.core.data.AbstractId;
 
-import org.apache.accumulo.core.dataImpl.thrift.TCondition;
-import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
-import org.apache.accumulo.server.data.ServerMutation;
-
-public class ServerConditionalMutation extends ServerMutation {
-
-  private long cmid;
-  private List<TCondition> conditions;
-
-  public ServerConditionalMutation(TConditionalMutation input) {
-    super(input.mutation);
-
-    this.cmid = input.id;
-    this.conditions = input.conditions;
-  }
+/**
+ * Intended to contain a globally unique id that identifies an operation 
running against a tablet.
+ * The purpose of this is to prevent race conditions.
+ */
+public class TabletOperationId extends AbstractId<TabletOperationId> {
 
-  public long getID() {
-    return cmid;
-  }
+  private static final long serialVersionUID = 1L;
 
-  public List<TCondition> getConditions() {
-    return conditions;
+  public TabletOperationId(String canonical) {
+    super(canonical);
   }
-
 }
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 4ba196b664..ca296f859f 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
@@ -20,9 +20,11 @@ package org.apache.accumulo.core.metadata.schema;
 
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.Map;
 import java.util.UUID;
 import java.util.stream.Stream;
 
+import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
@@ -34,6 +36,7 @@ import 
org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.TabletOperationId;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
@@ -182,6 +185,10 @@ public interface Ample {
     throw new UnsupportedOperationException();
   }
 
+  default ConditionalTabletsMutator conditionallyMutateTablets() {
+    throw new UnsupportedOperationException();
+  }
+
   default void putGcCandidates(TableId tableId, Collection<StoredTabletFile> 
candidates) {
     throw new UnsupportedOperationException();
   }
@@ -239,59 +246,84 @@ public interface Ample {
     void close();
   }
 
+  public interface ConditionalTabletsMutator extends AutoCloseable {
+
+    /**
+     * @return A fluent interface to conditional mutating a tablet. Ensure you 
call
+     *         {@link ConditionalTabletMutator#submit()} when finished.
+     */
+    OperationRequirements mutateTablet(KeyExtent extent);
+
+    /**
+     * After creating one or more conditional mutations using {@link 
#mutateTablet(KeyExtent)}, call
+     * this method to process them using a {@link ConditionalWriter}
+     *
+     * @return The result from the {@link ConditionalWriter} of processing 
each tablet.
+     */
+    Map<KeyExtent,ConditionalWriter.Result> process();
+
+    @Override
+    void close();
+  }
+
   /**
    * Interface for changing a tablets persistent data.
    */
-  interface TabletMutator {
-    TabletMutator putPrevEndRow(Text per);
+  interface TabletUpdates<T> {
+    T putPrevEndRow(Text per);
 
-    TabletMutator putFile(TabletFile path, DataFileValue dfv);
+    T putFile(TabletFile path, DataFileValue dfv);
 
-    TabletMutator deleteFile(StoredTabletFile path);
+    T deleteFile(StoredTabletFile path);
 
-    TabletMutator putScan(TabletFile path);
+    T putScan(TabletFile path);
 
-    TabletMutator deleteScan(StoredTabletFile path);
+    T deleteScan(StoredTabletFile path);
 
-    TabletMutator putCompactionId(long compactionId);
+    T putCompactionId(long compactionId);
 
-    TabletMutator putFlushId(long flushId);
+    T putFlushId(long flushId);
 
-    TabletMutator putLocation(Location location);
+    T putLocation(Location location);
 
-    TabletMutator deleteLocation(Location location);
+    T deleteLocation(Location location);
 
-    TabletMutator putZooLock(ServiceLock zooLock);
+    T putZooLock(ServiceLock zooLock);
 
-    TabletMutator putDirName(String dirName);
+    T putDirName(String dirName);
 
-    TabletMutator putWal(LogEntry logEntry);
+    T putWal(LogEntry logEntry);
 
-    TabletMutator deleteWal(String wal);
+    T deleteWal(String wal);
 
-    TabletMutator deleteWal(LogEntry logEntry);
+    T deleteWal(LogEntry logEntry);
 
-    TabletMutator putTime(MetadataTime time);
+    T putTime(MetadataTime time);
 
-    TabletMutator putBulkFile(TabletFile bulkref, long tid);
+    T putBulkFile(TabletFile bulkref, long tid);
 
-    TabletMutator deleteBulkFile(TabletFile bulkref);
+    T deleteBulkFile(TabletFile bulkref);
 
-    TabletMutator putChopped();
+    T putChopped();
 
-    TabletMutator putSuspension(TServerInstance tserver, long suspensionTime);
+    T putSuspension(TServerInstance tserver, long suspensionTime);
 
-    TabletMutator deleteSuspension();
+    T deleteSuspension();
 
-    TabletMutator putExternalCompaction(ExternalCompactionId ecid,
-        ExternalCompactionMetadata ecMeta);
+    T putExternalCompaction(ExternalCompactionId ecid, 
ExternalCompactionMetadata ecMeta);
 
-    TabletMutator deleteExternalCompaction(ExternalCompactionId ecid);
+    T deleteExternalCompaction(ExternalCompactionId ecid);
 
-    TabletMutator putOnDemand();
+    T putOperation(TabletOperation operation, TabletOperationId opId);
 
-    TabletMutator deleteOnDemand();
+    T deleteOperation();
 
+    T putOnDemand();
+
+    T deleteOnDemand();
+  }
+
+  interface TabletMutator extends TabletUpdates<TabletMutator> {
     /**
      * This method persist (or queues for persisting) previous put and deletes 
against this object.
      * Unless this method is called, previous calls will never be persisted. 
The purpose of this
@@ -307,6 +339,67 @@ public interface Ample {
     void mutate();
   }
 
+  /**
+   * A tablet operation is a mutually exclusive action that is running against 
a tablet. Its very
+   * important that every conditional mutation specifies requirements about 
operations in order to
+   * satisfy the mutual exclusion goal. This interface forces those 
requirements to specified by
+   * making it the only choice avialable before specifying other tablet 
requirements or mutations.
+   */
+  interface OperationRequirements {
+
+    /**
+     * Require a specific operation with a unique id is present. This would be 
normally be called by
+     * the code executing that operation.
+     */
+    ConditionalTabletMutator requireOperation(TabletOperation operation,
+        TabletOperationId operationId);
+
+    /**
+     * Require that no mutually exclusive operations are runnnig against this 
tablet.
+     */
+    ConditionalTabletMutator requireAbsentOperation();
+
+    /**
+     * Require an entire tablet is absent, so the tablet row has no columns. 
If the entire tablet is
+     * absent, then this implies the tablet operation is also absent so there 
is no need to specify
+     * that.
+     */
+    ConditionalTabletMutator requireAbsentTablet();
+  }
+
+  interface ConditionalTabletMutator extends 
TabletUpdates<ConditionalTabletMutator> {
+
+    /**
+     * Require that a tablet has no future or current location set.
+     */
+    ConditionalTabletMutator requireAbsentLocation();
+
+    /**
+     * Require that a tablet currently has the specified future or current 
location.
+     */
+    ConditionalTabletMutator requireLocation(Location location);
+
+    /**
+     * Require that a tablet currently has the specified file.
+     */
+    ConditionalTabletMutator requireFile(StoredTabletFile path);
+
+    /**
+     * Require that a tablet does not have the specfied bulk load marker.
+     */
+    ConditionalTabletMutator requireAbsentBulkFile(TabletFile bulkref);
+
+    /**
+     * Require that a tablet has the specified previous end row.
+     */
+    ConditionalTabletMutator requirePrevEndRow(Text per);
+
+    /**
+     * Submits or queues a conditional mutation for processing.
+     */
+    ConditionalTabletsMutator submit();
+  }
+
   /**
    * Insert ScanServer references to Tablet files
    *
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index f2c8c61679..e7a1c05963 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -73,6 +73,28 @@ public class MetadataSchema {
       return entry;
     }
 
+    /**
+     * Check is a metadata row is of the expected format and throws an 
exception if its not.
+     */
+    public static void validateRow(Text metadataRow) {
+      int semiPos = -1;
+      int ltPos = -1;
+
+      for (int i = 0; i < metadataRow.getLength(); i++) {
+        if (metadataRow.getBytes()[i] == ';' && semiPos < 0) {
+          // want the position of the first semicolon
+          semiPos = i;
+        }
+        if (metadataRow.getBytes()[i] == '<') {
+          ltPos = i;
+        }
+      }
+
+      if (semiPos < 0 && ltPos < 0) {
+        throw new IllegalArgumentException("Metadata row does not contain ; or 
<  " + metadataRow);
+      }
+    }
+
     /**
      * Decodes a metadata row into a pair of table ID and end row.
      */
@@ -226,6 +248,9 @@ public class MetadataSchema {
        */
       public static final String LOCK_QUAL = "lock";
       public static final ColumnFQ LOCK_COLUMN = new ColumnFQ(NAME, new 
Text(LOCK_QUAL));
+
+      public static final String OPID_QUAL = "opid";
+      public static final ColumnFQ OPID_COLUMN = new ColumnFQ(NAME, new 
Text(OPID_QUAL));
     }
 
     /**
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
index 3f01eed6c4..2bb8684e5b 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
@@ -27,6 +27,7 @@ import java.nio.charset.CharsetDecoder;
 import java.util.AbstractMap.SimpleImmutableEntry;
 import java.util.EnumSet;
 import java.util.Map.Entry;
+import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.function.Predicate;
 import java.util.stream.Stream;
@@ -145,19 +146,30 @@ public class RootTabletMetadata {
     }
   }
 
+  public SortedMap<Key,Value> toKeyValues() {
+    TreeMap<Key,Value> metamap = new TreeMap<>();
+    getKeyValues().forEach(e -> metamap.put(e.getKey(), e.getValue()));
+    return metamap;
+  }
+
   /**
    * Convert this class to a {@link TabletMetadata}
    */
   public TabletMetadata toTabletMetadata() {
-    String row = RootTable.EXTENT.toMetaRow().toString();
     // use a stream so we don't have to re-sort in a new TreeMap<Key,Value> 
structure
+    Stream<SimpleImmutableEntry<Key,Value>> entries = getKeyValues();
+    return TabletMetadata.convertRow(entries.iterator(),
+        EnumSet.allOf(TabletMetadata.ColumnType.class), false);
+  }
+
+  private Stream<SimpleImmutableEntry<Key,Value>> getKeyValues() {
+    String row = RootTable.EXTENT.toMetaRow().toString();
     Stream<SimpleImmutableEntry<Key,Value>> entries = 
data.columnValues.entrySet().stream()
         .flatMap(famToQualVal -> famToQualVal.getValue().entrySet().stream()
             .map(qualVal -> new SimpleImmutableEntry<>(
                 new Key(row, famToQualVal.getKey(), qualVal.getKey(), 1),
                 new Value(qualVal.getValue()))));
-    return TabletMetadata.convertRow(entries.iterator(),
-        EnumSet.allOf(TabletMetadata.ColumnType.class), false);
+    return entries;
   }
 
   /**
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index d1a83772cd..577baf7891 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -21,6 +21,7 @@ package org.apache.accumulo.core.metadata.schema;
 import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.COMPACT_QUAL;
 import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_QUAL;
 import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_QUAL;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.OPID_QUAL;
 import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.TIME_QUAL;
 import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.OLD_PREV_ROW_QUAL;
 import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_QUAL;
@@ -54,6 +55,7 @@ import org.apache.accumulo.core.metadata.SuspendingTServer;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.TabletLocationState;
+import org.apache.accumulo.core.metadata.TabletOperationId;
 import org.apache.accumulo.core.metadata.TabletState;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
@@ -108,6 +110,8 @@ public class TabletMetadata {
   private Double splitRatio = null;
   private Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
   private boolean chopped = false;
+  private TabletOperation operation;
+  private TabletOperationId operationId;
   private boolean onDemand = false;
 
   public enum LocationType {
@@ -132,6 +136,7 @@ public class TabletMetadata {
     SUSPEND,
     CHOPPED,
     ECOMP,
+    OPID,
     ON_DEMAND
   }
 
@@ -384,6 +389,16 @@ public class TabletMetadata {
     return extCompactions;
   }
 
+  public TabletOperation getOperation() {
+    ensureFetched(ColumnType.OPID);
+    return operation;
+  }
+
+  public TabletOperationId getOperationId() {
+    ensureFetched(ColumnType.OPID);
+    return operationId;
+  }
+
   @VisibleForTesting
   public static <E extends Entry<Key,Value>> TabletMetadata 
convertRow(Iterator<E> rowIter,
       EnumSet<ColumnType> fetchedColumns, boolean buildKeyValueMap) {
@@ -454,6 +469,11 @@ public class TabletMetadata {
             case COMPACT_QUAL:
               te.compact = OptionalLong.of(Long.parseLong(val));
               break;
+            case OPID_QUAL:
+              String[] tokens = val.split(":", 2);
+              te.operation = TabletOperation.valueOf(tokens[0]);
+              te.operationId = new TabletOperationId(tokens[1]);
+              break;
           }
           break;
         case DataFileColumnFamily.STR_NAME:
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletOperation.java
similarity index 55%
copy from 
server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
copy to 
core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletOperation.java
index ee0db927a1..fd58ddb61b 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletOperation.java
@@ -16,32 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.data;
-
-import java.util.List;
-
-import org.apache.accumulo.core.dataImpl.thrift.TCondition;
-import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
-import org.apache.accumulo.server.data.ServerMutation;
-
-public class ServerConditionalMutation extends ServerMutation {
-
-  private long cmid;
-  private List<TCondition> conditions;
-
-  public ServerConditionalMutation(TConditionalMutation input) {
-    super(input.mutation);
-
-    this.cmid = input.id;
-    this.conditions = input.conditions;
-  }
-
-  public long getID() {
-    return cmid;
-  }
-
-  public List<TCondition> getConditions() {
-    return conditions;
-  }
+package org.apache.accumulo.core.metadata.schema;
 
+/**
+ * Used to specify what kind of mutually exclusive operation is currently 
running against a tablet.
+ */
+public enum TabletOperation {
+  SPLITTING, MERGING, DELETING
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
 
b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
index 0350b88437..19bcca9fce 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
@@ -86,7 +86,8 @@ public class MetadataConstraints implements Constraint {
           ServerColumnFamily.TIME_COLUMN,
           ServerColumnFamily.LOCK_COLUMN,
           ServerColumnFamily.FLUSH_COLUMN,
-          ServerColumnFamily.COMPACT_COLUMN);
+          ServerColumnFamily.COMPACT_COLUMN,
+              ServerColumnFamily.OPID_COLUMN);
 
   private static final Set<Text> validColumnFams =
       Set.of(BulkFileColumnFamily.NAME,
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
 
b/server/base/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
similarity index 93%
copy from 
server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
copy to 
server/base/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
index ee0db927a1..849ee82841 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
@@ -16,13 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.data;
+package org.apache.accumulo.server.data;
 
 import java.util.List;
 
 import org.apache.accumulo.core.dataImpl.thrift.TCondition;
 import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
-import org.apache.accumulo.server.data.ServerMutation;
 
 public class ServerConditionalMutation extends ServerMutation {
 
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
new file mode 100644
index 0000000000..517e0b6a36
--- /dev/null
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java
@@ -0,0 +1,154 @@
+/*
+ * 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.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.OPID_COLUMN;
+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.Consumer;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.data.Condition;
+import org.apache.accumulo.core.data.ConditionalMutation;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.TabletOperationId;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
+import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletOperation;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.metadata.iterators.LocationExistsIterator;
+import org.apache.accumulo.server.metadata.iterators.PresentIterator;
+import org.apache.accumulo.server.metadata.iterators.TabletExistsIterator;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+
+public class ConditionalTabletMutatorImpl extends 
TabletMutatorBase<Ample.ConditionalTabletMutator>
+    implements Ample.ConditionalTabletMutator, Ample.OperationRequirements {
+
+  private static final int INITIAL_ITERATOR_PRIO = 1000000;
+
+  private final ConditionalMutation mutation;
+  private final Consumer<ConditionalMutation> mutationConsumer;
+  private final Ample.ConditionalTabletsMutator parent;
+
+  private boolean sawOperationRequirement = false;
+
+  protected ConditionalTabletMutatorImpl(Ample.ConditionalTabletsMutator 
parent,
+      ServerContext context, KeyExtent extent, Consumer<ConditionalMutation> 
mutationConsumer) {
+    super(context, new ConditionalMutation(extent.toMetaRow()));
+    this.mutation = (ConditionalMutation) super.mutation;
+    this.mutationConsumer = mutationConsumer;
+    this.parent = parent;
+  }
+
+  @Override
+  public Ample.ConditionalTabletMutator requireAbsentLocation() {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    IteratorSetting is = new IteratorSetting(INITIAL_ITERATOR_PRIO, 
LocationExistsIterator.class);
+    Condition c = new Condition("", "").setIterators(is);
+    mutation.addCondition(c);
+    return this;
+  }
+
+  @Override
+  public Ample.ConditionalTabletMutator requireLocation(Location location) {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    Preconditions.checkArgument(location.getType() == 
TabletMetadata.LocationType.FUTURE
+        || location.getType() == TabletMetadata.LocationType.CURRENT);
+    Condition c = new Condition(getLocationFamily(location.getType()), 
location.getSession())
+        .setValue(location.getHostPort());
+    mutation.addCondition(c);
+    return this;
+  }
+
+  @Override
+  public Ample.ConditionalTabletMutator requireFile(StoredTabletFile path) {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    IteratorSetting is = new IteratorSetting(INITIAL_ITERATOR_PRIO, 
PresentIterator.class);
+    Condition c = new Condition(DataFileColumnFamily.NAME, 
path.getMetaUpdateDeleteText())
+        .setValue(PresentIterator.VALUE).setIterators(is);
+    mutation.addCondition(c);
+    return this;
+  }
+
+  @Override
+  public Ample.ConditionalTabletMutator requireAbsentBulkFile(TabletFile 
bulkref) {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    Condition c = new Condition(BulkFileColumnFamily.NAME, 
bulkref.getMetaInsertText());
+    mutation.addCondition(c);
+    return this;
+  }
+
+  @Override
+  public Ample.ConditionalTabletMutator requirePrevEndRow(Text per) {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    Condition c =
+        new Condition(PREV_ROW_COLUMN.getColumnFamily(), 
PREV_ROW_COLUMN.getColumnQualifier())
+            .setValue(encodePrevEndRow(per).get());
+    mutation.addCondition(c);
+    return this;
+  }
+
+  @Override
+  public Ample.ConditionalTabletMutator requireAbsentTablet() {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    IteratorSetting is = new IteratorSetting(INITIAL_ITERATOR_PRIO, 
TabletExistsIterator.class);
+    Condition c = new Condition("", "").setIterators(is);
+    mutation.addCondition(c);
+    sawOperationRequirement = true;
+    return this;
+  }
+
+  @Override
+  public Ample.ConditionalTabletMutator requireAbsentOperation() {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    Condition c = new Condition(OPID_COLUMN.getColumnFamily(), 
OPID_COLUMN.getColumnQualifier());
+    mutation.addCondition(c);
+    sawOperationRequirement = true;
+    return this;
+  }
+
+  @Override
+  public Ample.ConditionalTabletMutator requireOperation(TabletOperation 
operation,
+      TabletOperationId opid) {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    Condition c = new Condition(OPID_COLUMN.getColumnFamily(), 
OPID_COLUMN.getColumnQualifier())
+        .setValue(operation.name() + ":" + opid.canonical());
+    mutation.addCondition(c);
+    sawOperationRequirement = true;
+    return this;
+  }
+
+  @Override
+  public Ample.ConditionalTabletsMutator submit() {
+    Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
+    Preconditions.checkState(sawOperationRequirement, "No operation 
requirements were seen");
+    getMutation();
+    mutationConsumer.accept(mutation);
+    return parent;
+  }
+}
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
new file mode 100644
index 0000000000..7aaf149211
--- /dev/null
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletsMutatorImpl.java
@@ -0,0 +1,117 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.server.ServerContext;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+
+public class ConditionalTabletsMutatorImpl implements 
Ample.ConditionalTabletsMutator {
+
+  private final ServerContext context;
+  private TableId currentTableId = null;
+
+  private List<ConditionalMutation> mutations = new ArrayList<>();
+
+  private Map<Text,KeyExtent> extents = new HashMap<>();
+
+  private boolean active = true;
+
+  public ConditionalTabletsMutatorImpl(ServerContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public Ample.OperationRequirements mutateTablet(KeyExtent extent) {
+    Preconditions.checkState(active);
+    if (currentTableId == null) {
+      currentTableId = extent.tableId();
+    } else if (!currentTableId.equals(extent.tableId())) {
+      throw new IllegalArgumentException(
+          "Can not mix tables ids " + currentTableId + " " + extent.tableId());
+    }
+
+    Preconditions.checkState(extents.putIfAbsent(extent.toMetaRow(), extent) 
== null,
+        "Duplicate extents not handled");
+    return new ConditionalTabletMutatorImpl(this, context, extent, 
mutations::add);
+  }
+
+  private ConditionalWriter createConditionalWriter(Ample.DataLevel dataLevel)
+      throws TableNotFoundException {
+    if (dataLevel == Ample.DataLevel.ROOT) {
+      return new RootConditionalWriter(context);
+    } else {
+      return context.createConditionalWriter(dataLevel.metaTable());
+    }
+  }
+
+  @Override
+  public Map<KeyExtent,ConditionalWriter.Result> process() {
+    Preconditions.checkState(active);
+    if (currentTableId != null) {
+      var dataLevel = Ample.DataLevel.of(currentTableId);
+      try (ConditionalWriter conditionalWriter = 
createConditionalWriter(dataLevel)) {
+        var results = conditionalWriter.write(mutations.iterator());
+
+        var resultsMap = new HashMap<KeyExtent,ConditionalWriter.Result>();
+
+        while (results.hasNext()) {
+          var result = results.next();
+          var row = new Text(result.getMutation().getRow());
+          resultsMap.put(extents.get(row), result);
+        }
+
+        if (!resultsMap.keySet().equals(Set.copyOf(extents.values()))) {
+          throw new AssertionError("Not all extents were seen, this is 
unexpected");
+        }
+
+        return resultsMap;
+      } catch (TableNotFoundException e) {
+        throw new RuntimeException(e);
+      } finally {
+        // render inoperable because reuse is not tested
+        extents.clear();
+        mutations.clear();
+        active = false;
+      }
+    } else {
+      // render inoperable because reuse is not tested
+      extents.clear();
+      mutations.clear();
+      active = false;
+      return Map.of();
+    }
+  }
+
+  @Override
+  public void close() {}
+}
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
new file mode 100644
index 0000000000..d83f56273d
--- /dev/null
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootConditionalWriter.java
@@ -0,0 +1,154 @@
+/*
+ * 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 java.nio.charset.StandardCharsets.UTF_8;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+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.clientImpl.CompressedIterators;
+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.SortedMapIterator;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.constraints.MetadataConstraints;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
+import org.apache.accumulo.server.tablets.ConditionCheckerContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A specialized conditional writer that writes metadata for the Accumulo root 
tablet which is
+ * stored in zookeeper.
+ */
+public class RootConditionalWriter implements ConditionalWriter {
+
+  private static final Logger log = 
LoggerFactory.getLogger(RootConditionalWriter.class);
+
+  private final ServerContext context;
+
+  RootConditionalWriter(ServerContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public Iterator<Result> write(Iterator<ConditionalMutation> mutations) {
+    List<Result> results = new ArrayList<>();
+    while (mutations.hasNext()) {
+      results.add(write(mutations.next()));
+    }
+
+    return results.iterator();
+  }
+
+  @Override
+  public Result write(ConditionalMutation mutation) {
+
+    MetadataConstraints metaConstraint = new MetadataConstraints();
+    List<Short> violations =
+        metaConstraint.check(new RootTabletMutatorImpl.RootEnv(context), 
mutation);
+
+    if (violations != null && !violations.isEmpty()) {
+      return new Result(Status.VIOLATED, mutation, "ZK");
+    }
+
+    CompressedIterators compressedIters = new CompressedIterators();
+    TConditionalMutation tcm =
+        ConditionalWriterImpl.convertConditionalMutation(compressedIters, 
mutation, 1);
+    ConditionCheckerContext checkerContext = new 
ConditionCheckerContext(context, compressedIters,
+        context.getTableConfiguration(RootTable.ID));
+
+    ServerConditionalMutation scm = new ServerConditionalMutation(tcm);
+
+    String zpath = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET;
+
+    context.getZooCache().clear(zpath);
+
+    List<ServerConditionalMutation> okMutations = new ArrayList<>();
+    List<TCMResult> results = new ArrayList<>();
+
+    try {
+      context.getZooReaderWriter().mutateExisting(zpath, currVal -> {
+        String currJson = new String(currVal, UTF_8);
+        var rtm = new RootTabletMetadata(currJson);
+
+        var iter = new SortedMapIterator(rtm.toKeyValues());
+
+        // This could be called multiple times so clear before calling
+        okMutations.clear();
+        results.clear();
+        var checker = checkerContext.newChecker(List.of(scm), okMutations, 
results);
+        try {
+          checker.check(iter);
+          if (getResult(okMutations, results, mutation).getStatus() == 
Status.ACCEPTED) {
+            rtm.update(mutation);
+            String newJson = rtm.toJson();
+            log.debug("mutation: from:[{}] to: [{}]", currJson, newJson);
+            return newJson.getBytes(UTF_8);
+          } else {
+            // conditions failed so make no updates
+            return null;
+          }
+
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        } catch (AccumuloException e) {
+          throw new RuntimeException(e);
+        } catch (AccumuloSecurityException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    // TODO this is racy...
+    context.getZooCache().clear(zpath);
+
+    return getResult(okMutations, results, mutation);
+  }
+
+  private static Result getResult(List<ServerConditionalMutation> okMutations,
+      List<TCMResult> results, ConditionalMutation mutation) {
+    if (okMutations.size() == 1 && results.isEmpty()) {
+      return new Result(Status.ACCEPTED, mutation, "ZK");
+    } else if (okMutations.isEmpty() && results.size() == 1) {
+      var tresult = results.get(0);
+      return new Result(ConditionalWriterImpl.fromThrift(tresult.getStatus()), 
mutation, "ZK");
+    } else {
+      throw new AssertionError("Unexpected case");
+    }
+  }
+
+  @Override
+  public void close() {
+
+  }
+}
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java
index 4ec2c5d14c..c9115a3e70 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java
@@ -36,12 +36,13 @@ import 
org.apache.accumulo.server.constraints.SystemEnvironment;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class RootTabletMutatorImpl extends TabletMutatorBase implements 
Ample.TabletMutator {
+public class RootTabletMutatorImpl extends 
TabletMutatorBase<Ample.TabletMutator>
+    implements Ample.TabletMutator {
   private final ServerContext context;
 
   private static final Logger log = 
LoggerFactory.getLogger(RootTabletMutatorImpl.class);
 
-  private static class RootEnv implements SystemEnvironment, 
Constraint.Environment {
+  static class RootEnv implements SystemEnvironment, Constraint.Environment {
 
     private final ServerContext context;
 
@@ -94,6 +95,7 @@ public class RootTabletMutatorImpl extends TabletMutatorBase 
implements Ample.Ta
       context.getZooCache().clear(zpath);
 
       // TODO examine implementation of getZooReaderWriter().mutate()
+      // TODO for efficiency this should maybe call mutateExisting
       context.getZooReaderWriter().mutateOrCreate(zpath, new byte[0], currVal 
-> {
         String currJson = new String(currVal, UTF_8);
         var rtm = new RootTabletMetadata(currJson);
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
index d23917f0a3..4136a79e0a 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
@@ -94,6 +94,11 @@ public class ServerAmpleImpl extends AmpleImpl implements 
Ample {
     return new TabletsMutatorImpl(context);
   }
 
+  @Override
+  public ConditionalTabletsMutator conditionallyMutateTablets() {
+    return new ConditionalTabletsMutatorImpl(context);
+  }
+
   private void mutateRootGcCandidates(Consumer<RootGcCandidates> mutator) {
     String zpath = context.getZooKeeperRoot() + ZROOT_TABLET_GC_CANDIDATES;
     try {
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
index 62d13017b4..b1d383bec2 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
@@ -27,8 +27,8 @@ import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.SuspendingTServer;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.TabletOperationId;
 import org.apache.accumulo.core.metadata.schema.Ample;
-import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionMetadata;
@@ -48,89 +48,102 @@ import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Ta
 import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.metadata.schema.TabletOperation;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
 
 import com.google.common.base.Preconditions;
 
-public abstract class TabletMutatorBase implements Ample.TabletMutator {
+public abstract class TabletMutatorBase<T extends Ample.TabletUpdates<T>>
+    implements Ample.TabletUpdates<T> {
 
   private final ServerContext context;
-  private final Mutation mutation;
+
+  protected final Mutation mutation;
   protected AutoCloseable closeAfterMutate;
-  private boolean updatesEnabled = true;
+  protected boolean updatesEnabled = true;
+
+  @SuppressWarnings("unchecked")
+  private T getThis() {
+    return (T) this;
+  }
 
   protected TabletMutatorBase(ServerContext context, KeyExtent extent) {
     this.context = context;
     mutation = new Mutation(extent.toMetaRow());
   }
 
+  protected TabletMutatorBase(ServerContext context, Mutation mutation) {
+    this.context = context;
+    this.mutation = mutation;
+  }
+
   @Override
-  public Ample.TabletMutator putPrevEndRow(Text per) {
+  public T putPrevEndRow(Text per) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     TabletColumnFamily.PREV_ROW_COLUMN.put(mutation, 
TabletColumnFamily.encodePrevEndRow(per));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putDirName(String dirName) {
+  public T putDirName(String dirName) {
     ServerColumnFamily.validateDirCol(dirName);
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     ServerColumnFamily.DIRECTORY_COLUMN.put(mutation, new Value(dirName));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putFile(TabletFile path, DataFileValue dfv) {
+  public T putFile(TabletFile path, DataFileValue dfv) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.put(DataFileColumnFamily.NAME, path.getMetaInsertText(), new 
Value(dfv.encode()));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator deleteFile(StoredTabletFile path) {
+  public T deleteFile(StoredTabletFile path) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.putDelete(DataFileColumnFamily.NAME, 
path.getMetaUpdateDeleteText());
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putScan(TabletFile path) {
+  public T putScan(TabletFile path) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.put(ScanFileColumnFamily.NAME, path.getMetaInsertText(), new 
Value());
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator deleteScan(StoredTabletFile path) {
+  public T deleteScan(StoredTabletFile path) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.putDelete(ScanFileColumnFamily.NAME, 
path.getMetaUpdateDeleteText());
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putCompactionId(long compactionId) {
+  public T putCompactionId(long compactionId) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     ServerColumnFamily.COMPACT_COLUMN.put(mutation, new 
Value(Long.toString(compactionId)));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putFlushId(long flushId) {
+  public T putFlushId(long flushId) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     ServerColumnFamily.FLUSH_COLUMN.put(mutation, new 
Value(Long.toString(flushId)));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putTime(MetadataTime time) {
+  public T putTime(MetadataTime time) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     ServerColumnFamily.TIME_COLUMN.put(mutation, new Value(time.encode()));
-    return this;
+    return getThis();
   }
 
-  private String getLocationFamily(LocationType type) {
+  protected String getLocationFamily(LocationType type) {
     switch (type) {
       case CURRENT:
         return CurrentLocationColumnFamily.STR_NAME;
@@ -144,99 +157,110 @@ public abstract class TabletMutatorBase implements 
Ample.TabletMutator {
   }
 
   @Override
-  public Ample.TabletMutator putLocation(Location location) {
+  public T putLocation(Location location) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.put(getLocationFamily(location.getType()), location.getSession(),
         location.getHostPort());
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator deleteLocation(Location location) {
+  public T deleteLocation(Location location) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.putDelete(getLocationFamily(location.getType()), 
location.getSession());
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putZooLock(ServiceLock zooLock) {
+  public T putZooLock(ServiceLock zooLock) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     ServerColumnFamily.LOCK_COLUMN.put(mutation,
         new Value(zooLock.getLockID().serialize(context.getZooKeeperRoot() + 
"/")));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putWal(LogEntry logEntry) {
+  public T putWal(LogEntry logEntry) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), 
logEntry.getValue());
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator deleteWal(LogEntry logEntry) {
+  public T deleteWal(LogEntry logEntry) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.putDelete(logEntry.getColumnFamily(), 
logEntry.getColumnQualifier());
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator deleteWal(String wal) {
+  public T deleteWal(String wal) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.putDelete(LogColumnFamily.STR_NAME, wal);
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putBulkFile(TabletFile bulkref, long tid) {
+  public T putBulkFile(TabletFile bulkref, long tid) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.put(BulkFileColumnFamily.NAME, bulkref.getMetaInsertText(),
         new Value(FateTxId.formatTid(tid)));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator deleteBulkFile(TabletFile bulkref) {
+  public T deleteBulkFile(TabletFile bulkref) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.putDelete(BulkFileColumnFamily.NAME, bulkref.getMetaInsertText());
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putChopped() {
+  public T putChopped() {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     ChoppedColumnFamily.CHOPPED_COLUMN.put(mutation, new Value("chopped"));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator putSuspension(TServerInstance tServer, long 
suspensionTime) {
+  public T putSuspension(TServerInstance tServer, long suspensionTime) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.put(SuspendLocationColumn.SUSPEND_COLUMN.getColumnFamily(),
         SuspendLocationColumn.SUSPEND_COLUMN.getColumnQualifier(),
         SuspendingTServer.toValue(tServer, suspensionTime));
-    return this;
+    return getThis();
   }
 
   @Override
-  public Ample.TabletMutator deleteSuspension() {
+  public T deleteSuspension() {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after 
calling mutate.");
     mutation.putDelete(SuspendLocationColumn.SUSPEND_COLUMN.getColumnFamily(),
         SuspendLocationColumn.SUSPEND_COLUMN.getColumnQualifier());
-    return this;
+    return getThis();
   }
 
   @Override
-  public TabletMutator putExternalCompaction(ExternalCompactionId ecid,
-      ExternalCompactionMetadata ecMeta) {
+  public T putExternalCompaction(ExternalCompactionId ecid, 
ExternalCompactionMetadata ecMeta) {
     mutation.put(ExternalCompactionColumnFamily.STR_NAME, ecid.canonical(), 
ecMeta.toJson());
-    return this;
+    return getThis();
   }
 
   @Override
-  public TabletMutator deleteExternalCompaction(ExternalCompactionId ecid) {
+  public T deleteExternalCompaction(ExternalCompactionId ecid) {
     mutation.putDelete(ExternalCompactionColumnFamily.STR_NAME, 
ecid.canonical());
-    return this;
+    return getThis();
+  }
+
+  @Override
+  public T putOperation(TabletOperation top, TabletOperationId opId) {
+    ServerColumnFamily.OPID_COLUMN.put(mutation, new Value(top.name() + ":" + 
opId.canonical()));
+    return getThis();
+  }
+
+  @Override
+  public T deleteOperation() {
+    ServerColumnFamily.OPID_COLUMN.putDelete(mutation);
+    return getThis();
   }
 
   protected Mutation getMutation() {
@@ -244,19 +268,19 @@ public abstract class TabletMutatorBase implements 
Ample.TabletMutator {
     return mutation;
   }
 
-  public void setCloseAfterMutate(AutoCloseable closeable) {
-    this.closeAfterMutate = closeable;
-  }
-
   @Override
-  public TabletMutator putOnDemand() {
+  public T putOnDemand() {
     mutation.put(OnDemandAssignmentStateColumnFamily.STR_NAME, "", "");
-    return this;
+    return getThis();
   }
 
   @Override
-  public TabletMutator deleteOnDemand() {
+  public T deleteOnDemand() {
     mutation.putDelete(OnDemandAssignmentStateColumnFamily.STR_NAME, "");
-    return this;
+    return getThis();
+  }
+
+  public void setCloseAfterMutate(AutoCloseable closeable) {
+    this.closeAfterMutate = closeable;
   }
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
index e4889871da..10adbc3fbd 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
@@ -23,7 +23,8 @@ import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.server.ServerContext;
 
-class TabletMutatorImpl extends TabletMutatorBase implements 
Ample.TabletMutator {
+class TabletMutatorImpl extends TabletMutatorBase<Ample.TabletMutator>
+    implements Ample.TabletMutator {
 
   private BatchWriter writer;
 
@@ -44,5 +45,4 @@ class TabletMutatorImpl extends TabletMutatorBase implements 
Ample.TabletMutator
       throw new RuntimeException(e);
     }
   }
-
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/LocationExistsIterator.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/LocationExistsIterator.java
new file mode 100644
index 0000000000..bbe5d14a2e
--- /dev/null
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/LocationExistsIterator.java
@@ -0,0 +1,68 @@
+/*
+ * 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.iterators;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
+import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A specialized iterator used for conditional mutations to check if a 
location is present in a
+ * tablet.
+ */
+public class LocationExistsIterator extends WrappingIterator {
+  private static final Collection<ByteSequence> LOC_FAMS =
+      List.of(new ArrayByteSequence(FutureLocationColumnFamily.STR_NAME),
+          new ArrayByteSequence(CurrentLocationColumnFamily.STR_NAME));
+
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, 
boolean inclusive)
+      throws IOException {
+
+    Text tabletRow = getTabletRow(range);
+    Key startKey = new Key(tabletRow, FutureLocationColumnFamily.NAME);
+    Key endKey =
+        new Key(tabletRow, 
CurrentLocationColumnFamily.NAME).followingKey(PartialKey.ROW_COLFAM);
+
+    Range r = new Range(startKey, true, endKey, false);
+
+    super.seek(r, LOC_FAMS, true);
+  }
+
+  static Text getTabletRow(Range range) {
+    var row = range.getStartKey().getRow();
+    // expecting this range to cover a single metadata row, so validate the 
range meets expectations
+    TabletsSection.validateRow(row);
+    Preconditions.checkArgument(row.equals(range.getEndKey().getRow()));
+    return range.getStartKey().getRow();
+  }
+}
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/PresentIterator.java
similarity index 55%
copy from 
server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
copy to 
server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/PresentIterator.java
index ee0db927a1..bfb22884c7 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/PresentIterator.java
@@ -16,32 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.data;
 
-import java.util.List;
+package org.apache.accumulo.server.metadata.iterators;
 
-import org.apache.accumulo.core.dataImpl.thrift.TCondition;
-import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
-import org.apache.accumulo.server.data.ServerMutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.WrappingIterator;
 
-public class ServerConditionalMutation extends ServerMutation {
+public class PresentIterator extends WrappingIterator {
 
-  private long cmid;
-  private List<TCondition> conditions;
+  public static final String VALUE = "present";
 
-  public ServerConditionalMutation(TConditionalMutation input) {
-    super(input.mutation);
-
-    this.cmid = input.id;
-    this.conditions = input.conditions;
-  }
-
-  public long getID() {
-    return cmid;
+  @Override
+  public Value getTopValue() {
+    super.getTopValue();
+    return new Value(VALUE);
   }
-
-  public List<TCondition> getConditions() {
-    return conditions;
-  }
-
 }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/TabletExistsIterator.java
similarity index 51%
copy from 
server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
copy to 
server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/TabletExistsIterator.java
index e4889871da..4b9672f8dc 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metadata/iterators/TabletExistsIterator.java
@@ -16,33 +16,31 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.metadata;
+package org.apache.accumulo.server.metadata.iterators;
 
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.schema.Ample;
-import org.apache.accumulo.server.ServerContext;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Set;
 
-class TabletMutatorImpl extends TabletMutatorBase implements 
Ample.TabletMutator {
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.apache.hadoop.io.Text;
 
-  private BatchWriter writer;
-
-  TabletMutatorImpl(ServerContext context, KeyExtent extent, BatchWriter 
batchWriter) {
-    super(context, extent);
-    this.writer = batchWriter;
-  }
+/**
+ * A specialized iterator used for conditional mutations to check if a tablet 
has any entries in the
+ * metadata table.
+ */
+public class TabletExistsIterator extends WrappingIterator {
 
   @Override
-  public void mutate() {
-    try {
-      writer.addMutation(getMutation());
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, 
boolean inclusive)
+      throws IOException {
 
-      if (closeAfterMutate != null) {
-        closeAfterMutate.close();
-      }
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
+    Text tabletRow = LocationExistsIterator.getTabletRow(range);
+
+    Range r = new Range(tabletRow, true, tabletRow, false);
 
+    super.seek(r, Set.of(), false);
+  }
 }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
 
b/server/base/src/main/java/org/apache/accumulo/server/tablets/ConditionCheckerContext.java
similarity index 96%
rename from 
server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
rename to 
server/base/src/main/java/org/apache/accumulo/server/tablets/ConditionCheckerContext.java
index dd34a73552..a16df5143c 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/tablets/ConditionCheckerContext.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver;
+package org.apache.accumulo.server.tablets;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
@@ -46,8 +46,8 @@ import 
org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.conf.TableConfiguration.ParsedIteratorConfig;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
 import org.apache.accumulo.server.iterators.TabletIteratorEnvironment;
-import org.apache.accumulo.tserver.data.ServerConditionalMutation;
 import org.apache.hadoop.io.Text;
 
 public class ConditionCheckerContext {
@@ -70,7 +70,7 @@ public class ConditionCheckerContext {
 
   private Map<ByteSequence,MergedIterConfig> mergedIterCache = new HashMap<>();
 
-  ConditionCheckerContext(ServerContext context, CompressedIterators 
compressedIters,
+  public ConditionCheckerContext(ServerContext context, CompressedIterators 
compressedIters,
       TableConfiguration tableConf) {
     this.compressedIters = compressedIters;
 
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 c3660796ba..ca56c2ffe4 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
@@ -70,20 +70,25 @@ 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 TabletMutatorBase(null, extent) {
-
-        @Override
-        public void mutate() {
-          Mutation m = getMutation();
-
-          var rtm = new RootTabletMetadata(json);
-          rtm.update(m);
-          json = rtm.toJson();
-        }
-      };
+      return new TestTabletMutator(null, RootTable.EXTENT);
     }
 
   }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/TabletOperations.java
similarity index 55%
rename from 
server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
rename to 
server/manager/src/main/java/org/apache/accumulo/manager/TabletOperations.java
index ee0db927a1..1feb9e459c 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/data/ServerConditionalMutation.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/TabletOperations.java
@@ -16,32 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.data;
+package org.apache.accumulo.manager;
 
-import java.util.List;
-
-import org.apache.accumulo.core.dataImpl.thrift.TCondition;
-import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
-import org.apache.accumulo.server.data.ServerMutation;
-
-public class ServerConditionalMutation extends ServerMutation {
-
-  private long cmid;
-  private List<TCondition> conditions;
-
-  public ServerConditionalMutation(TConditionalMutation input) {
-    super(input.mutation);
-
-    this.cmid = input.id;
-    this.conditions = input.conditions;
-  }
-
-  public long getID() {
-    return cmid;
-  }
-
-  public List<TCondition> getConditions() {
-    return conditions;
-  }
+import org.apache.accumulo.core.dataImpl.KeyExtent;
 
+public interface TabletOperations {
+  AutoCloseable unassign(KeyExtent tablet);
 }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionalMutationSet.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionalMutationSet.java
index faafaf1274..0704073179 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionalMutationSet.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionalMutationSet.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.tserver.data.ServerConditionalMutation;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
 import org.apache.hadoop.io.WritableComparator;
 
 public class ConditionalMutationSet {
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/RowLocks.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/RowLocks.java
index 805a1f8196..775bda26cc 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/RowLocks.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/RowLocks.java
@@ -29,8 +29,8 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
 import org.apache.accumulo.tserver.ConditionalMutationSet.DeferFilter;
-import org.apache.accumulo.tserver.data.ServerConditionalMutation;
 
 import com.google.common.base.Preconditions;
 
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
index 39798f2be6..e238bdcf18 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
@@ -109,15 +109,16 @@ import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.compaction.CompactionInfo;
 import org.apache.accumulo.server.compaction.FileCompactor;
 import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
 import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.server.fs.TooManyFilesException;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.rpc.TServerUtils;
 import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.accumulo.server.tablets.ConditionCheckerContext;
+import 
org.apache.accumulo.server.tablets.ConditionCheckerContext.ConditionChecker;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
-import org.apache.accumulo.tserver.ConditionCheckerContext.ConditionChecker;
 import org.apache.accumulo.tserver.RowLocks.RowLock;
-import org.apache.accumulo.tserver.data.ServerConditionalMutation;
 import org.apache.accumulo.tserver.session.ConditionalSession;
 import org.apache.accumulo.tserver.session.SummarySession;
 import org.apache.accumulo.tserver.session.UpdateSession;
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 4fafe5a6c4..281726eb02 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -100,12 +100,12 @@ import 
org.apache.accumulo.server.fs.VolumeUtil.TabletFiles;
 import org.apache.accumulo.server.problems.ProblemReport;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
+import 
org.apache.accumulo.server.tablets.ConditionCheckerContext.ConditionChecker;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.tablets.UniqueNameAllocator;
 import org.apache.accumulo.server.util.FileUtil;
 import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.tserver.ConditionCheckerContext.ConditionChecker;
 import org.apache.accumulo.tserver.InMemoryMap;
 import org.apache.accumulo.tserver.MinorCompactionReason;
 import org.apache.accumulo.tserver.TabletServer;
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java
 
b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java
new file mode 100644
index 0000000000..2c897200c4
--- /dev/null
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java
@@ -0,0 +1,428 @@
+/*
+ * 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.test.functional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.ConditionalWriter.Status;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.TabletOperationId;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.metadata.schema.TabletOperation;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.server.metadata.ConditionalTabletsMutatorImpl;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Test;
+
+public class AmpleConditionalWriterIT extends AccumuloClusterHarness {
+
+  @Test
+  public void testLocations() throws Exception {
+    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
+
+      String tableName = getUniqueNames(1)[0];
+
+      SortedSet<Text> splits = new TreeSet<>(List.of(new Text("c"), new 
Text("f"), new Text("j")));
+      c.tableOperations().create(tableName,
+          new NewTableConfiguration().withSplits(splits).createOffline());
+
+      c.securityOperations().grantTablePermission("root", MetadataTable.NAME,
+          TablePermission.WRITE);
+
+      var tid = TableId.of(c.tableOperations().tableIdMap().get(tableName));
+
+      var e1 = new KeyExtent(tid, new Text("c"), null);
+
+      var ts1 = new TServerInstance("localhost:9997", 5000L);
+      var ts2 = new TServerInstance("localhost:9997", 6000L);
+
+      var context = cluster.getServerContext();
+
+      assertNull(context.getAmple().readTablet(e1).getLocation());
+
+      var ctmi = new ConditionalTabletsMutatorImpl(context);
+      ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.future(ts1)).submit();
+      var results = ctmi.process();
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+
+      assertEquals(Location.future(ts1), 
context.getAmple().readTablet(e1).getLocation());
+
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.future(ts2)).submit();
+      results = ctmi.process();
+      assertEquals(Status.REJECTED, results.get(e1).getStatus());
+
+      assertEquals(Location.future(ts1), 
context.getAmple().readTablet(e1).getLocation());
+
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireLocation(Location.future(ts1))
+          
.putLocation(Location.current(ts1)).deleteLocation(Location.future(ts1)).submit();
+      results = ctmi.process();
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+
+      assertEquals(Location.current(ts1), 
context.getAmple().readTablet(e1).getLocation());
+
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireLocation(Location.future(ts1))
+          
.putLocation(Location.current(ts1)).deleteLocation(Location.future(ts1)).submit();
+      results = ctmi.process();
+      assertEquals(Status.REJECTED, results.get(e1).getStatus());
+
+      assertEquals(Location.current(ts1), 
context.getAmple().readTablet(e1).getLocation());
+
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireLocation(Location.future(ts2))
+          
.putLocation(Location.current(ts2)).deleteLocation(Location.future(ts2)).submit();
+      results = ctmi.process();
+      assertEquals(Status.REJECTED, results.get(e1).getStatus());
+
+      assertEquals(Location.current(ts1), 
context.getAmple().readTablet(e1).getLocation());
+
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireLocation(Location.current(ts1))
+          .deleteLocation(Location.current(ts1)).submit();
+      results = ctmi.process();
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+
+      assertNull(context.getAmple().readTablet(e1).getLocation());
+    }
+  }
+
+  @Test
+  public void testFiles() throws Exception {
+
+    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      SortedSet<Text> splits = new TreeSet<>(List.of(new Text("c"), new 
Text("f"), new Text("j")));
+      c.tableOperations().create(tableName,
+          new NewTableConfiguration().withSplits(splits).createOffline());
+
+      c.securityOperations().grantTablePermission("root", MetadataTable.NAME,
+          TablePermission.WRITE);
+
+      var tid = TableId.of(c.tableOperations().tableIdMap().get(tableName));
+
+      var e1 = new KeyExtent(tid, new Text("c"), null);
+
+      var ts1 = new TServerInstance("localhost:9997", 5000L);
+      var ts2 = new TServerInstance("localhost:9997", 6000L);
+
+      var context = cluster.getServerContext();
+
+      var stf1 = new StoredTabletFile(
+          
"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf");
+      var stf2 = new StoredTabletFile(
+          
"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000071.rf");
+      var stf3 = new StoredTabletFile(
+          
"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000072.rf");
+      var stf4 = new StoredTabletFile(
+          
"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/C0000073.rf");
+
+      System.out.println(context.getAmple().readTablet(e1).getLocation());
+
+      // simulate a compaction where the tablet location is not set
+      var ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireFile(stf1).requireFile(stf2)
+          .requireFile(stf3).putFile(stf4, new DataFileValue(0, 0)).submit();
+      var results = ctmi.process();
+      assertEquals(Status.REJECTED, results.get(e1).getStatus());
+
+      assertEquals(Set.of(), context.getAmple().readTablet(e1).getFiles());
+
+      // simulate minor compacts where the tablet location is not set
+      for (StoredTabletFile file : List.of(stf1, stf2, stf3)) {
+        ctmi = new ConditionalTabletsMutatorImpl(context);
+        
ctmi.mutateTablet(e1).requireAbsentOperation().requireLocation(Location.current(ts1))
+            .putFile(file, new DataFileValue(0, 0)).submit();
+        results = ctmi.process();
+        assertEquals(Status.REJECTED, results.get(e1).getStatus());
+      }
+
+      assertEquals(Set.of(), context.getAmple().readTablet(e1).getFiles());
+
+      // set the location
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.current(ts1)).submit();
+      results = ctmi.process();
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+
+      // simulate minor compacts where the tablet location is wrong
+      for (StoredTabletFile file : List.of(stf1, stf2, stf3)) {
+        ctmi = new ConditionalTabletsMutatorImpl(context);
+        
ctmi.mutateTablet(e1).requireAbsentOperation().requireLocation(Location.current(ts2))
+            .putFile(file, new DataFileValue(0, 0)).submit();
+        results = ctmi.process();
+        assertEquals(Status.REJECTED, results.get(e1).getStatus());
+      }
+
+      assertEquals(Set.of(), context.getAmple().readTablet(e1).getFiles());
+
+      // simulate minor compacts where the tablet location is set
+      for (StoredTabletFile file : List.of(stf1, stf2, stf3)) {
+        ctmi = new ConditionalTabletsMutatorImpl(context);
+        
ctmi.mutateTablet(e1).requireAbsentOperation().requireLocation(Location.current(ts1))
+            .putFile(file, new DataFileValue(0, 0)).submit();
+        results = ctmi.process();
+        assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+      }
+
+      assertEquals(Set.of(stf1, stf2, stf3), 
context.getAmple().readTablet(e1).getFiles());
+
+      // simulate a compaction
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireFile(stf1).requireFile(stf2)
+          .requireFile(stf3).putFile(stf4, new DataFileValue(0, 
0)).deleteFile(stf1)
+          .deleteFile(stf2).deleteFile(stf3).submit();
+      results = ctmi.process();
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+
+      assertEquals(Set.of(stf4), context.getAmple().readTablet(e1).getFiles());
+
+      // without this the metadata constraint will not allow the bulk file to 
be added to metadata
+      TransactionWatcher.ZooArbitrator.start(context, 
Constants.BULK_ARBITRATOR_TYPE, 9L);
+
+      // simulate a bulk import
+      var stf5 =
+          new 
StoredTabletFile("hdfs://localhost:8020/accumulo/tables/2a/b-0000009/I0000074.rf");
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentBulkFile(stf5)
+          .putFile(stf5, new DataFileValue(0, 0)).putBulkFile(stf5, 9L)
+          .putFile(stf5, new DataFileValue(0, 0)).submit();
+      results = ctmi.process();
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+
+      assertEquals(Set.of(stf4, stf5), 
context.getAmple().readTablet(e1).getFiles());
+
+      // simulate a compaction
+      var stf6 = new StoredTabletFile(
+          
"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/A0000075.rf");
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireFile(stf4).requireFile(stf5)
+          .putFile(stf6, new DataFileValue(0, 
0)).deleteFile(stf4).deleteFile(stf5).submit();
+      results = ctmi.process();
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+
+      assertEquals(Set.of(stf6), context.getAmple().readTablet(e1).getFiles());
+
+      // simulate trying to re bulk import file after a compaction
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentBulkFile(stf5)
+          .putFile(stf5, new DataFileValue(0, 0)).putBulkFile(stf5, 9L)
+          .putFile(stf5, new DataFileValue(0, 0)).submit();
+      results = ctmi.process();
+      assertEquals(Status.REJECTED, results.get(e1).getStatus());
+
+      assertEquals(Set.of(stf6), context.getAmple().readTablet(e1).getFiles());
+    }
+  }
+
+  @Test
+  public void testMultipleExtents() throws Exception {
+    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      SortedSet<Text> splits = new TreeSet<>(List.of(new Text("c"), new 
Text("f"), new Text("j")));
+      c.tableOperations().create(tableName,
+          new NewTableConfiguration().withSplits(splits).createOffline());
+
+      c.securityOperations().grantTablePermission("root", MetadataTable.NAME,
+          TablePermission.WRITE);
+
+      var tid = TableId.of(c.tableOperations().tableIdMap().get(tableName));
+
+      var e1 = new KeyExtent(tid, new Text("c"), null);
+      var e2 = new KeyExtent(tid, new Text("f"), new Text("c"));
+      var e3 = new KeyExtent(tid, new Text("j"), new Text("f"));
+      var e4 = new KeyExtent(tid, null, new Text("j"));
+
+      var ts1 = new TServerInstance("localhost:9997", 5000L);
+      var ts2 = new TServerInstance("localhost:9997", 6000L);
+
+      var context = cluster.getServerContext();
+
+      var ctmi = new ConditionalTabletsMutatorImpl(context);
+      ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.future(ts1)).submit();
+      ctmi.mutateTablet(e2).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.future(ts2)).submit();
+      var results = ctmi.process();
+
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+      assertEquals(Status.ACCEPTED, results.get(e2).getStatus());
+
+      assertEquals(Location.future(ts1), 
context.getAmple().readTablet(e1).getLocation());
+      assertEquals(Location.future(ts2), 
context.getAmple().readTablet(e2).getLocation());
+      assertNull(context.getAmple().readTablet(e3).getLocation());
+      assertNull(context.getAmple().readTablet(e4).getLocation());
+
+      assertEquals(Set.of(e1, e2), results.keySet());
+
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      ctmi.mutateTablet(e1).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.future(ts2)).submit();
+      ctmi.mutateTablet(e2).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.future(ts1)).submit();
+      ctmi.mutateTablet(e3).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.future(ts1)).submit();
+      ctmi.mutateTablet(e4).requireAbsentOperation().requireAbsentLocation()
+          .putLocation(Location.future(ts2)).submit();
+      results = ctmi.process();
+
+      assertEquals(Status.REJECTED, results.get(e1).getStatus());
+      assertEquals(Status.REJECTED, results.get(e2).getStatus());
+      assertEquals(Status.ACCEPTED, results.get(e3).getStatus());
+      assertEquals(Status.ACCEPTED, results.get(e4).getStatus());
+
+      assertEquals(Location.future(ts1), 
context.getAmple().readTablet(e1).getLocation());
+      assertEquals(Location.future(ts2), 
context.getAmple().readTablet(e2).getLocation());
+      assertEquals(Location.future(ts1), 
context.getAmple().readTablet(e3).getLocation());
+      assertEquals(Location.future(ts2), 
context.getAmple().readTablet(e4).getLocation());
+
+      assertEquals(Set.of(e1, e2, e3, e4), results.keySet());
+
+    }
+  }
+
+  @Test
+  public void testOperations() throws Exception {
+    try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      SortedSet<Text> splits = new TreeSet<>(List.of(new Text("c"), new 
Text("f"), new Text("j")));
+      c.tableOperations().create(tableName,
+          new NewTableConfiguration().withSplits(splits).createOffline());
+
+      c.securityOperations().grantTablePermission("root", MetadataTable.NAME,
+          TablePermission.WRITE);
+
+      var tid = TableId.of(c.tableOperations().tableIdMap().get(tableName));
+
+      var e1 = new KeyExtent(tid, new Text("c"), null);
+      var e2 = new KeyExtent(tid, new Text("f"), new Text("c"));
+      var e3 = new KeyExtent(tid, new Text("j"), new Text("f"));
+
+      var context = cluster.getServerContext();
+
+      var opid1 = new TabletOperationId("1234");
+      var opid2 = new TabletOperationId("5678");
+
+      var ctmi = new ConditionalTabletsMutatorImpl(context);
+      
ctmi.mutateTablet(e1).requireAbsentOperation().putOperation(TabletOperation.SPLITTING,
 opid1)
+          .submit();
+      
ctmi.mutateTablet(e2).requireAbsentOperation().putOperation(TabletOperation.MERGING,
 opid2)
+          .submit();
+      ctmi.mutateTablet(e3).requireOperation(TabletOperation.SPLITTING, 
opid1).deleteOperation()
+          .submit();
+      var results = ctmi.process();
+
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+      assertEquals(Status.ACCEPTED, results.get(e2).getStatus());
+      assertEquals(Status.REJECTED, results.get(e3).getStatus());
+      assertEquals(TabletOperation.SPLITTING, 
context.getAmple().readTablet(e1).getOperation());
+      assertEquals(opid1, context.getAmple().readTablet(e1).getOperationId());
+      assertEquals(TabletOperation.MERGING, 
context.getAmple().readTablet(e2).getOperation());
+      assertEquals(opid2, context.getAmple().readTablet(e2).getOperationId());
+      assertEquals(null, context.getAmple().readTablet(e3).getOperation());
+      assertEquals(null, context.getAmple().readTablet(e3).getOperationId());
+
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      ctmi.mutateTablet(e1).requireOperation(TabletOperation.MERGING, 
opid2).deleteOperation()
+          .submit();
+      ctmi.mutateTablet(e2).requireOperation(TabletOperation.SPLITTING, 
opid1).deleteOperation()
+          .submit();
+      results = ctmi.process();
+
+      assertEquals(Status.REJECTED, results.get(e1).getStatus());
+      assertEquals(Status.REJECTED, results.get(e2).getStatus());
+      assertEquals(TabletOperation.SPLITTING, 
context.getAmple().readTablet(e1).getOperation());
+      assertEquals(TabletOperation.MERGING, 
context.getAmple().readTablet(e2).getOperation());
+
+      ctmi = new ConditionalTabletsMutatorImpl(context);
+      ctmi.mutateTablet(e1).requireOperation(TabletOperation.SPLITTING, 
opid1).deleteOperation()
+          .submit();
+      ctmi.mutateTablet(e2).requireOperation(TabletOperation.MERGING, 
opid2).deleteOperation()
+          .submit();
+      results = ctmi.process();
+
+      assertEquals(Status.ACCEPTED, results.get(e1).getStatus());
+      assertEquals(Status.ACCEPTED, results.get(e2).getStatus());
+      assertEquals(null, context.getAmple().readTablet(e1).getOperation());
+      assertEquals(null, context.getAmple().readTablet(e1).getOperationId());
+      assertEquals(null, context.getAmple().readTablet(e2).getOperation());
+      assertEquals(null, context.getAmple().readTablet(e2).getOperationId());
+    }
+  }
+
+  @Test
+  public void testRootTabletUpdate() throws Exception {
+    var context = cluster.getServerContext();
+
+    var rootMeta = context.getAmple().readTablet(RootTable.EXTENT);
+    var loc = rootMeta.getLocation();
+
+    assertEquals(LocationType.CURRENT, loc.getType());
+    assertFalse(rootMeta.getCompactId().isPresent());
+
+    var ctmi = new ConditionalTabletsMutatorImpl(context);
+    
ctmi.mutateTablet(RootTable.EXTENT).requireAbsentOperation().requireAbsentLocation()
+        .putCompactionId(7).submit();
+    var results = ctmi.process();
+    assertEquals(Status.REJECTED, results.get(RootTable.EXTENT).getStatus());
+    
assertFalse(context.getAmple().readTablet(RootTable.EXTENT).getCompactId().isPresent());
+
+    ctmi = new ConditionalTabletsMutatorImpl(context);
+    ctmi.mutateTablet(RootTable.EXTENT).requireAbsentOperation()
+        
.requireLocation(Location.future(loc.getServerInstance())).putCompactionId(7).submit();
+    results = ctmi.process();
+    assertEquals(Status.REJECTED, results.get(RootTable.EXTENT).getStatus());
+    
assertFalse(context.getAmple().readTablet(RootTable.EXTENT).getCompactId().isPresent());
+
+    ctmi = new ConditionalTabletsMutatorImpl(context);
+    ctmi.mutateTablet(RootTable.EXTENT).requireAbsentOperation()
+        
.requireLocation(Location.current(loc.getServerInstance())).putCompactionId(7).submit();
+    results = ctmi.process();
+    assertEquals(Status.ACCEPTED, results.get(RootTable.EXTENT).getStatus());
+    assertEquals(7L, 
context.getAmple().readTablet(RootTable.EXTENT).getCompactId().getAsLong());
+  }
+
+}


Reply via email to