You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/08/17 20:12:49 UTC

[GitHub] keith-turner closed pull request #598: ACCUMULO-4629 Add table option to fail when deletes seen.

keith-turner closed pull request #598: ACCUMULO-4629 Add table option to fail when deletes seen.
URL: https://github.com/apache/accumulo/pull/598
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 af5626d66c..ce51605823 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 @@ private void nextTablet() throws TableNotFoundException, AccumuloException, IOEx
     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 15b7543483..b073ada999 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 @@ public SamplerConfiguration getSamplerConfiguration() {
         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 5427d79310..cc658c0fe2 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 @@ Integer getDeprecatedScanThreads(String name) {
    * 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 dd76695d96..ffe90d7374 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 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 @@
           + " 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 @@
       "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 static boolean isValidTablePropertyKey(String key) {
       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 0000000000..af23022854
--- /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 766b212c07..42094a2cf0 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 void putDelete(byte[] columnFamily, byte[] columnQualifier,
   }
 
   /**
-   * 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 d6e904c208..0f922c8db0 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 static IteratorConfig toIteratorConfig(List<IteratorSetting> iterators) {
 
   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 edb28f5508..35a5778b6e 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.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 @@
   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 void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> op
       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 0000000000..ba89b1a0cb
--- /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 d0b6a1f339..644f453b7d 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.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 void test1() {
 
     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 void test1() {
     }
 
     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 void test2() throws IOException {
     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 void test3() throws IOException {
     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 void test4() throws IOException {
     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 void test4() throws IOException {
     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 fdfbca8e2c..1721dbe4f6 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 @@ private void compactLocalityGroup(String lgName, Set<ByteSequence> columnFamilie
 
       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 e75918f676..3fc3b64404 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 @@ public boolean isCurrent() {
 
     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 0000000000..1281ff6d35
--- /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 ea09a1512b..2e0e96556b 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.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 @@ private static void reportHdfsBlockLocations(List<FileRef> files) throws Excepti
     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,


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services