You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2018/08/17 20:12:50 UTC

[accumulo] branch master updated: ACCUMULO-4629 Add table option to fail when deletes seen. (#598)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 72064e5  ACCUMULO-4629 Add table option to fail when deletes seen. (#598)
72064e5 is described below

commit 72064e52097fd1b5d319603b8728c85925b09ab0
Author: Keith Turner <ke...@deenlo.com>
AuthorDate: Fri Aug 17 16:12:48 2018 -0400

    ACCUMULO-4629 Add table option to fail when deletes seen. (#598)
    
    This experimental option allows for tables that do not use
    deletes to have fast seeks within the timestamp range.
---
 .../accumulo/core/client/impl/OfflineIterator.java |  2 +-
 .../accumulo/core/client/rfile/RFileScanner.java   |  2 +-
 .../accumulo/core/conf/AccumuloConfiguration.java  |  1 -
 .../org/apache/accumulo/core/conf/Property.java    | 17 +++++--
 .../core/constraints/NoDeleteConstraint.java       | 53 ++++++++++++++++++++
 .../org/apache/accumulo/core/data/Mutation.java    |  4 +-
 .../accumulo/core/iterators/IteratorUtil.java      |  5 +-
 .../core/iterators/system/DeletingIterator.java    | 37 ++++++++++++--
 .../core/constraints/NoDeleteConstraintTest.java   | 45 +++++++++++++++++
 .../iterators/system/DeletingIteratorTest.java     | 37 ++++++++++++--
 .../apache/accumulo/tserver/tablet/Compactor.java  |  3 +-
 .../accumulo/tserver/tablet/ScanDataSource.java    |  2 +-
 .../accumulo/test/functional/DeleteFailIT.java     | 57 ++++++++++++++++++++++
 .../test/performance/scan/CollectTabletStats.java  |  4 +-
 14 files changed, 247 insertions(+), 22 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineIterator.java
index af5626d..ce51605 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineIterator.java
@@ -383,7 +383,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
     defaultSecurityLabel = cv.getExpression();
 
     SortedKeyValueIterator<Key,Value> visFilter = IteratorUtil.setupSystemScanIterators(multiIter,
-        new HashSet<>(options.fetchedColumns), authorizations, defaultSecurityLabel);
+        new HashSet<>(options.fetchedColumns), authorizations, defaultSecurityLabel, acuTableConf);
 
     return iterEnv.getTopLevelIterator(
         IteratorUtil.loadIterators(IteratorScope.scan, visFilter, extent, acuTableConf,
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
index 15b7543..b073ada 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
@@ -382,7 +382,7 @@ class RFileScanner extends ScannerOptions implements Scanner {
         SortedSet<Column> cols = this.getFetchedColumns();
         families = LocalityGroupUtil.families(cols);
         iterator = IteratorUtil.setupSystemScanIterators(iterator, cols, getAuthorizations(),
-            EMPTY_BYTES);
+            EMPTY_BYTES, tableConf);
       }
 
       try {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
index 5427d79..cc658c0 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
@@ -489,5 +489,4 @@ public abstract class AccumuloConfiguration implements Iterable<Entry<String,Str
    * this configuration.
    */
   public void invalidateCache() {}
-
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index dd76695..ffe90d7 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -27,8 +27,10 @@ import java.util.Properties;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.constraints.NoDeleteConstraint;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.system.DeletingIterator;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher;
 import org.apache.accumulo.core.spi.scan.ScanPrioritizer;
@@ -809,6 +811,15 @@ public enum Property {
           + " To add a summarizer set "
           + "`table.summarizer.<unique id>=<summarizer class name>.` If the summarizer has options"
           + ", then for each option set" + " `table.summarizer.<unique id>.opt.<key>=<value>`."),
+  @Experimental
+  TABLE_DELETE_BEHAVIOR("table.delete.behavior",
+      DeletingIterator.Behavior.PROCESS.name().toLowerCase(), PropertyType.STRING,
+      "This determines what action to take when a delete marker is seen."
+          + " Valid values are `process` and `fail` with `process` being the default.  When set to "
+          + "`process`, deletes will supress data.  When set to `fail`, any deletes seen will cause an"
+          + " exception. The purpose of `fail` is to support tables that never delete data and need "
+          + "fast seeks within the timestamp range of a column. When setting this to fail, also "
+          + "consider configuring the `" + NoDeleteConstraint.class.getName() + "` constraint."),
 
   // VFS ClassLoader properties
   VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY(
@@ -884,9 +895,7 @@ public enum Property {
       "The sampling percentage to use for replication traces"),
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
-          + " the attempt. See replication.work.attempts."),
-
-  ;
+          + " the attempt. See replication.work.attempts.");
 
   private String key;
   private String defaultValue;
@@ -1141,7 +1150,7 @@ public enum Property {
       Property.TSERV_MAJC_MAXCONCURRENT, Property.REPLICATION_WORKER_THREADS,
       Property.TABLE_DURABILITY, Property.INSTANCE_ZK_TIMEOUT, Property.TABLE_CLASSPATH,
       Property.MASTER_METADATA_SUSPENDABLE, Property.TABLE_FAILURES_IGNORE,
-      Property.TABLE_SCAN_MAXMEM, Property.INSTANCE_CRYPTO_SERVICE);
+      Property.TABLE_SCAN_MAXMEM, Property.INSTANCE_CRYPTO_SERVICE, Property.TABLE_DELETE_BEHAVIOR);
 
   private static final EnumSet<Property> fixedProperties = EnumSet.of(Property.TSERV_CLIENTPORT,
       Property.TSERV_NATIVEMAP_ENABLED, Property.TSERV_SCAN_MAX_OPENFILES,
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java
new file mode 100644
index 0000000..af23022
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.accumulo.core.constraints;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.accumulo.core.data.ColumnUpdate;
+import org.apache.accumulo.core.data.Mutation;
+
+/**
+ * This constraint ensures mutations do not have deletes.
+ *
+ * @since 2.0.0
+ */
+public class NoDeleteConstraint implements Constraint {
+
+  @Override
+  public String getViolationDescription(short violationCode) {
+    if (violationCode == 1) {
+      return "Deletes are not allowed";
+    }
+    return null;
+  }
+
+  @Override
+  public List<Short> check(Environment env, Mutation mutation) {
+    List<ColumnUpdate> updates = mutation.getUpdates();
+    for (ColumnUpdate update : updates) {
+      if (update.isDeleted()) {
+        return Collections.singletonList((short) 1);
+      }
+    }
+
+    return null;
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
index 766b212..42094a2 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
@@ -954,8 +954,8 @@ public class Mutation implements Writable {
   }
 
   /**
-   * Fluent API for putting or deleting to a Mutation that makes it easy use different types
-   * (i.e byte[], CharSequence, etc) when specifying the family, qualifier, value, etc.
+   * Fluent API for putting or deleting to a Mutation that makes it easy use different types (i.e
+   * byte[], CharSequence, etc) when specifying the family, qualifier, value, etc.
    *
    * <p>
    * Methods are optional but must follow this order: family, qualifier, visibility, timestamp.
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
index d6e904c..0f922c8 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
@@ -449,8 +449,9 @@ public class IteratorUtil {
 
   public static SortedKeyValueIterator<Key,Value> setupSystemScanIterators(
       SortedKeyValueIterator<Key,Value> source, Set<Column> cols, Authorizations auths,
-      byte[] defaultVisibility) throws IOException {
-    DeletingIterator delIter = new DeletingIterator(source, false);
+      byte[] defaultVisibility, AccumuloConfiguration conf) throws IOException {
+    SortedKeyValueIterator<Key,Value> delIter = DeletingIterator.wrap(source, false,
+        DeletingIterator.getBehavior(conf));
     ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
     SortedKeyValueIterator<Key,Value> colFilter = ColumnQualifierFilter.wrap(cfsi, cols);
     return VisibilityFilter.wrap(colFilter, auths, defaultVisibility);
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java
index edb28f5..35a5778 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java
@@ -20,6 +20,8 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
 
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
@@ -34,18 +36,21 @@ public class DeletingIterator extends ServerWrappingIterator {
   private boolean propogateDeletes;
   private Key workKey = new Key();
 
+  public enum Behavior {
+    PROCESS, FAIL
+  }
+
   @Override
   public DeletingIterator deepCopy(IteratorEnvironment env) {
     return new DeletingIterator(this, env);
   }
 
-  public DeletingIterator(DeletingIterator other, IteratorEnvironment env) {
+  private DeletingIterator(DeletingIterator other, IteratorEnvironment env) {
     super(other.source.deepCopy(env));
     propogateDeletes = other.propogateDeletes;
   }
 
-  public DeletingIterator(SortedKeyValueIterator<Key,Value> iterator, boolean propogateDeletes)
-      throws IOException {
+  private DeletingIterator(SortedKeyValueIterator<Key,Value> iterator, boolean propogateDeletes) {
     super(iterator);
     this.propogateDeletes = propogateDeletes;
   }
@@ -105,4 +110,30 @@ public class DeletingIterator extends ServerWrappingIterator {
       IteratorEnvironment env) {
     throw new UnsupportedOperationException();
   }
+
+  public static SortedKeyValueIterator<Key,Value> wrap(SortedKeyValueIterator<Key,Value> source,
+      boolean propogateDeletes, Behavior behavior) {
+    switch (behavior) {
+      case PROCESS:
+        return new DeletingIterator(source, propogateDeletes);
+      case FAIL:
+        return new ServerWrappingIterator(source) {
+          @Override
+          public Key getTopKey() {
+            Key top = source.getTopKey();
+            if (top.isDeleted()) {
+              throw new IllegalStateException("Saw unexpected delete " + top);
+            }
+            return top;
+          }
+        };
+      default:
+        throw new IllegalArgumentException("Unknown behavior " + behavior);
+    }
+  }
+
+  public static Behavior getBehavior(AccumuloConfiguration conf) {
+    return DeletingIterator.Behavior
+        .valueOf(conf.get(Property.TABLE_DELETE_BEHAVIOR).toUpperCase());
+  }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/constraints/NoDeleteConstraintTest.java b/core/src/test/java/org/apache/accumulo/core/constraints/NoDeleteConstraintTest.java
new file mode 100644
index 0000000..ba89b1a
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/constraints/NoDeleteConstraintTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.constraints;
+
+import java.util.List;
+
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class NoDeleteConstraintTest {
+
+  @Test
+  public void testConstraint() {
+    Mutation m1 = new Mutation("r1");
+    m1.putDelete("f1", "q1");
+
+    NoDeleteConstraint ndc = new NoDeleteConstraint();
+
+    List<Short> results = ndc.check(null, m1);
+    Assert.assertEquals(1, results.size());
+    Assert.assertEquals(1, results.get(0).intValue());
+
+    Mutation m2 = new Mutation("r1");
+    m2.put("f1", "q1", new Value("v1"));
+
+    results = ndc.check(null, m2);
+    Assert.assertNull(results);
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/system/DeletingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/system/DeletingIteratorTest.java
index d0b6a1f..644f453 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/system/DeletingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/system/DeletingIteratorTest.java
@@ -26,7 +26,9 @@ import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
+import org.apache.accumulo.core.iterators.system.DeletingIterator.Behavior;
 import org.apache.hadoop.io.Text;
 
 import junit.framework.TestCase;
@@ -64,7 +66,8 @@ public class DeletingIteratorTest extends TestCase {
 
     Text checkRow = new Text("000");
     try {
-      DeletingIterator it = new DeletingIterator(new SortedMapIterator(tm), false);
+      SortedKeyValueIterator<Key,Value> it = DeletingIterator.wrap(new SortedMapIterator(tm), false,
+          Behavior.PROCESS);
       it.seek(new Range(), EMPTY_COL_FAMS, false);
 
       TreeMap<Key,Value> tmOut = new TreeMap<>();
@@ -86,7 +89,8 @@ public class DeletingIteratorTest extends TestCase {
     }
 
     try {
-      DeletingIterator it = new DeletingIterator(new SortedMapIterator(tm), true);
+      SortedKeyValueIterator<Key,Value> it = DeletingIterator.wrap(new SortedMapIterator(tm), true,
+          Behavior.PROCESS);
       it.seek(new Range(), EMPTY_COL_FAMS, false);
       TreeMap<Key,Value> tmOut = new TreeMap<>();
       while (it.hasTop()) {
@@ -122,7 +126,8 @@ public class DeletingIteratorTest extends TestCase {
     newKeyValue(tm, "r000", 2, true, "v2");
     newKeyValue(tm, "r000", 1, false, "v1");
 
-    DeletingIterator it = new DeletingIterator(new SortedMapIterator(tm), false);
+    SortedKeyValueIterator<Key,Value> it = DeletingIterator.wrap(new SortedMapIterator(tm), false,
+        Behavior.PROCESS);
 
     // SEEK two keys before delete
     it.seek(newRange("r000", 4), EMPTY_COL_FAMS, false);
@@ -172,7 +177,8 @@ public class DeletingIteratorTest extends TestCase {
     newKeyValue(tm, "r000", 2, true, "");
     newKeyValue(tm, "r000", 1, false, "v1");
 
-    DeletingIterator it = new DeletingIterator(new SortedMapIterator(tm), false);
+    SortedKeyValueIterator<Key,Value> it = DeletingIterator.wrap(new SortedMapIterator(tm), false,
+        Behavior.PROCESS);
     it.seek(new Range(), EMPTY_COL_FAMS, false);
 
     assertTrue(it.hasTop());
@@ -197,7 +203,8 @@ public class DeletingIteratorTest extends TestCase {
     newKeyValue(tm, "r000", 2, true, "");
     newKeyValue(tm, "r000", 1, false, "v1");
 
-    DeletingIterator it = new DeletingIterator(new SortedMapIterator(tm), false);
+    SortedKeyValueIterator<Key,Value> it = DeletingIterator.wrap(new SortedMapIterator(tm), false,
+        Behavior.PROCESS);
 
     it.seek(newRange("r000", 3), EMPTY_COL_FAMS, false);
 
@@ -214,6 +221,26 @@ public class DeletingIteratorTest extends TestCase {
     assertFalse(it.hasTop());
   }
 
+  public void testFail() throws IOException {
+    TreeMap<Key,Value> tm = new TreeMap<>();
+
+    newKeyValue(tm, "r000", 3, false, "v3");
+    newKeyValue(tm, "r000", 2, false, "v2");
+    newKeyValue(tm, "r000", 2, true, "");
+    newKeyValue(tm, "r000", 1, false, "v1");
+
+    SortedKeyValueIterator<Key,Value> it = DeletingIterator.wrap(new SortedMapIterator(tm), false,
+        Behavior.FAIL);
+    it.seek(new Range(), EMPTY_COL_FAMS, false);
+    try {
+      while (it.hasTop()) {
+        it.getTopKey();
+        it.next();
+      }
+      fail();
+    } catch (IllegalStateException e) {}
+  }
+
   private Range newRange(String row, long ts, boolean inclusive) {
     return new Range(newKey(row, ts), inclusive, null, true);
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
index fdfbca8..1721dbe 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
@@ -344,7 +344,8 @@ public class Compactor implements Callable<CompactionStats> {
 
       CountingIterator citr = new CountingIterator(new MultiIterator(iters, extent.toDataRange()),
           entriesRead);
-      DeletingIterator delIter = new DeletingIterator(citr, propogateDeletes);
+      SortedKeyValueIterator<Key,Value> delIter = DeletingIterator.wrap(citr, propogateDeletes,
+          DeletingIterator.getBehavior(acuTableConf));
       ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
 
       // if(env.getIteratorScope() )
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
index e75918f..3fc3b64 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
@@ -200,7 +200,7 @@ class ScanDataSource implements DataSource {
 
     SortedKeyValueIterator<Key,Value> visFilter = IteratorUtil.setupSystemScanIterators(
         statsIterator, options.getColumnSet(), options.getAuthorizations(),
-        options.getDefaultLabels());
+        options.getDefaultLabels(), tablet.getTableConfiguration());
 
     if (!loadIters) {
       return visFilter;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteFailIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteFailIT.java
new file mode 100644
index 0000000..1281ff6
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteFailIT.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.functional;
+
+import java.util.Collections;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.iterators.system.DeletingIterator.Behavior;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class DeleteFailIT extends AccumuloClusterHarness {
+
+  @Test
+  public void testFail() throws Exception {
+    Connector c = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.setProperties(
+        Collections.singletonMap(Property.TABLE_DELETE_BEHAVIOR.getKey(), Behavior.FAIL.name()));
+    c.tableOperations().create(tableName, ntc);
+
+    try (BatchWriter writer = c.createBatchWriter(tableName)) {
+      Mutation m = new Mutation("1234");
+      m.putDelete("f1", "q1", 2L);
+      writer.addMutation(m);
+    }
+
+    Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);
+
+    try {
+      scanner.forEach(e -> {});
+      Assert.fail("Expected scan to fail because  deletes are present.");
+    } catch (RuntimeException e) {}
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index ea09a15..2e0e965 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -57,6 +57,7 @@ import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
 import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter;
 import org.apache.accumulo.core.iterators.system.DeletingIterator;
+import org.apache.accumulo.core.iterators.system.DeletingIterator.Behavior;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.accumulo.core.iterators.system.VisibilityFilter;
 import org.apache.accumulo.core.metadata.MetadataServicer;
@@ -439,7 +440,8 @@ public class CollectTabletStats {
     iters.add(smi);
 
     MultiIterator multiIter = new MultiIterator(iters, ke);
-    DeletingIterator delIter = new DeletingIterator(multiIter, false);
+    SortedKeyValueIterator<Key,Value> delIter = DeletingIterator.wrap(multiIter, false,
+        Behavior.PROCESS);
     ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
     SortedKeyValueIterator<Key,Value> colFilter = ColumnQualifierFilter.wrap(cfsi, columnSet);
     SortedKeyValueIterator<Key,Value> visFilter = VisibilityFilter.wrap(colFilter, authorizations,