You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mm...@apache.org on 2017/04/27 21:34:55 UTC
[1/2] accumulo git commit: ACCUMULO-3079: Collapsed iterator stack by
creating new optimized iterators
Repository: accumulo
Updated Branches:
refs/heads/master ce4017884 -> f81a8ec74
ACCUMULO-3079: Collapsed iterator stack by creating new optimized iterators
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/b56a3349
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/b56a3349
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/b56a3349
Branch: refs/heads/master
Commit: b56a334932362f1b00f2c6a80f9d7298c204d1d6
Parents: ce40178
Author: Mike Miller <mm...@apache.org>
Authored: Fri Mar 31 12:43:45 2017 -0400
Committer: Mike Miller <mm...@apache.org>
Committed: Thu Apr 27 15:32:50 2017 -0400
----------------------------------------------------------------------
.../accumulo/core/iterators/IteratorUtil.java | 3 +-
.../accumulo/core/iterators/ServerFilter.java | 78 ++++++++++++++++
.../core/iterators/ServerSkippingIterator.java | 54 +++++++++++
.../core/iterators/ServerWrappingIterator.java | 79 ++++++++++++++++
.../iterators/SynchronizedServerFilter.java | 94 ++++++++++++++++++++
.../system/ColumnFamilySkippingIterator.java | 30 +++----
.../iterators/system/ColumnQualifierFilter.java | 62 ++++++-------
.../core/iterators/system/DeletingIterator.java | 28 +++---
.../core/iterators/system/StatsIterator.java | 12 +--
.../iterators/system/SynchronizedIterator.java | 7 +-
.../core/iterators/system/VisibilityFilter.java | 40 +++++----
.../core/iterators/user/VisibilityFilter.java | 49 +++++++---
.../problems/ProblemReportingIterator.java | 4 +-
.../apache/accumulo/tserver/InMemoryMap.java | 7 --
14 files changed, 428 insertions(+), 119 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
----------------------------------------------------------------------
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 8bd72bf..338da90 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
@@ -44,7 +44,6 @@ import org.apache.accumulo.core.data.thrift.IterInfo;
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.SynchronizedIterator;
import org.apache.accumulo.core.iterators.system.VisibilityFilter;
import org.apache.accumulo.core.iterators.user.VersioningIterator;
import org.apache.accumulo.core.security.Authorizations;
@@ -255,7 +254,7 @@ public class IteratorUtil {
Collection<IterInfo> iters, Map<String,Map<String,String>> iterOpts, IteratorEnvironment env, boolean useAccumuloClassLoader, String context,
Map<String,Class<? extends SortedKeyValueIterator<K,V>>> classCache) throws IOException {
// wrap the source in a SynchronizedIterator in case any of the additional configured iterators want to use threading
- SortedKeyValueIterator<K,V> prev = new SynchronizedIterator<>(source);
+ SortedKeyValueIterator<K,V> prev = source;
try {
for (IterInfo iterInfo : iters) {
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java
new file mode 100644
index 0000000..c6db6c4
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java
@@ -0,0 +1,78 @@
+/*
+ * 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.iterators;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+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;
+
+/**
+ * An optimized version of {@link org.apache.accumulo.core.iterators.Filter}. This class grants protected access to the read only <code>source</code> iterator.
+ * For performance reasons, the <code>source</code> iterator is declared final and subclasses directly access it, no longer requiring calls to getSource(). The
+ * {@link #init(SortedKeyValueIterator, Map, IteratorEnvironment)} method is not supported since the source can only be assigned in the constructor.
+ *
+ * @since 2.0
+ */
+public abstract class ServerFilter extends ServerWrappingIterator {
+
+ public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
+ super(source);
+ }
+
+ @Override
+ public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
+
+ @Override
+ public void next() throws IOException {
+ source.next();
+ findTop();
+ }
+
+ @Override
+ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+ source.seek(range, columnFamilies, inclusive);
+ findTop();
+ }
+
+ /**
+ * Iterates over the source until an acceptable key/value pair is found.
+ */
+ private void findTop() throws IOException {
+ while (source.hasTop()) {
+ Key top = source.getTopKey();
+ if (top.isDeleted() || (accept(top, source.getTopValue()))) {
+ break;
+ }
+ source.next();
+ }
+ }
+
+ /**
+ * @return <tt>true</tt> if the key/value pair is accepted by the filter.
+ */
+ public abstract boolean accept(Key k, Value v);
+
+ @Override
+ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java
new file mode 100644
index 0000000..fb6c59c
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java
@@ -0,0 +1,54 @@
+/*
+ * 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.iterators;
+
+import java.io.IOException;
+import java.util.Collection;
+
+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;
+
+/**
+ * An optimized version of {@link org.apache.accumulo.core.iterators.SkippingIterator}. This class grants protected access to the read only <code>source</code>
+ * iterator. For performance reasons, the <code>source</code> iterator is declared final and subclasses directly access it, no longer requiring calls to
+ * getSource().
+ *
+ * @since 2.0
+ */
+public abstract class ServerSkippingIterator extends ServerWrappingIterator {
+
+ public ServerSkippingIterator(SortedKeyValueIterator<Key,Value> source) {
+ super(source);
+ }
+
+ @Override
+ public void next() throws IOException {
+ source.next();
+ consume();
+ }
+
+ protected abstract void consume() throws IOException;
+
+ @Override
+ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+ source.seek(range, columnFamilies, inclusive);
+ consume();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java
new file mode 100644
index 0000000..7f084c9
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java
@@ -0,0 +1,79 @@
+/*
+ * 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.iterators;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+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;
+
+/**
+ * An optimized version of {@link org.apache.accumulo.core.iterators.WrappingIterator}. This class grants protected access to the read only <code>source</code>
+ * iterator. For performance reasons, the <code>source</code> iterator is declared final and subclasses directly access it, no longer requiring calls to
+ * getSource(). The {@link #init(SortedKeyValueIterator, Map, IteratorEnvironment)} method is not supported since the source can only be assigned in the
+ * constructor. As with the WrappingIterator, the {@link #deepCopy(IteratorEnvironment)} method is not supported.
+ *
+ * @since 2.0
+ */
+public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
+
+ protected final SortedKeyValueIterator<Key,Value> source;
+
+ public ServerWrappingIterator(SortedKeyValueIterator<Key,Value> source) {
+ this.source = source;
+ }
+
+ @Override
+ public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Key getTopKey() {
+ return source.getTopKey();
+ }
+
+ @Override
+ public Value getTopValue() {
+ return source.getTopValue();
+ }
+
+ @Override
+ public boolean hasTop() {
+ return source.hasTop();
+ }
+
+ @Override
+ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void next() throws IOException {
+ source.next();
+ }
+
+ @Override
+ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+ source.seek(range, columnFamilies, inclusive);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/SynchronizedServerFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/SynchronizedServerFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/SynchronizedServerFilter.java
new file mode 100644
index 0000000..b2ad4c2
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/SynchronizedServerFilter.java
@@ -0,0 +1,94 @@
+/*
+ * 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.iterators;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+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;
+
+/**
+ * A SortedKeyValueIterator similar to {@link org.apache.accumulo.core.iterators.ServerFilter} but with the implemented methods marked as synchronized. The
+ * {@link #init(SortedKeyValueIterator, Map, IteratorEnvironment)} method is also not supported since the source can only be assigned in the constructor.
+ *
+ * @since 2.0
+ */
+public abstract class SynchronizedServerFilter implements SortedKeyValueIterator<Key,Value> {
+
+ protected final SortedKeyValueIterator<Key,Value> source;
+
+ public SynchronizedServerFilter(SortedKeyValueIterator<Key,Value> source) {
+ this.source = source;
+ }
+
+ @Override
+ public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
+
+ @Override
+ public synchronized void next() throws IOException {
+ source.next();
+ findTop();
+ }
+
+ @Override
+ public synchronized void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+ source.seek(range, columnFamilies, inclusive);
+ findTop();
+ }
+
+ @Override
+ public synchronized Key getTopKey() {
+ return source.getTopKey();
+ }
+
+ @Override
+ public synchronized Value getTopValue() {
+ return source.getTopValue();
+ }
+
+ @Override
+ public synchronized boolean hasTop() {
+ return source.hasTop();
+ }
+
+ /**
+ * Iterates over the source until an acceptable key/value pair is found.
+ */
+ private void findTop() throws IOException {
+ while (source.hasTop()) {
+ Key top = source.getTopKey();
+ if (top.isDeleted() || (accept(top, source.getTopValue()))) {
+ break;
+ }
+ source.next();
+ }
+ }
+
+ /**
+ * @return <tt>true</tt> if the key/value pair is accepted by the filter.
+ */
+ protected abstract boolean accept(Key k, Value v);
+
+ @Override
+ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java
index 53f3643..8728445 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java
@@ -29,10 +29,10 @@ import org.apache.accumulo.core.data.PartialKey;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SkippingIterator;
+import org.apache.accumulo.core.iterators.ServerSkippingIterator;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-public class ColumnFamilySkippingIterator extends SkippingIterator implements InterruptibleIterator {
+public class ColumnFamilySkippingIterator extends ServerSkippingIterator implements InterruptibleIterator {
protected Set<ByteSequence> colFamSet = null;
protected TreeSet<ByteSequence> sortedColFams = null;
@@ -41,7 +41,7 @@ public class ColumnFamilySkippingIterator extends SkippingIterator implements In
protected Range range;
public ColumnFamilySkippingIterator(SortedKeyValueIterator<Key,Value> source) {
- this.setSource(source);
+ super(source);
}
protected ColumnFamilySkippingIterator(SortedKeyValueIterator<Key,Value> source, Set<ByteSequence> colFamSet, boolean inclusive) {
@@ -55,33 +55,33 @@ public class ColumnFamilySkippingIterator extends SkippingIterator implements In
int count = 0;
if (inclusive)
- while (getSource().hasTop() && !colFamSet.contains(getSource().getTopKey().getColumnFamilyData())) {
+ while (source.hasTop() && !colFamSet.contains(source.getTopKey().getColumnFamilyData())) {
if (count < 10) {
// it is quicker to call next if we are close, but we never know if we are close
// so give next a try a few times
- getSource().next();
+ source.next();
count++;
} else {
- ByteSequence higherCF = sortedColFams.higher(getSource().getTopKey().getColumnFamilyData());
+ ByteSequence higherCF = sortedColFams.higher(source.getTopKey().getColumnFamilyData());
if (higherCF == null) {
// seek to the next row
- reseek(getSource().getTopKey().followingKey(PartialKey.ROW));
+ reseek(source.getTopKey().followingKey(PartialKey.ROW));
} else {
// seek to the next column family in the sorted list of column families
- reseek(new Key(getSource().getTopKey().getRowData().toArray(), higherCF.toArray(), new byte[0], new byte[0], Long.MAX_VALUE));
+ reseek(new Key(source.getTopKey().getRowData().toArray(), higherCF.toArray(), new byte[0], new byte[0], Long.MAX_VALUE));
}
count = 0;
}
}
else if (colFamSet != null && colFamSet.size() > 0)
- while (getSource().hasTop() && colFamSet.contains(getSource().getTopKey().getColumnFamilyData())) {
+ while (source.hasTop() && colFamSet.contains(source.getTopKey().getColumnFamilyData())) {
if (count < 10) {
- getSource().next();
+ source.next();
count++;
} else {
// seek to the next column family in the data
- reseek(getSource().getTopKey().followingKey(PartialKey.ROW_COLFAM));
+ reseek(source.getTopKey().followingKey(PartialKey.ROW_COLFAM));
count = 0;
}
}
@@ -90,16 +90,16 @@ public class ColumnFamilySkippingIterator extends SkippingIterator implements In
private void reseek(Key key) throws IOException {
if (range.afterEndKey(key)) {
range = new Range(range.getEndKey(), true, range.getEndKey(), range.isEndKeyInclusive());
- getSource().seek(range, colFamSet, inclusive);
+ source.seek(range, colFamSet, inclusive);
} else {
range = new Range(key, true, range.getEndKey(), range.isEndKeyInclusive());
- getSource().seek(range, colFamSet, inclusive);
+ source.seek(range, colFamSet, inclusive);
}
}
@Override
public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new ColumnFamilySkippingIterator(getSource().deepCopy(env), colFamSet, inclusive);
+ return new ColumnFamilySkippingIterator(source.deepCopy(env), colFamSet, inclusive);
}
@Override
@@ -125,7 +125,7 @@ public class ColumnFamilySkippingIterator extends SkippingIterator implements In
@Override
public void setInterruptFlag(AtomicBoolean flag) {
- ((InterruptibleIterator) getSource()).setInterruptFlag(flag);
+ ((InterruptibleIterator) source).setInterruptFlag(flag);
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
index 866f04f..dbd9171 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.core.iterators.system;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.Iterator;
import java.util.Set;
import org.apache.accumulo.core.data.ArrayByteSequence;
@@ -26,25 +25,43 @@ import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Column;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.Filter;
import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.ServerFilter;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-public class ColumnQualifierFilter extends Filter {
- private boolean scanColumns;
+public class ColumnQualifierFilter extends ServerFilter {
+ private final boolean scanColumns;
private HashSet<ByteSequence> columnFamilies;
private HashMap<ByteSequence,HashSet<ByteSequence>> columnsQualifiers;
- public ColumnQualifierFilter() {}
-
public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, Set<Column> columns) {
- setSource(iterator);
- init(columns);
+ super(iterator);
+ this.columnFamilies = new HashSet<>();
+ this.columnsQualifiers = new HashMap<>();
+
+ for (Column col : columns) {
+ if (col.columnQualifier != null) {
+ ArrayByteSequence cq = new ArrayByteSequence(col.columnQualifier);
+ HashSet<ByteSequence> cfset = this.columnsQualifiers.get(cq);
+ if (cfset == null) {
+ cfset = new HashSet<>();
+ this.columnsQualifiers.put(cq, cfset);
+ }
+
+ cfset.add(new ArrayByteSequence(col.columnFamily));
+ } else {
+ // this whole column family should pass
+ columnFamilies.add(new ArrayByteSequence(col.columnFamily));
+ }
+ }
+
+ // only take action when column qualifies are present
+ scanColumns = this.columnsQualifiers.size() > 0;
}
public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, HashSet<ByteSequence> columnFamilies,
HashMap<ByteSequence,HashSet<ByteSequence>> columnsQualifiers, boolean scanColumns) {
- setSource(iterator);
+ super(iterator);
this.columnFamilies = columnFamilies;
this.columnsQualifiers = columnsQualifiers;
this.scanColumns = scanColumns;
@@ -65,33 +82,8 @@ public class ColumnQualifierFilter extends Filter {
return cfset != null && cfset.contains(key.getColumnFamilyData());
}
- public void init(Set<Column> columns) {
- this.columnFamilies = new HashSet<>();
- this.columnsQualifiers = new HashMap<>();
-
- for (Iterator<Column> iter = columns.iterator(); iter.hasNext();) {
- Column col = iter.next();
- if (col.columnQualifier != null) {
- ArrayByteSequence cq = new ArrayByteSequence(col.columnQualifier);
- HashSet<ByteSequence> cfset = this.columnsQualifiers.get(cq);
- if (cfset == null) {
- cfset = new HashSet<>();
- this.columnsQualifiers.put(cq, cfset);
- }
-
- cfset.add(new ArrayByteSequence(col.columnFamily));
- } else {
- // this whole column family should pass
- columnFamilies.add(new ArrayByteSequence(col.columnFamily));
- }
- }
-
- // only take action when column qualifies are present
- scanColumns = this.columnsQualifiers.size() > 0;
- }
-
@Override
public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new ColumnQualifierFilter(getSource().deepCopy(env), columnFamilies, columnsQualifiers, scanColumns);
+ return new ColumnQualifierFilter(source.deepCopy(env), columnFamilies, columnsQualifiers, scanColumns);
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java
----------------------------------------------------------------------
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 abdb6c1..e33ee3a 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
@@ -27,10 +27,10 @@ import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.iterators.IteratorEnvironment;
import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.ServerWrappingIterator;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.WrappingIterator;
-public class DeletingIterator extends WrappingIterator {
+public class DeletingIterator extends ServerWrappingIterator {
private boolean propogateDeletes;
private Key workKey = new Key();
@@ -40,23 +40,21 @@ public class DeletingIterator extends WrappingIterator {
}
public DeletingIterator(DeletingIterator other, IteratorEnvironment env) {
- setSource(other.getSource().deepCopy(env));
+ super(other.source.deepCopy(env));
propogateDeletes = other.propogateDeletes;
}
- public DeletingIterator() {}
-
public DeletingIterator(SortedKeyValueIterator<Key,Value> iterator, boolean propogateDeletes) throws IOException {
- this.setSource(iterator);
+ super(iterator);
this.propogateDeletes = propogateDeletes;
}
@Override
public void next() throws IOException {
- if (super.getTopKey().isDeleted())
+ if (source.getTopKey().isDeleted())
skipRowColumn();
else
- getSource().next();
+ source.next();
findTop();
}
@@ -65,11 +63,11 @@ public class DeletingIterator extends WrappingIterator {
// do not want to seek to the middle of a row
Range seekRange = IteratorUtil.maximizeStartKeyTimeStamp(range);
- super.seek(seekRange, columnFamilies, inclusive);
+ source.seek(seekRange, columnFamilies, inclusive);
findTop();
if (range.getStartKey() != null) {
- while (getSource().hasTop() && getSource().getTopKey().compareTo(range.getStartKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME) < 0) {
+ while (source.hasTop() && source.getTopKey().compareTo(range.getStartKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME) < 0) {
next();
}
@@ -81,20 +79,20 @@ public class DeletingIterator extends WrappingIterator {
private void findTop() throws IOException {
if (!propogateDeletes) {
- while (getSource().hasTop() && getSource().getTopKey().isDeleted()) {
+ while (source.hasTop() && source.getTopKey().isDeleted()) {
skipRowColumn();
}
}
}
private void skipRowColumn() throws IOException {
- workKey.set(getSource().getTopKey());
+ workKey.set(source.getTopKey());
Key keyToSkip = workKey;
- getSource().next();
+ source.next();
- while (getSource().hasTop() && getSource().getTopKey().equals(keyToSkip, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) {
- getSource().next();
+ while (source.hasTop() && source.getTopKey().equals(keyToSkip, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) {
+ source.next();
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java
index f92d1ec..160a29f 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java
@@ -25,27 +25,27 @@ 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.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.ServerWrappingIterator;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.WrappingIterator;
/**
*
*/
-public class StatsIterator extends WrappingIterator {
+public class StatsIterator extends ServerWrappingIterator {
private int numRead = 0;
private AtomicLong seekCounter;
private AtomicLong readCounter;
public StatsIterator(SortedKeyValueIterator<Key,Value> source, AtomicLong seekCounter, AtomicLong readCounter) {
- super.setSource(source);
+ super(source);
this.seekCounter = seekCounter;
this.readCounter = readCounter;
}
@Override
public void next() throws IOException {
- super.next();
+ source.next();
numRead++;
if (numRead % 23 == 0) {
@@ -56,12 +56,12 @@ public class StatsIterator extends WrappingIterator {
@Override
public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new StatsIterator(getSource().deepCopy(env), seekCounter, readCounter);
+ return new StatsIterator(source.deepCopy(env), seekCounter, readCounter);
}
@Override
public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
- super.seek(range, columnFamilies, inclusive);
+ source.seek(range, columnFamilies, inclusive);
seekCounter.incrementAndGet();
readCounter.addAndGet(numRead);
numRead = 0;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.java
index 43da54d..fab0439 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.java
@@ -33,12 +33,11 @@ import org.apache.hadoop.io.WritableComparable;
*/
public class SynchronizedIterator<K extends WritableComparable<?>,V extends Writable> implements SortedKeyValueIterator<K,V> {
- private SortedKeyValueIterator<K,V> source = null;
+ private final SortedKeyValueIterator<K,V> source;
@Override
public synchronized void init(SortedKeyValueIterator<K,V> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
- this.source = source;
- source.init(source, options, env);
+ throw new UnsupportedOperationException();
}
@Override
@@ -71,8 +70,6 @@ public class SynchronizedIterator<K extends WritableComparable<?>,V extends Writ
return new SynchronizedIterator<>(source.deepCopy(env));
}
- public SynchronizedIterator() {}
-
public SynchronizedIterator(SortedKeyValueIterator<K,V> source) {
this.source = source;
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
index a204ad1..b260bc4 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
@@ -16,54 +16,56 @@
*/
package org.apache.accumulo.core.iterators.system;
+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.Value;
-import org.apache.accumulo.core.iterators.Filter;
import org.apache.accumulo.core.iterators.IteratorEnvironment;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.SynchronizedServerFilter;
import org.apache.accumulo.core.security.Authorizations;
import org.apache.accumulo.core.security.ColumnVisibility;
import org.apache.accumulo.core.security.VisibilityEvaluator;
import org.apache.accumulo.core.security.VisibilityParseException;
import org.apache.accumulo.core.util.BadArgumentException;
-import org.apache.accumulo.core.util.TextUtil;
import org.apache.commons.collections.map.LRUMap;
-import org.apache.hadoop.io.Text;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class VisibilityFilter extends Filter {
+/**
+ * A SortedKeyValueIterator that filters based on ColumnVisibility and optimized for use with system iterators. Prior to 2.0, this class extended
+ * {@link org.apache.accumulo.core.iterators.Filter} and all system iterators where wrapped with a <code>SynchronizedIterator</code> during creation of the
+ * iterator stack in {@link org.apache.accumulo.core.iterators.IteratorUtil} .loadIterators(). For performance reasons, the synchronization was pushed down the
+ * stack to this class.
+ */
+public class VisibilityFilter extends SynchronizedServerFilter {
protected VisibilityEvaluator ve;
- protected Text defaultVisibility;
+ protected ByteSequence defaultVisibility;
protected LRUMap cache;
- protected Text tmpVis;
protected Authorizations authorizations;
private static final Logger log = LoggerFactory.getLogger(VisibilityFilter.class);
- public VisibilityFilter() {}
-
public VisibilityFilter(SortedKeyValueIterator<Key,Value> iterator, Authorizations authorizations, byte[] defaultVisibility) {
- setSource(iterator);
+ super(iterator);
this.ve = new VisibilityEvaluator(authorizations);
this.authorizations = authorizations;
- this.defaultVisibility = new Text(defaultVisibility);
+ this.defaultVisibility = new ArrayByteSequence(defaultVisibility);
this.cache = new LRUMap(1000);
- this.tmpVis = new Text();
}
@Override
- public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new VisibilityFilter(getSource().deepCopy(env), authorizations, TextUtil.getBytes(defaultVisibility));
+ public synchronized SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+ return new VisibilityFilter(source.deepCopy(env), authorizations, defaultVisibility.toArray());
}
@Override
- public boolean accept(Key k, Value v) {
- Text testVis = k.getColumnVisibility(tmpVis);
+ protected boolean accept(Key k, Value v) {
+ ByteSequence testVis = k.getColumnVisibilityData();
- if (testVis.getLength() == 0 && defaultVisibility.getLength() == 0)
+ if (testVis.length() == 0 && defaultVisibility.length() == 0)
return true;
- else if (testVis.getLength() == 0)
+ else if (testVis.length() == 0)
testVis = defaultVisibility;
Boolean b = (Boolean) cache.get(testVis);
@@ -71,8 +73,8 @@ public class VisibilityFilter extends Filter {
return b;
try {
- Boolean bb = ve.evaluate(new ColumnVisibility(testVis));
- cache.put(new Text(testVis), bb);
+ Boolean bb = ve.evaluate(new ColumnVisibility(testVis.toArray()));
+ cache.put(testVis, bb);
return bb;
} catch (VisibilityParseException e) {
log.error("Parse Error", e);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
index f7007a1..e0c0b0f 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
@@ -22,21 +22,30 @@ import java.io.IOException;
import java.util.Map;
import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.Filter;
import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.OptionDescriber;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
import org.apache.accumulo.core.security.Authorizations;
import org.apache.accumulo.core.security.ColumnVisibility;
import org.apache.accumulo.core.security.VisibilityEvaluator;
+import org.apache.accumulo.core.security.VisibilityParseException;
import org.apache.accumulo.core.util.BadArgumentException;
import org.apache.commons.collections.map.LRUMap;
-import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
/**
- *
+ * A SortedKeyValueIterator that filters based on ColumnVisibility.
*/
-public class VisibilityFilter extends org.apache.accumulo.core.iterators.system.VisibilityFilter {
+public class VisibilityFilter extends Filter implements OptionDescriber {
+
+ protected VisibilityEvaluator ve;
+ protected LRUMap cache;
+
+ private static final Logger log = Logger.getLogger(VisibilityFilter.class);
private static final String AUTHS = "auths";
private static final String FILTER_INVALID_ONLY = "filterInvalid";
@@ -46,9 +55,7 @@ public class VisibilityFilter extends org.apache.accumulo.core.iterators.system.
/**
*
*/
- public VisibilityFilter() {
- super();
- }
+ public VisibilityFilter() {}
@Override
public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
@@ -60,29 +67,45 @@ public class VisibilityFilter extends org.apache.accumulo.core.iterators.system.
String auths = options.get(AUTHS);
Authorizations authObj = auths == null || auths.isEmpty() ? new Authorizations() : new Authorizations(auths.getBytes(UTF_8));
this.ve = new VisibilityEvaluator(authObj);
- this.defaultVisibility = new Text();
}
this.cache = new LRUMap(1000);
- this.tmpVis = new Text();
}
@Override
public boolean accept(Key k, Value v) {
+ ByteSequence testVis = k.getColumnVisibilityData();
if (filterInvalid) {
- Text testVis = k.getColumnVisibility(tmpVis);
Boolean b = (Boolean) cache.get(testVis);
if (b != null)
return b;
try {
- new ColumnVisibility(testVis);
- cache.put(new Text(testVis), true);
+ new ColumnVisibility(testVis.toArray());
+ cache.put(testVis, true);
return true;
} catch (BadArgumentException e) {
- cache.put(new Text(testVis), false);
+ cache.put(testVis, false);
return false;
}
} else {
- return super.accept(k, v);
+ if (testVis.length() == 0) {
+ return true;
+ }
+
+ Boolean b = (Boolean) cache.get(testVis);
+ if (b != null)
+ return b;
+
+ try {
+ Boolean bb = ve.evaluate(new ColumnVisibility(testVis.toArray()));
+ cache.put(testVis, bb);
+ return bb;
+ } catch (VisibilityParseException e) {
+ log.error("Parse Error", e);
+ return false;
+ } catch (BadArgumentException e) {
+ log.error("Parse Error", e);
+ return false;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
index 83b4615..7e1676a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
@@ -31,9 +31,9 @@ import org.apache.accumulo.core.iterators.system.InterruptibleIterator;
import org.apache.accumulo.server.AccumuloServerContext;
public class ProblemReportingIterator implements InterruptibleIterator {
- private SortedKeyValueIterator<Key,Value> source;
+ private final SortedKeyValueIterator<Key,Value> source;
private boolean sawError = false;
- private boolean continueOnError;
+ private final boolean continueOnError;
private String resource;
private String tableId;
private final AccumuloServerContext context;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b56a3349/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
index 9d5e0d0..29d5ca0 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
@@ -683,13 +683,6 @@ public class InMemoryMap {
private SourceSwitchingIterator ssi;
private MemoryDataSource mds;
- @Override
- protected SortedKeyValueIterator<Key,Value> getSource() {
- if (closed.get())
- throw new IllegalStateException("Memory iterator is closed");
- return super.getSource();
- }
-
private MemoryIterator(InterruptibleIterator source) {
this(source, new AtomicBoolean(false));
}
[2/2] accumulo git commit: ACCUMULO-3079 Only add qualifier filter if
needed ACCUMULO-3079 Optimize vis filtering when auths empty ACCUMULO-3079
Avoid WrapinIter.getSource() in loops
Posted by mm...@apache.org.
ACCUMULO-3079 Only add qualifier filter if needed
ACCUMULO-3079 Optimize vis filtering when auths empty
ACCUMULO-3079 Avoid WrapinIter.getSource() in loops
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/f81a8ec7
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/f81a8ec7
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/f81a8ec7
Branch: refs/heads/master
Commit: f81a8ec7410e789d11941351d5899b8894c6a322
Parents: b56a334
Author: Keith Turner <kt...@apache.org>
Authored: Mon Apr 10 13:42:33 2017 -0400
Committer: Mike Miller <mm...@apache.org>
Committed: Thu Apr 27 16:54:56 2017 -0400
----------------------------------------------------------------------
.../core/client/mock/MockScannerBase.java | 4 +-
.../apache/accumulo/core/iterators/Filter.java | 5 +-
.../core/iterators/FirstEntryInRowIterator.java | 11 ++--
.../accumulo/core/iterators/IteratorUtil.java | 4 +-
.../iterators/system/ColumnQualifierFilter.java | 32 ++++++-----
.../core/iterators/system/VisibilityFilter.java | 27 +++++++++-
.../core/iterators/user/VersioningIterator.java | 5 +-
.../core/iterators/system/ColumnFilterTest.java | 57 ++++++++++++++------
.../iterators/system/VisibilityFilterTest.java | 20 ++++++-
.../core/iterators/user/FilterTest.java | 12 ++---
.../user/TransformingIteratorTest.java | 2 +-
.../performance/scan/CollectTabletStats.java | 4 +-
12 files changed, 132 insertions(+), 51 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
index 8684697..6dd5737 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
@@ -133,8 +133,8 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase {
public SortedKeyValueIterator<Key,Value> createFilter(SortedKeyValueIterator<Key,Value> inner) throws IOException {
byte[] defaultLabels = {};
inner = new ColumnFamilySkippingIterator(new DeletingIterator(inner, false));
- ColumnQualifierFilter cqf = new ColumnQualifierFilter(inner, new HashSet<>(fetchedColumns));
- VisibilityFilter vf = new VisibilityFilter(cqf, auths, defaultLabels);
+ SortedKeyValueIterator<Key,Value> cqf = ColumnQualifierFilter.wrap(inner, new HashSet<>(fetchedColumns));
+ SortedKeyValueIterator<Key,Value> vf = VisibilityFilter.wrap(cqf, auths, defaultLabels);
AccumuloConfiguration conf = new MockConfiguration(table.settings);
MockIteratorEnvironment iterEnv = new MockIteratorEnvironment(auths);
SortedKeyValueIterator<Key,Value> result = iterEnv.getTopLevelIterator(IteratorUtil.loadIterators(IteratorScope.scan, vf, null, conf,
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java b/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
index 8b135c7..e2d13d6 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
@@ -69,9 +69,10 @@ public abstract class Filter extends WrappingIterator implements OptionDescriber
* Iterates over the source until an acceptable key/value pair is found.
*/
protected void findTop() {
- while (getSource().hasTop() && !getSource().getTopKey().isDeleted() && (negate == accept(getSource().getTopKey(), getSource().getTopValue()))) {
+ SortedKeyValueIterator<Key,Value> source = getSource();
+ while (source.hasTop() && !source.getTopKey().isDeleted() && (negate == accept(source.getTopKey(), source.getTopValue()))) {
try {
- getSource().next();
+ source.next();
} catch (IOException e) {
throw new RuntimeException(e);
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
index 17b8f43..e43b4f7 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/FirstEntryInRowIterator.java
@@ -79,27 +79,28 @@ public class FirstEntryInRowIterator extends SkippingIterator implements OptionD
if (finished || lastRowFound == null)
return;
int count = 0;
- while (getSource().hasTop() && lastRowFound.equals(getSource().getTopKey().getRow())) {
+ SortedKeyValueIterator<Key,Value> source = getSource();
+ while (source.hasTop() && lastRowFound.equals(source.getTopKey().getRow())) {
// try to efficiently jump to the next matching key
if (count < numscans) {
++count;
- getSource().next(); // scan
+ source.next(); // scan
} else {
// too many scans, just seek
count = 0;
// determine where to seek to, but don't go beyond the user-specified range
- Key nextKey = getSource().getTopKey().followingKey(PartialKey.ROW);
+ Key nextKey = source.getTopKey().followingKey(PartialKey.ROW);
if (!latestRange.afterEndKey(nextKey))
- getSource().seek(new Range(nextKey, true, latestRange.getEndKey(), latestRange.isEndKeyInclusive()), latestColumnFamilies, latestInclusive);
+ source.seek(new Range(nextKey, true, latestRange.getEndKey(), latestRange.isEndKeyInclusive()), latestColumnFamilies, latestInclusive);
else {
finished = true;
break;
}
}
}
- lastRowFound = getSource().hasTop() ? getSource().getTopKey().getRow(lastRowFound) : null;
+ lastRowFound = source.hasTop() ? source.getTopKey().getRow(lastRowFound) : null;
}
private boolean finished = true;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
----------------------------------------------------------------------
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 338da90..2e09782 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
@@ -395,7 +395,7 @@ public class IteratorUtil {
byte[] defaultVisibility) throws IOException {
DeletingIterator delIter = new DeletingIterator(source, false);
ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
- ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, cols);
- return new VisibilityFilter(colFilter, auths, defaultVisibility);
+ SortedKeyValueIterator<Key,Value> colFilter = ColumnQualifierFilter.wrap(cfsi, cols);
+ return VisibilityFilter.wrap(colFilter, auths, defaultVisibility);
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
index dbd9171..86fc2c8 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
@@ -30,11 +30,10 @@ import org.apache.accumulo.core.iterators.ServerFilter;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
public class ColumnQualifierFilter extends ServerFilter {
- private final boolean scanColumns;
private HashSet<ByteSequence> columnFamilies;
private HashMap<ByteSequence,HashSet<ByteSequence>> columnsQualifiers;
- public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, Set<Column> columns) {
+ private ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, Set<Column> columns) {
super(iterator);
this.columnFamilies = new HashSet<>();
this.columnsQualifiers = new HashMap<>();
@@ -54,24 +53,17 @@ public class ColumnQualifierFilter extends ServerFilter {
columnFamilies.add(new ArrayByteSequence(col.columnFamily));
}
}
-
- // only take action when column qualifies are present
- scanColumns = this.columnsQualifiers.size() > 0;
}
- public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, HashSet<ByteSequence> columnFamilies,
- HashMap<ByteSequence,HashSet<ByteSequence>> columnsQualifiers, boolean scanColumns) {
+ private ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, HashSet<ByteSequence> columnFamilies,
+ HashMap<ByteSequence,HashSet<ByteSequence>> columnsQualifiers) {
super(iterator);
this.columnFamilies = columnFamilies;
this.columnsQualifiers = columnsQualifiers;
- this.scanColumns = scanColumns;
}
@Override
public boolean accept(Key key, Value v) {
- if (!scanColumns)
- return true;
-
if (columnFamilies.contains(key.getColumnFamilyData()))
return true;
@@ -84,6 +76,22 @@ public class ColumnQualifierFilter extends ServerFilter {
@Override
public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new ColumnQualifierFilter(source.deepCopy(env), columnFamilies, columnsQualifiers, scanColumns);
+ return new ColumnQualifierFilter(source.deepCopy(env), columnFamilies, columnsQualifiers);
+ }
+
+ public static SortedKeyValueIterator<Key,Value> wrap(SortedKeyValueIterator<Key,Value> source, Set<Column> cols) {
+ boolean sawNonNullQual = false;
+ for (Column col : cols) {
+ if (col.getColumnQualifier() != null) {
+ sawNonNullQual = true;
+ break;
+ }
+ }
+
+ if (sawNonNullQual) {
+ return new ColumnQualifierFilter(source, cols);
+ } else {
+ return source;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
index b260bc4..6037233 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
@@ -46,7 +46,7 @@ public class VisibilityFilter extends SynchronizedServerFilter {
private static final Logger log = LoggerFactory.getLogger(VisibilityFilter.class);
- public VisibilityFilter(SortedKeyValueIterator<Key,Value> iterator, Authorizations authorizations, byte[] defaultVisibility) {
+ private VisibilityFilter(SortedKeyValueIterator<Key,Value> iterator, Authorizations authorizations, byte[] defaultVisibility) {
super(iterator);
this.ve = new VisibilityEvaluator(authorizations);
this.authorizations = authorizations;
@@ -84,4 +84,29 @@ public class VisibilityFilter extends SynchronizedServerFilter {
return false;
}
}
+
+ private static class EmptyAuthsVisibilityFilter extends SynchronizedServerFilter {
+
+ public EmptyAuthsVisibilityFilter(SortedKeyValueIterator<Key,Value> source) {
+ super(source);
+ }
+
+ @Override
+ public synchronized SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+ return new EmptyAuthsVisibilityFilter(source.deepCopy(env));
+ }
+
+ @Override
+ protected boolean accept(Key k, Value v) {
+ return k.getColumnVisibilityData().length() == 0;
+ }
+ }
+
+ public static SortedKeyValueIterator<Key,Value> wrap(SortedKeyValueIterator<Key,Value> source, Authorizations authorizations, byte[] defaultVisibility) {
+ if (authorizations.isEmpty() && defaultVisibility.length == 0) {
+ return new EmptyAuthsVisibilityFilter(source);
+ } else {
+ return new VisibilityFilter(source, authorizations, defaultVisibility);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java
index 88ba20d..3334d9f 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java
@@ -97,11 +97,12 @@ public class VersioningIterator extends WrappingIterator implements OptionDescri
super.next();
int count = 0;
- while (getSource().hasTop() && getSource().getTopKey().equals(keyToSkip, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) {
+ SortedKeyValueIterator<Key,Value> source = getSource();
+ while (source.hasTop() && source.getTopKey().equals(keyToSkip, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) {
if (count < maxCount) {
// it is quicker to call next if we are close, but we never know if we are close
// so give next a try a few times
- getSource().next();
+ source.next();
count++;
} else {
reseek(keyToSkip.followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/test/java/org/apache/accumulo/core/iterators/system/ColumnFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/system/ColumnFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/system/ColumnFilterTest.java
index 6158476..4d9d3ab 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/system/ColumnFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/system/ColumnFilterTest.java
@@ -16,14 +16,20 @@
*/
package org.apache.accumulo.core.iterators.system;
+import java.util.Collections;
import java.util.HashSet;
+import java.util.TreeMap;
import junit.framework.TestCase;
import org.apache.accumulo.core.data.Column;
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.hadoop.io.Text;
+import org.junit.Assert;
public class ColumnFilterTest extends TestCase {
@@ -40,39 +46,60 @@ public class ColumnFilterTest extends TestCase {
}
public void test1() {
- HashSet<Column> columns = new HashSet<>();
+ TreeMap<Key,Value> data = new TreeMap<Key,Value>();
+ data.put(newKey("r1", "cf1", "cq1"), new Value(""));
+ data.put(newKey("r1", "cf2", "cq1"), new Value(""));
+ HashSet<Column> columns = new HashSet<>();
columns.add(newColumn("cf1"));
- ColumnQualifierFilter cf = new ColumnQualifierFilter(null, columns);
-
- assertTrue(cf.accept(newKey("r1", "cf1", "cq1"), new Value(new byte[0])));
- assertTrue(cf.accept(newKey("r1", "cf2", "cq1"), new Value(new byte[0])));
+ SortedMapIterator smi = new SortedMapIterator(data);
+ SortedKeyValueIterator<Key,Value> cf = ColumnQualifierFilter.wrap(smi, columns);
+ Assert.assertSame(smi, cf);
}
- public void test2() {
+ public void test2() throws Exception {
+
+ TreeMap<Key,Value> data = new TreeMap<Key,Value>();
+ data.put(newKey("r1", "cf1", "cq1"), new Value(""));
+ data.put(newKey("r1", "cf2", "cq1"), new Value(""));
+ data.put(newKey("r1", "cf2", "cq2"), new Value(""));
+
HashSet<Column> columns = new HashSet<>();
columns.add(newColumn("cf1"));
columns.add(newColumn("cf2", "cq1"));
- ColumnQualifierFilter cf = new ColumnQualifierFilter(null, columns);
+ SortedKeyValueIterator<Key,Value> cf = ColumnQualifierFilter.wrap(new SortedMapIterator(data), columns);
+ cf.seek(new Range(), Collections.emptySet(), false);
- assertTrue(cf.accept(newKey("r1", "cf1", "cq1"), new Value(new byte[0])));
- assertTrue(cf.accept(newKey("r1", "cf2", "cq1"), new Value(new byte[0])));
- assertFalse(cf.accept(newKey("r1", "cf2", "cq2"), new Value(new byte[0])));
+ Assert.assertTrue(cf.hasTop());
+ Assert.assertEquals(newKey("r1", "cf1", "cq1"), cf.getTopKey());
+ cf.next();
+ Assert.assertTrue(cf.hasTop());
+ Assert.assertEquals(newKey("r1", "cf2", "cq1"), cf.getTopKey());
+ cf.next();
+ Assert.assertFalse(cf.hasTop());
}
- public void test3() {
+ public void test3() throws Exception {
+
+ TreeMap<Key,Value> data = new TreeMap<Key,Value>();
+ data.put(newKey("r1", "cf1", "cq1"), new Value(""));
+ data.put(newKey("r1", "cf2", "cq1"), new Value(""));
+ data.put(newKey("r1", "cf2", "cq2"), new Value(""));
+
HashSet<Column> columns = new HashSet<>();
columns.add(newColumn("cf2", "cq1"));
- ColumnQualifierFilter cf = new ColumnQualifierFilter(null, columns);
+ SortedKeyValueIterator<Key,Value> cf = ColumnQualifierFilter.wrap(new SortedMapIterator(data), columns);
+ cf.seek(new Range(), Collections.emptySet(), false);
- assertFalse(cf.accept(newKey("r1", "cf1", "cq1"), new Value(new byte[0])));
- assertTrue(cf.accept(newKey("r1", "cf2", "cq1"), new Value(new byte[0])));
- assertFalse(cf.accept(newKey("r1", "cf2", "cq2"), new Value(new byte[0])));
+ Assert.assertTrue(cf.hasTop());
+ Assert.assertEquals(newKey("r1", "cf2", "cq1"), cf.getTopKey());
+ cf.next();
+ Assert.assertFalse(cf.hasTop());
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/test/java/org/apache/accumulo/core/iterators/system/VisibilityFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/system/VisibilityFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/system/VisibilityFilterTest.java
index 68323c6..2e5ca04 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/system/VisibilityFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/system/VisibilityFilterTest.java
@@ -26,6 +26,7 @@ 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.security.Authorizations;
import org.apache.log4j.Level;
@@ -37,7 +38,7 @@ public class VisibilityFilterTest extends TestCase {
TreeMap<Key,Value> tm = new TreeMap<>();
tm.put(new Key("r1", "cf1", "cq1", "A&"), new Value(new byte[0]));
- VisibilityFilter filter = new VisibilityFilter(new SortedMapIterator(tm), new Authorizations("A"), "".getBytes());
+ SortedKeyValueIterator<Key,Value> filter = VisibilityFilter.wrap(new SortedMapIterator(tm), new Authorizations("A"), "".getBytes());
// suppress logging
Level prevLevel = Logger.getLogger(VisibilityFilter.class).getLevel();
@@ -49,4 +50,21 @@ public class VisibilityFilterTest extends TestCase {
Logger.getLogger(VisibilityFilter.class).setLevel(prevLevel);
}
+ public void testEmptyAuths() throws IOException {
+ TreeMap<Key,Value> tm = new TreeMap<>();
+
+ tm.put(new Key("r1", "cf1", "cq1", ""), new Value(new byte[0]));
+ tm.put(new Key("r1", "cf1", "cq2", "C"), new Value(new byte[0]));
+ tm.put(new Key("r1", "cf1", "cq3", ""), new Value(new byte[0]));
+ SortedKeyValueIterator<Key,Value> filter = VisibilityFilter.wrap(new SortedMapIterator(tm), Authorizations.EMPTY, "".getBytes());
+
+ filter.seek(new Range(), new HashSet<ByteSequence>(), false);
+ assertTrue(filter.hasTop());
+ assertEquals(new Key("r1", "cf1", "cq1", ""), filter.getTopKey());
+ filter.next();
+ assertTrue(filter.hasTop());
+ assertEquals(new Key("r1", "cf1", "cq3", ""), filter.getTopKey());
+ filter.next();
+ assertFalse(filter.hasTop());
+ }
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
index b26c218..244699a 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
@@ -358,19 +358,19 @@ public class FilterTest {
}
assertEquals(1000, tm.size());
- ColumnQualifierFilter a = new ColumnQualifierFilter(new SortedMapIterator(tm), hsc);
+ SortedKeyValueIterator<Key,Value> a = ColumnQualifierFilter.wrap(new SortedMapIterator(tm), hsc);
a.seek(new Range(), EMPTY_COL_FAMS, false);
assertEquals(1000, size(a));
hsc = new HashSet<>();
hsc.add(new Column("a".getBytes(), "b".getBytes(), null));
- a = new ColumnQualifierFilter(new SortedMapIterator(tm), hsc);
+ a = ColumnQualifierFilter.wrap(new SortedMapIterator(tm), hsc);
a.seek(new Range(), EMPTY_COL_FAMS, false);
int size = size(a);
assertEquals(500, size);
hsc = new HashSet<>();
- a = new ColumnQualifierFilter(new SortedMapIterator(tm), hsc);
+ a = ColumnQualifierFilter.wrap(new SortedMapIterator(tm), hsc);
a.seek(new Range(), EMPTY_COL_FAMS, false);
size = size(a);
assertEquals(1000, size);
@@ -394,20 +394,20 @@ public class FilterTest {
}
assertEquals(1000, tm.size());
- VisibilityFilter a = new VisibilityFilter(new SortedMapIterator(tm), auths, le2.getExpression());
+ SortedKeyValueIterator<Key,Value> a = VisibilityFilter.wrap(new SortedMapIterator(tm), auths, le2.getExpression());
a.seek(new Range(), EMPTY_COL_FAMS, false);
int size = size(a);
assertEquals(750, size);
}
- private ColumnQualifierFilter ncqf(TreeMap<Key,Value> tm, Column... columns) throws IOException {
+ private SortedKeyValueIterator<Key,Value> ncqf(TreeMap<Key,Value> tm, Column... columns) throws IOException {
HashSet<Column> hsc = new HashSet<>();
for (Column column : columns) {
hsc.add(column);
}
- ColumnQualifierFilter a = new ColumnQualifierFilter(new SortedMapIterator(tm), hsc);
+ SortedKeyValueIterator<Key,Value> a = ColumnQualifierFilter.wrap(new SortedMapIterator(tm), hsc);
a.seek(new Range(), EMPTY_COL_FAMS, false);
return a;
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
index d02b7f2..7ff08ec 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
@@ -81,7 +81,7 @@ public class TransformingIteratorTest {
private void setUpTransformIterator(Class<? extends TransformingIterator> clazz, boolean setupAuths) throws IOException {
SortedMapIterator source = new SortedMapIterator(data);
ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(source);
- VisibilityFilter visFilter = new VisibilityFilter(cfsi, authorizations, new byte[0]);
+ SortedKeyValueIterator<Key,Value> visFilter = VisibilityFilter.wrap(cfsi, authorizations, new byte[0]);
ReuseIterator reuserIter = new ReuseIterator();
reuserIter.init(visFilter, EMPTY_OPTS, null);
try {
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f81a8ec7/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
----------------------------------------------------------------------
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 d299aa0..f4c9078 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
@@ -431,8 +431,8 @@ public class CollectTabletStats {
MultiIterator multiIter = new MultiIterator(iters, ke);
DeletingIterator delIter = new DeletingIterator(multiIter, false);
ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
- ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, columnSet);
- VisibilityFilter visFilter = new VisibilityFilter(colFilter, authorizations, defaultLabels);
+ SortedKeyValueIterator<Key,Value> colFilter = ColumnQualifierFilter.wrap(cfsi, columnSet);
+ SortedKeyValueIterator<Key,Value> visFilter = VisibilityFilter.wrap(colFilter, authorizations, defaultLabels);
if (useTableIterators)
return IteratorUtil.loadIterators(IteratorScope.scan, visFilter, ke, conf, ssiList, ssio, null);