accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ctubb...@apache.org
Subject [19/61] [abbrv] [partial] accumulo git commit: ACCUMULO-722 put trunk in my sandbox
Date Thu, 03 Mar 2016 21:59:44 GMT
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java
new file mode 100644
index 0000000..701bf66
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     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.conf;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.iterators.conf.ColumnUtil.ColFamHashKey;
+import org.apache.accumulo.core.iterators.conf.ColumnUtil.ColHashKey;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.io.Text;
+
+public class ColumnSet {
+  private Set<ColFamHashKey> objectsCF;
+  private Set<ColHashKey> objectsCol;
+  
+  private ColHashKey lookupCol = new ColHashKey();
+  private ColFamHashKey lookupCF = new ColFamHashKey();
+  
+  public ColumnSet() {
+    objectsCF = new HashSet<ColFamHashKey>();
+    objectsCol = new HashSet<ColHashKey>();
+  }
+  
+  public ColumnSet(Collection<String> objectStrings) {
+    this();
+    
+    for (String column : objectStrings) {
+      Pair<Text,Text> pcic = ColumnSet.decodeColumns(column);
+      
+      if (pcic.getSecond() == null) {
+        add(pcic.getFirst());
+      } else {
+        add(pcic.getFirst(), pcic.getSecond());
+      }
+    }
+  }
+  
+  protected void add(Text colf) {
+    objectsCF.add(new ColFamHashKey(new Text(colf)));
+  }
+  
+  protected void add(Text colf, Text colq) {
+    objectsCol.add(new ColHashKey(colf, colq));
+  }
+  
+  public boolean contains(Key key) {
+    // lookup column family and column qualifier
+    if (objectsCol.size() > 0) {
+      lookupCol.set(key);
+      if (objectsCol.contains(lookupCol))
+        return true;
+    }
+    
+    // lookup just column family
+    if (objectsCF.size() > 0) {
+      lookupCF.set(key);
+      return objectsCF.contains(lookupCF);
+    }
+    
+    return false;
+  }
+  
+  public boolean isEmpty() {
+    return objectsCol.size() == 0 && objectsCF.size() == 0;
+  }
+
+  public static String encodeColumns(Text columnFamily, Text columnQualifier) {
+    StringBuilder sb = new StringBuilder();
+    
+    encode(sb, columnFamily);
+    if (columnQualifier != null) {
+      sb.append(':');
+      encode(sb, columnQualifier);
+    }
+    
+    return sb.toString();
+  }
+
+  static void encode(StringBuilder sb, Text t) {
+    for (int i = 0; i < t.getLength(); i++) {
+      int b = (0xff & t.getBytes()[i]);
+      
+      // very inefficient code
+      if ((b >= 'a' && b <= 'z') || (b >= 'A' && b <= 'Z') || (b >= '0' && b <= '9') || b == '_' || b == '-') {
+        sb.append((char) b);
+      } else {
+        sb.append('%');
+        sb.append(String.format("%02x", b));
+      }
+    }
+  }
+
+  public static boolean isValidEncoding(String enc) {
+    for (char c : enc.toCharArray()) {
+      boolean validChar = (c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || (c >= '0' && c <= '9') || c == '_' || c == '-' || c == ':' || c == '%';
+      if (!validChar)
+        return false;
+    }
+    
+    return true;
+  }
+
+  public static Pair<Text,Text> decodeColumns(String columns) {
+    if (!isValidEncoding(columns))
+      throw new IllegalArgumentException("Invalid encoding " + columns);
+  
+    String[] cols = columns.split(":");
+    
+    if (cols.length == 1) {
+      return new Pair<Text,Text>(decode(cols[0]), null);
+    } else if (cols.length == 2) {
+      return new Pair<Text,Text>(decode(cols[0]), decode(cols[1]));
+    } else {
+      throw new IllegalArgumentException(columns);
+    }
+  }
+
+  static Text decode(String s) {
+    Text t = new Text();
+    
+    byte[] sb = s.getBytes();
+    
+    // very inefficient code
+    for (int i = 0; i < sb.length; i++) {
+      if (sb[i] != '%') {
+        t.append(new byte[] {sb[i]}, 0, 1);
+      } else {
+        byte hex[] = new byte[] {sb[++i], sb[++i]};
+        String hs = new String(hex);
+        int b = Integer.parseInt(hs, 16);
+        t.append(new byte[] {(byte) b}, 0, 1);
+      }
+    }
+    
+    return t;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java
new file mode 100644
index 0000000..a9d59f8
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java
@@ -0,0 +1,95 @@
+/*
+ * 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.conf;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.iterators.conf.ColumnUtil.ColFamHashKey;
+import org.apache.accumulo.core.iterators.conf.ColumnUtil.ColHashKey;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.start.classloader.AccumuloClassLoader;
+import org.apache.hadoop.io.Text;
+
+public class ColumnToClassMapping<K> {
+  
+  private HashMap<ColFamHashKey,K> objectsCF;
+  private HashMap<ColHashKey,K> objectsCol;
+  
+  private ColHashKey lookupCol = new ColHashKey();
+  private ColFamHashKey lookupCF = new ColFamHashKey();
+  
+  public ColumnToClassMapping() {
+    objectsCF = new HashMap<ColFamHashKey,K>();
+    objectsCol = new HashMap<ColHashKey,K>();
+  }
+  
+  public ColumnToClassMapping(Map<String,String> objectStrings, Class<? extends K> c) throws InstantiationException, IllegalAccessException,
+      ClassNotFoundException {
+    this();
+    
+    for (Entry<String,String> entry : objectStrings.entrySet()) {
+      String column = entry.getKey();
+      String className = entry.getValue();
+      
+      Pair<Text,Text> pcic = ColumnSet.decodeColumns(column);
+      
+      Class<? extends K> clazz = AccumuloClassLoader.loadClass(className, c);
+      
+      if (pcic.getSecond() == null) {
+        addObject(pcic.getFirst(), clazz.newInstance());
+      } else {
+        addObject(pcic.getFirst(), pcic.getSecond(), clazz.newInstance());
+      }
+    }
+  }
+  
+  protected void addObject(Text colf, K obj) {
+    objectsCF.put(new ColFamHashKey(new Text(colf)), obj);
+  }
+  
+  protected void addObject(Text colf, Text colq, K obj) {
+    objectsCol.put(new ColHashKey(colf, colq), obj);
+  }
+  
+  public K getObject(Key key) {
+    K obj = null;
+    
+    // lookup column family and column qualifier
+    if (objectsCol.size() > 0) {
+      lookupCol.set(key);
+      obj = objectsCol.get(lookupCol);
+      if (obj != null) {
+        return obj;
+      }
+    }
+    
+    // lookup just column family
+    if (objectsCF.size() > 0) {
+      lookupCF.set(key);
+      obj = objectsCF.get(lookupCF);
+    }
+    
+    return obj;
+  }
+  
+  public boolean isEmpty() {
+    return objectsCol.size() == 0 && objectsCF.size() == 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnUtil.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnUtil.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnUtil.java
new file mode 100644
index 0000000..984f069
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnUtil.java
@@ -0,0 +1,116 @@
+/*
+ * 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.conf;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.hadoop.io.Text;
+
+public class ColumnUtil {
+  private static int hash(byte[] bytes, int offset, int len) {
+    int hash = 1;
+    int end = offset + len;
+    
+    for (int i = offset; i < end; i++)
+      hash = (31 * hash) + bytes[i];
+    
+    return hash;
+  }
+  
+  private static int hash(ByteSequence bs) {
+    return hash(bs.getBackingArray(), bs.offset(), bs.length());
+  }
+  
+  public static class ColFamHashKey {
+    Text columnFamily;
+    
+    Key key;
+    
+    private int hashCode;
+    
+    ColFamHashKey() {
+      columnFamily = null;
+    }
+    
+    ColFamHashKey(Text cf) {
+      columnFamily = cf;
+      hashCode = hash(columnFamily.getBytes(), 0, columnFamily.getLength());
+    }
+    
+    void set(Key key) {
+      this.key = key;
+      hashCode = hash(key.getColumnFamilyData());
+    }
+    
+    public int hashCode() {
+      return hashCode;
+    }
+    
+    public boolean equals(Object o) {
+      if (o instanceof ColFamHashKey)
+        return equals((ColFamHashKey) o);
+      return false;
+    }
+    
+    public boolean equals(ColFamHashKey ohk) {
+      if (columnFamily == null)
+        return key.compareColumnFamily(ohk.columnFamily) == 0;
+      return ohk.key.compareColumnFamily(columnFamily) == 0;
+    }
+  }
+  
+  public static class ColHashKey {
+    Text columnFamily;
+    Text columnQualifier;
+    
+    Key key;
+    
+    private int hashValue;
+    
+    ColHashKey() {
+      columnFamily = null;
+      columnQualifier = null;
+    }
+    
+    ColHashKey(Text cf, Text cq) {
+      columnFamily = cf;
+      columnQualifier = cq;
+      hashValue = hash(columnFamily.getBytes(), 0, columnFamily.getLength()) + hash(columnQualifier.getBytes(), 0, columnQualifier.getLength());
+    }
+    
+    void set(Key key) {
+      this.key = key;
+      hashValue = hash(key.getColumnFamilyData()) + hash(key.getColumnQualifierData());
+    }
+    
+    public int hashCode() {
+      return hashValue;
+    }
+    
+    public boolean equals(Object o) {
+      if (o instanceof ColHashKey)
+        return equals((ColHashKey) o);
+      return false;
+    }
+    
+    public boolean equals(ColHashKey ohk) {
+      if (columnFamily == null)
+        return key.compareColumnFamily(ohk.columnFamily) == 0 && key.compareColumnQualifier(ohk.columnQualifier) == 0;
+      return ohk.key.compareColumnFamily(columnFamily) == 0 && ohk.key.compareColumnQualifier(columnQualifier) == 0;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java
new file mode 100644
index 0000000..6b90d22
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java
@@ -0,0 +1,80 @@
+/*
+ * 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.conf;
+
+import org.apache.hadoop.io.Text;
+
+/**
+ * @deprecated since 1.4
+ * 
+ * @see org.apache.accumulo.core.client.IteratorSetting.Column
+ * @see org.apache.accumulo.core.iterators.Combiner#setColumns(org.apache.accumulo.core.client.IteratorSetting, java.util.List)
+ */
+public class PerColumnIteratorConfig {
+  
+  private String parameter;
+  private Text colq;
+  private Text colf;
+  
+  public PerColumnIteratorConfig(Text columnFamily, String parameter) {
+    this.colf = columnFamily;
+    this.colq = null;
+    this.parameter = parameter;
+  }
+  
+  public PerColumnIteratorConfig(Text columnFamily, Text columnQualifier, String parameter) {
+    this.colf = columnFamily;
+    this.colq = columnQualifier;
+    this.parameter = parameter;
+  }
+  
+  public Text getColumnFamily() {
+    return colf;
+  }
+  
+  public Text getColumnQualifier() {
+    return colq;
+  }
+  
+  public String encodeColumns() {
+    return encodeColumns(this);
+  }
+  
+  public String getClassName() {
+    return parameter;
+  }
+  
+  private static String encodeColumns(PerColumnIteratorConfig pcic) {
+    return ColumnSet.encodeColumns(pcic.colf, pcic.colq);
+  }
+  
+  public static String encodeColumns(Text columnFamily, Text columnQualifier) {
+    return ColumnSet.encodeColumns(columnFamily, columnQualifier);
+  }
+
+  public static PerColumnIteratorConfig decodeColumns(String columns, String className) {
+    String[] cols = columns.split(":");
+    
+    if (cols.length == 1) {
+      return new PerColumnIteratorConfig(ColumnSet.decode(cols[0]), className);
+    } else if (cols.length == 2) {
+      return new PerColumnIteratorConfig(ColumnSet.decode(cols[0]), ColumnSet.decode(cols[1]), className);
+    } else {
+      throw new IllegalArgumentException(columns);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java
new file mode 100644
index 0000000..7df57c5
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java
@@ -0,0 +1,131 @@
+/*
+ * 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.system;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SkippingIterator;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+
+public class ColumnFamilySkippingIterator extends SkippingIterator implements InterruptibleIterator {
+  
+  protected Set<ByteSequence> colFamSet = null;
+  protected TreeSet<ByteSequence> sortedColFams = null;
+  
+  protected boolean inclusive = false;
+  protected Range range;
+  
+  public ColumnFamilySkippingIterator(SortedKeyValueIterator<Key,Value> source) {
+    this.setSource(source);
+  }
+  
+  protected ColumnFamilySkippingIterator(SortedKeyValueIterator<Key,Value> source, Set<ByteSequence> colFamSet, boolean inclusive) {
+    this(source);
+    this.colFamSet = colFamSet;
+    this.inclusive = inclusive;
+  }
+  
+  @Override
+  protected void consume() throws IOException {
+    int count = 0;
+    
+    if (inclusive)
+      while (getSource().hasTop() && !colFamSet.contains(getSource().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();
+          count++;
+        } else {
+          ByteSequence higherCF = sortedColFams.higher(getSource().getTopKey().getColumnFamilyData());
+          if (higherCF == null) {
+            // seek to the next row
+            reseek(getSource().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));
+          }
+          
+          count = 0;
+        }
+      }
+    else if (colFamSet != null && colFamSet.size() > 0)
+      while (getSource().hasTop() && colFamSet.contains(getSource().getTopKey().getColumnFamilyData())) {
+        if (count < 10) {
+          getSource().next();
+          count++;
+        } else {
+          // seek to the next column family in the data
+          reseek(getSource().getTopKey().followingKey(PartialKey.ROW_COLFAM));
+          count = 0;
+        }
+      }
+  }
+  
+  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);
+    } else {
+      range = new Range(key, true, range.getEndKey(), range.isEndKeyInclusive());
+      getSource().seek(range, colFamSet, inclusive);
+    }
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    return new ColumnFamilySkippingIterator(getSource().deepCopy(env), colFamSet, inclusive);
+  }
+  
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    
+    if (columnFamilies instanceof Set<?>) {
+      colFamSet = (Set<ByteSequence>) columnFamilies;
+    } else {
+      colFamSet = new HashSet<ByteSequence>();
+      colFamSet.addAll(columnFamilies);
+    }
+    
+    if (inclusive) {
+      sortedColFams = new TreeSet<ByteSequence>(colFamSet);
+    } else {
+      sortedColFams = null;
+    }
+    
+    this.range = range;
+    this.inclusive = inclusive;
+    super.seek(range, colFamSet, inclusive);
+  }
+  
+  @Override
+  public void setInterruptFlag(AtomicBoolean flag) {
+    ((InterruptibleIterator) getSource()).setInterruptFlag(flag);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
new file mode 100644
index 0000000..1595f5a
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
@@ -0,0 +1,95 @@
+/*
+ * 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.system;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+
+import org.apache.accumulo.core.data.ArrayByteSequence;
+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.SortedKeyValueIterator;
+
+public class ColumnQualifierFilter extends Filter {
+  private boolean scanColumns;
+  private HashSet<ByteSequence> columnFamilies;
+  private HashMap<ByteSequence,HashSet<ByteSequence>> columnsQualifiers;
+  
+  public ColumnQualifierFilter() {}
+  
+  public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, HashSet<Column> columns) {
+    setSource(iterator);
+    init(columns);
+  }
+  
+  public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, HashSet<ByteSequence> columnFamilies,
+      HashMap<ByteSequence,HashSet<ByteSequence>> columnsQualifiers, boolean scanColumns) {
+    setSource(iterator);
+    this.columnFamilies = columnFamilies;
+    this.columnsQualifiers = columnsQualifiers;
+    this.scanColumns = scanColumns;
+  }
+  
+  public boolean accept(Key key, Value v) {
+    if (!scanColumns)
+      return true;
+    
+    if (columnFamilies.contains(key.getColumnFamilyData()))
+      return true;
+    
+    HashSet<ByteSequence> cfset = columnsQualifiers.get(key.getColumnQualifierData());
+    // ensure the columm qualifier goes with a paired column family,
+    // it is possible that a column qualifier could occur with a
+    // column family it was not paired with
+    return cfset != null && cfset.contains(key.getColumnFamilyData());
+  }
+  
+  public void init(HashSet<Column> columns) {
+    this.columnFamilies = new HashSet<ByteSequence>();
+    this.columnsQualifiers = new HashMap<ByteSequence,HashSet<ByteSequence>>();
+    
+    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<ByteSequence>();
+          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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/CountingIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/CountingIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/CountingIterator.java
new file mode 100644
index 0000000..010136a
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/CountingIterator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.system;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+
+public class CountingIterator extends WrappingIterator {
+  
+  private long count;
+  
+  public CountingIterator deepCopy(IteratorEnvironment env) {
+    return new CountingIterator(this, env);
+  }
+  
+  private CountingIterator(CountingIterator other, IteratorEnvironment env) {
+    setSource(other.getSource().deepCopy(env));
+    count = 0;
+  }
+  
+  public CountingIterator(SortedKeyValueIterator<Key,Value> source) {
+    this.setSource(source);
+    count = 0;
+  }
+  
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void next() throws IOException {
+    super.next();
+    count++;
+  }
+  
+  public long getCount() {
+    return count;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java
new file mode 100644
index 0000000..e770351
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/DeletingIterator.java
@@ -0,0 +1,104 @@
+/*
+ * 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.system;
+
+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.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.IteratorUtil;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+
+public class DeletingIterator extends WrappingIterator {
+  private boolean propogateDeletes;
+  private Key workKey = new Key();
+  
+  public DeletingIterator deepCopy(IteratorEnvironment env) {
+    return new DeletingIterator(this, env);
+  }
+  
+  public DeletingIterator(DeletingIterator other, IteratorEnvironment env) {
+    setSource(other.getSource().deepCopy(env));
+    propogateDeletes = other.propogateDeletes;
+  }
+  
+  public DeletingIterator() {}
+  
+  public DeletingIterator(SortedKeyValueIterator<Key,Value> iterator, boolean propogateDeletes) throws IOException {
+    this.setSource(iterator);
+    this.propogateDeletes = propogateDeletes;
+  }
+  
+  @Override
+  public void next() throws IOException {
+    if (super.getTopKey().isDeleted())
+      skipRowColumn();
+    else
+      getSource().next();
+    findTop();
+  }
+  
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    // do not want to seek to the middle of a row
+    Range seekRange = IteratorUtil.maximizeStartKeyTimeStamp(range);
+    
+    super.seek(seekRange, columnFamilies, inclusive);
+    findTop();
+    
+    if (range.getStartKey() != null) {
+      while (getSource().hasTop() && getSource().getTopKey().compareTo(range.getStartKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME) < 0) {
+        next();
+      }
+      
+      while (hasTop() && range.beforeStartKey(getTopKey())) {
+        next();
+      }
+    }
+  }
+  
+  private void findTop() throws IOException {
+    if (!propogateDeletes) {
+      while (getSource().hasTop() && getSource().getTopKey().isDeleted()) {
+        skipRowColumn();
+      }
+    }
+  }
+  
+  private void skipRowColumn() throws IOException {
+    workKey.set(getSource().getTopKey());
+    
+    Key keyToSkip = workKey;
+    getSource().next();
+    
+    while (getSource().hasTop() && getSource().getTopKey().equals(keyToSkip, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) {
+      getSource().next();
+    }
+  }
+  
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/HeapIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/HeapIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/HeapIterator.java
new file mode 100644
index 0000000..e54f37c
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/HeapIterator.java
@@ -0,0 +1,115 @@
+/*
+ * 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.system;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.commons.collections.buffer.PriorityBuffer;
+
+public abstract class HeapIterator implements SortedKeyValueIterator<Key,Value> {
+  private PriorityBuffer heap;
+  private SortedKeyValueIterator<Key,Value> currentIter;
+  
+  private static class Index implements Comparable<Index> {
+    SortedKeyValueIterator<Key,Value> iter;
+    
+    public Index(SortedKeyValueIterator<Key,Value> iter) {
+      this.iter = iter;
+    }
+    
+    public int compareTo(Index o) {
+      return iter.getTopKey().compareTo(o.iter.getTopKey());
+    }
+  }
+  
+  protected HeapIterator() {
+    heap = null;
+  }
+  
+  protected HeapIterator(int maxSize) {
+    createHeap(maxSize);
+  }
+  
+  protected void createHeap(int maxSize) {
+    if (heap != null)
+      throw new IllegalStateException("heap already exist");
+    
+    heap = new PriorityBuffer(maxSize == 0 ? 1 : maxSize);
+  }
+  
+  @Override
+  final public Key getTopKey() {
+    return currentIter.getTopKey();
+  }
+  
+  @Override
+  final public Value getTopValue() {
+    return currentIter.getTopValue();
+  }
+  
+  @Override
+  final public boolean hasTop() {
+    return heap.size() > 0;
+  }
+  
+  @Override
+  final public void next() throws IOException {
+    switch (heap.size()) {
+      case 0:
+        throw new IllegalStateException("Called next() when there is no top");
+      case 1:
+        // optimization for case when heap contains one entry,
+        // avoids remove and add
+        currentIter.next();
+        if (!currentIter.hasTop()) {
+          heap.remove();
+          currentIter = null;
+        }
+        break;
+      default:
+        Index idx = (Index) heap.remove();
+        idx.iter.next();
+        if (idx.iter.hasTop()) {
+          heap.add(idx);
+        }
+        // to get to the default case heap has at least
+        // two entries, therefore there must be at least
+        // one entry when get() is called below
+        currentIter = ((Index) heap.get()).iter;
+    }
+  }
+  
+  final protected void clear() {
+    heap.clear();
+    currentIter = null;
+  }
+  
+  final protected void addSource(SortedKeyValueIterator<Key,Value> source) {
+    
+    if (source.hasTop())
+      heap.add(new Index(source));
+    
+    if (heap.size() > 0)
+      currentIter = ((Index) heap.get()).iter;
+    else
+      currentIter = null;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/InterruptibleIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/InterruptibleIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/InterruptibleIterator.java
new file mode 100644
index 0000000..08a86b2
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/InterruptibleIterator.java
@@ -0,0 +1,27 @@
+/*
+ * 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.system;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+
+public interface InterruptibleIterator extends SortedKeyValueIterator<Key,Value> {
+  public void setInterruptFlag(AtomicBoolean flag);
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/MapFileIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/MapFileIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/MapFileIterator.java
new file mode 100644
index 0000000..e5fe62a
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/MapFileIterator.java
@@ -0,0 +1,162 @@
+/*
+ * 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.system;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+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.file.FileSKVIterator;
+import org.apache.accumulo.core.file.NoSuchMetaStoreException;
+import org.apache.accumulo.core.file.map.MapFileUtil;
+import org.apache.accumulo.core.iterators.IterationInterruptedException;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.MapFile.Reader;
+import org.apache.log4j.Logger;
+
+public class MapFileIterator implements FileSKVIterator {
+  private static final Logger log = Logger.getLogger(MapFileIterator.class);
+
+  private Reader reader;
+  private Value topValue;
+  private Key topKey;
+  private AtomicBoolean interruptFlag;
+  private int interruptCheckCount = 0;
+  private FileSystem fs;
+  private String dirName;
+  
+  /**
+   * @param acuconf
+   * @param fs
+   * @param dir
+   * @param conf
+   * @throws IOException
+   */
+  public MapFileIterator(AccumuloConfiguration acuconf, FileSystem fs, String dir, Configuration conf) throws IOException {
+    this.reader = MapFileUtil.openMapFile(acuconf, fs, dir, conf);
+    this.fs = fs;
+    this.dirName = dir;
+  }
+
+  @Override
+  public void setInterruptFlag(AtomicBoolean flag) {
+    this.interruptFlag = flag;
+  }
+  
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public boolean hasTop() {
+    return topKey != null;
+  }
+  
+  @Override
+  public void next() throws IOException {
+    if (interruptFlag != null && interruptCheckCount++ % 100 == 0 && interruptFlag.get())
+      throw new IterationInterruptedException();
+    
+    reader.next(topKey, topValue);
+  }
+  
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    if (columnFamilies.size() != 0 || inclusive) {
+      throw new IllegalArgumentException("I do not know how to filter column families");
+    }
+    
+    if (range == null)
+      throw new IllegalArgumentException("Cannot seek to null range");
+    
+    if (interruptFlag != null && interruptFlag.get())
+      throw new IterationInterruptedException();
+    
+    Key key = range.getStartKey();
+    if (key == null) {
+      key = new Key();
+    }
+    
+    reader.seek(key);
+    
+    while (hasTop() && range.beforeStartKey(getTopKey())) {
+      next();
+    }
+  }
+  
+  @Override
+  public Key getTopKey() {
+    return topKey;
+  }
+  
+  @Override
+  public Value getTopValue() {
+    return topValue;
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    try {
+      SortedKeyValueIterator<Key,Value> other = env.reserveMapFileReader(dirName);
+      ((InterruptibleIterator) other).setInterruptFlag(interruptFlag);
+      log.debug("deep copying MapFile: " + this + " -> " + other);
+      return other;
+    } catch (IOException e) {
+      log.error("failed to clone map file reader", e);
+      throw new RuntimeException(e);
+    }
+  }
+  
+  @Override
+  public Key getFirstKey() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public Key getLastKey() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public DataInputStream getMetaStore(String name) throws IOException {
+    Path path = new Path(this.dirName, name);
+    if (!fs.exists(path))
+      throw new NoSuchMetaStoreException("name = " + name);
+    return fs.open(path);
+  }
+  
+  @Override
+  public void closeDeepCopies() throws IOException {
+    // nothing to do, deep copies are externally managed/closed
+  }
+  
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java
new file mode 100644
index 0000000..f406fee
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java
@@ -0,0 +1,114 @@
+/*
+ * 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.system;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+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.SortedKeyValueIterator;
+
+/**
+ * An iterator capable of iterating over other iterators in sorted order.
+ * 
+ * 
+ * 
+ */
+
+public class MultiIterator extends HeapIterator {
+  
+  private List<SortedKeyValueIterator<Key,Value>> iters;
+  private Range fence;
+  
+  // deep copy with no seek/scan state
+  public MultiIterator deepCopy(IteratorEnvironment env) {
+    return new MultiIterator(this, env);
+  }
+  
+  private MultiIterator(MultiIterator other, IteratorEnvironment env) {
+    super(other.iters.size());
+    this.iters = new ArrayList<SortedKeyValueIterator<Key,Value>>();
+    this.fence = other.fence;
+    for (SortedKeyValueIterator<Key,Value> iter : other.iters) {
+      iters.add(iter.deepCopy(env));
+    }
+  }
+  
+  private void init() {
+    for (SortedKeyValueIterator<Key,Value> skvi : iters)
+      addSource(skvi);
+  }
+  
+  private MultiIterator(List<SortedKeyValueIterator<Key,Value>> iters, Range seekFence, boolean init) {
+    super(iters.size());
+    
+    if (seekFence != null && init) {
+      // throw this exception because multi-iterator does not seek on init, therefore the
+      // fence would not be enforced in anyway, so do not want to give the impression it
+      // will enforce this
+      throw new IllegalArgumentException("Initializing not supported when seek fence set");
+    }
+    
+    this.fence = seekFence;
+    this.iters = iters;
+    
+    if (init) {
+      init();
+    }
+  }
+  
+  public MultiIterator(List<SortedKeyValueIterator<Key,Value>> iters, Range seekFence) {
+    this(iters, seekFence, false);
+  }
+  
+  public MultiIterator(List<SortedKeyValueIterator<Key,Value>> iters2, KeyExtent extent) {
+    this(iters2, new Range(extent.getPrevEndRow(), false, extent.getEndRow(), true), false);
+  }
+  
+  public MultiIterator(List<SortedKeyValueIterator<Key,Value>> readers, boolean init) {
+    this(readers, (Range) null, init);
+  }
+  
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    clear();
+    
+    if (fence != null) {
+      range = fence.clip(range, true);
+      if (range == null)
+        return;
+    }
+    
+    for (SortedKeyValueIterator<Key,Value> skvi : iters) {
+      skvi.seek(range, columnFamilies, inclusive);
+      addSource(skvi);
+    }
+  }
+  
+  @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/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SequenceFileIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SequenceFileIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SequenceFileIterator.java
new file mode 100644
index 0000000..f593ee2
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SequenceFileIterator.java
@@ -0,0 +1,123 @@
+/*
+ * 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.system;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+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.file.FileSKVIterator;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+
+public class SequenceFileIterator implements FileSKVIterator {
+  
+  private Reader reader;
+  private Value top_value;
+  private Key top_key;
+  private boolean readValue;
+  
+  public SequenceFileIterator deepCopy(IteratorEnvironment env) {
+    throw new UnsupportedOperationException("SequenceFileIterator does not yet support cloning");
+  }
+  
+  @Override
+  public void closeDeepCopies() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  public SequenceFileIterator(SequenceFile.Reader reader, boolean readValue) throws IOException {
+    this.reader = reader;
+    this.readValue = readValue;
+    
+    top_key = new Key();
+    
+    if (readValue)
+      top_value = new Value();
+    
+    next();
+  }
+  
+  public Key getTopKey() {
+    return top_key;
+  }
+  
+  public Value getTopValue() {
+    return top_value;
+  }
+  
+  public boolean hasTop() {
+    return top_key != null;
+  }
+  
+  public void next() throws IOException {
+    boolean valid;
+    if (readValue)
+      valid = reader.next(top_key, top_value);
+    else
+      valid = reader.next(top_key);
+    
+    if (!valid) {
+      top_key = null;
+      top_value = null;
+    }
+    
+  }
+  
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    throw new UnsupportedOperationException("seek() not supported");
+  }
+  
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+    throw new UnsupportedOperationException();
+    
+  }
+  
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+  
+  @Override
+  public Key getFirstKey() throws IOException {
+    throw new UnsupportedOperationException("getFirstKey() not supported");
+  }
+  
+  @Override
+  public Key getLastKey() throws IOException {
+    throw new UnsupportedOperationException("getLastKey() not supported");
+  }
+  
+  @Override
+  public DataInputStream getMetaStore(String name) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void setInterruptFlag(AtomicBoolean flag) {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java
new file mode 100644
index 0000000..b7069c9
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java
@@ -0,0 +1,200 @@
+/*
+ * 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.system;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+
+public class SourceSwitchingIterator implements SortedKeyValueIterator<Key,Value>, InterruptibleIterator {
+  
+  public interface DataSource {
+    boolean isCurrent();
+    
+    DataSource getNewDataSource();
+    
+    DataSource getDeepCopyDataSource(IteratorEnvironment env);
+    
+    SortedKeyValueIterator<Key,Value> iterator() throws IOException;
+  }
+  
+  private DataSource source;
+  private SortedKeyValueIterator<Key,Value> iter;
+  
+  private Key key;
+  private Value val;
+  
+  private Range range;
+  private boolean inclusive;
+  private Collection<ByteSequence> columnFamilies;
+  
+  private boolean onlySwitchAfterRow;
+  private AtomicBoolean iflag;
+  
+  private List<SourceSwitchingIterator> copies;
+  
+  private SourceSwitchingIterator(DataSource source, boolean onlySwitchAfterRow, List<SourceSwitchingIterator> copies, AtomicBoolean iflag) {
+    this.source = source;
+    this.onlySwitchAfterRow = onlySwitchAfterRow;
+    this.copies = copies;
+    this.iflag = iflag;
+    copies.add(this);
+  }
+  
+  public SourceSwitchingIterator(DataSource source, boolean onlySwitchAfterRow) {
+    this(source, onlySwitchAfterRow, Collections.synchronizedList(new ArrayList<SourceSwitchingIterator>()), null);
+  }
+  
+  public SourceSwitchingIterator(DataSource source) {
+    this(source, false);
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    return new SourceSwitchingIterator(source.getDeepCopyDataSource(env), onlySwitchAfterRow, copies, iflag);
+  }
+  
+  @Override
+  public Key getTopKey() {
+    return key;
+  }
+  
+  @Override
+  public Value getTopValue() {
+    return val;
+  }
+  
+  @Override
+  public boolean hasTop() {
+    return key != null;
+  }
+  
+  @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 {
+    readNext(false);
+  }
+  
+  private synchronized void readNext(boolean initialSeek) throws IOException {
+    
+    // check of initialSeek second is intentional so that it does not short
+    // circuit the call to switchSource
+    boolean seekNeeded = (!onlySwitchAfterRow && switchSource()) || initialSeek;
+    
+    if (seekNeeded)
+      if (initialSeek)
+        iter.seek(range, columnFamilies, inclusive);
+      else
+        iter.seek(new Range(key, false, range.getEndKey(), range.isEndKeyInclusive()), columnFamilies, inclusive);
+    else {
+      iter.next();
+      if (onlySwitchAfterRow && iter.hasTop() && !source.isCurrent() && !key.getRowData().equals(iter.getTopKey().getRowData())) {
+        switchSource();
+        iter.seek(new Range(key.followingKey(PartialKey.ROW), true, range.getEndKey(), range.isEndKeyInclusive()), columnFamilies, inclusive);
+      }
+    }
+    
+    if (iter.hasTop()) {
+      Key nextKey = iter.getTopKey();
+      Value nextVal = iter.getTopValue();
+      
+      try {
+        key = (Key) nextKey.clone();
+      } catch (CloneNotSupportedException e) {
+        throw new IOException(e);
+      }
+      val = nextVal;
+    } else {
+      key = null;
+      val = null;
+    }
+  }
+  
+  private boolean switchSource() throws IOException {
+    while (!source.isCurrent()) {
+      source = source.getNewDataSource();
+      iter = source.iterator();
+      if (iflag != null)
+        ((InterruptibleIterator) iter).setInterruptFlag(iflag);
+      
+      return true;
+    }
+    
+    return false;
+  }
+  
+  @Override
+  public synchronized void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    this.range = range;
+    this.inclusive = inclusive;
+    this.columnFamilies = columnFamilies;
+    
+    if (iter == null) {
+      iter = source.iterator();
+      if (iflag != null)
+        ((InterruptibleIterator) iter).setInterruptFlag(iflag);
+    }
+    
+    readNext(true);
+  }
+  
+  private synchronized void _switchNow() throws IOException {
+    if (onlySwitchAfterRow)
+      throw new IllegalStateException("Can only switch on row boundries");
+    
+    if (switchSource()) {
+      if (key != null) {
+        iter.seek(new Range(key, true, range.getEndKey(), range.isEndKeyInclusive()), columnFamilies, inclusive);
+      }
+    }
+  }
+  
+  public void switchNow() throws IOException {
+    synchronized (copies) {
+      for (SourceSwitchingIterator ssi : copies)
+        ssi._switchNow();
+    }
+  }
+  
+  @Override
+  public synchronized void setInterruptFlag(AtomicBoolean flag) {
+    if (copies.size() != 1)
+      throw new IllegalStateException("setInterruptFlag() called after deep copies made " + copies.size());
+    
+    this.iflag = flag;
+    if (iter != null)
+      ((InterruptibleIterator) iter).setInterruptFlag(flag);
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java
new file mode 100644
index 0000000..ea1ecac
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/StatsIterator.java
@@ -0,0 +1,74 @@
+/**
+ * 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.system;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicLong;
+
+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.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+
+/**
+ * 
+ */
+public class StatsIterator extends WrappingIterator {
+  
+  private int numRead = 0;
+  private AtomicLong seekCounter;
+  private AtomicLong readCounter;
+  
+  public StatsIterator(SortedKeyValueIterator<Key,Value> source, AtomicLong seekCounter, AtomicLong readCounter) {
+    super.setSource(source);
+    this.seekCounter = seekCounter;
+    this.readCounter = readCounter;
+  }
+  
+  @Override
+  public void next() throws IOException {
+    super.next();
+    numRead++;
+    
+    if (numRead % 23 == 0) {
+      readCounter.addAndGet(numRead);
+      numRead = 0;
+    }
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    return new StatsIterator(getSource().deepCopy(env), seekCounter, readCounter);
+  }
+  
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    super.seek(range, columnFamilies, inclusive);
+    seekCounter.incrementAndGet();
+    readCounter.addAndGet(numRead);
+    numRead = 0;
+  }
+  
+  public void report() {
+    readCounter.addAndGet(numRead);
+    numRead = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.java
new file mode 100644
index 0000000..2657bab
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/SynchronizedIterator.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.system;
+
+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.Range;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/***
+ * SynchronizedIterator: wrap a SortedKeyValueIterator so that all of its methods are synchronized
+ */
+public class SynchronizedIterator<K extends WritableComparable<?>,V extends Writable> implements SortedKeyValueIterator<K,V> {
+  
+  private SortedKeyValueIterator<K,V> source = null;
+  
+  @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);
+  }
+  
+  @Override
+  public synchronized boolean hasTop() {
+    return source.hasTop();
+  }
+  
+  @Override
+  public synchronized void next() throws IOException {
+    source.next();
+  }
+  
+  @Override
+  public synchronized void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    source.seek(range, columnFamilies, inclusive);
+  }
+  
+  @Override
+  public synchronized K getTopKey() {
+    return source.getTopKey();
+  }
+  
+  @Override
+  public synchronized V getTopValue() {
+    return source.getTopValue();
+  }
+  
+  @Override
+  public synchronized SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env) {
+    return new SynchronizedIterator<K,V>(source.deepCopy(env));
+  }
+  
+  public SynchronizedIterator() {}
+  
+  public SynchronizedIterator(SortedKeyValueIterator<K,V> source) {
+    this.source = source;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/TimeSettingIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/TimeSettingIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/TimeSettingIterator.java
new file mode 100644
index 0000000..4eef14d
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/TimeSettingIterator.java
@@ -0,0 +1,92 @@
+/*
+ * 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.system;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+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.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+
+public class TimeSettingIterator implements InterruptibleIterator {
+  
+  private SortedKeyValueIterator<Key,Value> source;
+  private long time;
+  private Range range;
+  
+  public TimeSettingIterator(SortedKeyValueIterator<Key,Value> source, long time) {
+    this.source = source;
+    this.time = time;
+  }
+  
+  @Override
+  public Key getTopKey() {
+    Key key = new Key(source.getTopKey());
+    key.setTimestamp(time);
+    return key;
+  }
+  
+  @Override
+  public void setInterruptFlag(AtomicBoolean flag) {
+    ((InterruptibleIterator) source).setInterruptFlag(flag);
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    return new TimeSettingIterator(source.deepCopy(env), time);
+  }
+  
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+    
+  }
+  
+  @Override
+  public boolean hasTop() {
+    return source.hasTop() && !range.afterEndKey(getTopKey());
+  }
+  
+  @Override
+  public void next() throws IOException {
+    source.next();
+  }
+  
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    Range seekRange = IteratorUtil.maximizeStartKeyTimeStamp(range);
+    seekRange = IteratorUtil.minimizeEndKeyTimeStamp(seekRange);
+    source.seek(seekRange, columnFamilies, inclusive);
+    this.range = range;
+    while (hasTop() && range.beforeStartKey(getTopKey())) {
+      next();
+    }
+    
+  }
+  
+  @Override
+  public Value getTopValue() {
+    return source.getTopValue();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
new file mode 100644
index 0000000..a41f7be
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.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.system;
+
+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.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.TextUtil;
+import org.apache.commons.collections.map.LRUMap;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
+
+public class VisibilityFilter extends Filter {
+  private VisibilityEvaluator ve;
+  private Text defaultVisibility;
+  private LRUMap cache;
+  private Text tmpVis;
+  
+  private static final Logger log = Logger.getLogger(VisibilityFilter.class);
+  
+  public VisibilityFilter() {}
+  
+  public VisibilityFilter(SortedKeyValueIterator<Key,Value> iterator, Authorizations authorizations, byte[] defaultVisibility) {
+    setSource(iterator);
+    this.ve = new VisibilityEvaluator(authorizations);
+    this.defaultVisibility = new Text(defaultVisibility);
+    this.cache = new LRUMap(1000);
+    this.tmpVis = new Text();
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    return new VisibilityFilter(getSource().deepCopy(env), ve.getAuthorizations(), TextUtil.getBytes(defaultVisibility));
+  }
+  
+  @Override
+  public boolean accept(Key k, Value v) {
+    Text testVis = k.getColumnVisibility(tmpVis);
+    
+    if (testVis.getLength() == 0 && defaultVisibility.getLength() == 0)
+      return true;
+    else if (testVis.getLength() == 0)
+      testVis = defaultVisibility;
+    
+    Boolean b = (Boolean) cache.get(testVis);
+    if (b != null)
+      return b;
+    
+    try {
+      Boolean bb = ve.evaluate(new ColumnVisibility(testVis));
+      cache.put(new Text(testVis), bb);
+      return bb;
+    } catch (VisibilityParseException e) {
+      log.error("Parse Error", e);
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java
new file mode 100644
index 0000000..486e6cb
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/AgeOffFilter.java
@@ -0,0 +1,126 @@
+/*
+ * 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.user;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+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;
+
+/**
+ * A filter that ages off key/value pairs based on the Key's timestamp. It removes an entry if its timestamp is less than currentTime - threshold.
+ * 
+ * This filter requires a "ttl" option, in milliseconds, to determine the age off threshold.
+ */
+public class AgeOffFilter extends Filter {
+  private static final String TTL = "ttl";
+  private static final String CURRENT_TIME = "currentTime";
+  private long threshold;
+  private long currentTime;
+  
+  /**
+   * Accepts entries whose timestamps are less than currentTime - threshold.
+   * 
+   * @see org.apache.accumulo.core.iterators.Filter#accept(org.apache.accumulo.core.data.Key, org.apache.accumulo.core.data.Value)
+   */
+  @Override
+  public boolean accept(Key k, Value v) {
+    if (currentTime - k.getTimestamp() > threshold)
+      return false;
+    return true;
+  }
+  
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+    super.init(source, options, env);
+    threshold = -1;
+    if (options == null)
+      throw new IllegalArgumentException(TTL + " must be set for AgeOffFilter");
+    
+    String ttl = options.get(TTL);
+    if (ttl == null)
+      throw new IllegalArgumentException(TTL + " must be set for AgeOffFilter");
+    
+    threshold = Long.parseLong(ttl);
+    
+    String time = options.get(CURRENT_TIME);
+    if (time != null)
+      currentTime = Long.parseLong(time);
+    else
+      currentTime = System.currentTimeMillis();
+    
+    // add sanity checks for threshold and currentTime?
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    AgeOffFilter copy = (AgeOffFilter) super.deepCopy(env);
+    copy.currentTime = currentTime;
+    copy.threshold = threshold;
+    return copy;
+  }
+  
+  @Override
+  public IteratorOptions describeOptions() {
+    IteratorOptions io = super.describeOptions();
+    io.addNamedOption(TTL, "time to live (milliseconds)");
+    io.addNamedOption(CURRENT_TIME, "if set, use the given value as the absolute time in milliseconds as the current time of day");
+    io.setName("ageoff");
+    io.setDescription("AgeOffFilter removes entries with timestamps more than <ttl> milliseconds old");
+    return io;
+  }
+  
+  @Override
+  public boolean validateOptions(Map<String,String> options) {
+    super.validateOptions(options);
+    try {
+      Long.parseLong(options.get(TTL));
+    } catch (NumberFormatException e) {
+      return false;
+    }
+    return true;
+  }
+  
+  /**
+   * A convenience method for setting the age off threshold.
+   * 
+   * @param is
+   *          IteratorSetting object to configure.
+   * @param ttl
+   *          age off threshold in milliseconds.
+   */
+  public static void setTTL(IteratorSetting is, Long ttl) {
+    is.addOption(TTL, Long.toString(ttl));
+  }
+  
+  /**
+   * A convenience method for setting the current time (from which to measure the age off threshold).
+   * 
+   * @param is
+   *          IteratorSetting object to configure.
+   * @param currentTime
+   *          time in milliseconds.
+   */
+  public static void setCurrentTime(IteratorSetting is, Long currentTime) {
+    is.addOption(CURRENT_TIME, Long.toString(currentTime));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java
new file mode 100644
index 0000000..8c95728
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/ColumnAgeOffFilter.java
@@ -0,0 +1,132 @@
+/*
+ * 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.user;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+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.conf.ColumnSet;
+import org.apache.accumulo.core.iterators.conf.ColumnToClassMapping;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.io.Text;
+
+/**
+ * A filter that ages off key/value pairs based on the Key's column and timestamp. It removes an entry if its timestamp is less than currentTime - threshold.
+ * Different thresholds are set for each column.
+ */
+public class ColumnAgeOffFilter extends Filter {
+  public static class TTLSet extends ColumnToClassMapping<Long> {
+    public TTLSet(Map<String,String> objectStrings) {
+      super();
+      
+      for (Entry<String,String> entry : objectStrings.entrySet()) {
+        String column = entry.getKey();
+        String ttl = entry.getValue();
+        Long l = Long.parseLong(ttl);
+        
+        Pair<Text,Text> colPair = ColumnSet.decodeColumns(column);
+        
+        if (colPair.getSecond() == null) {
+          addObject(colPair.getFirst(), l);
+        } else {
+          addObject(colPair.getFirst(), colPair.getSecond(), l);
+        }
+      }
+    }
+  }
+  
+  TTLSet ttls;
+  long currentTime = 0;
+  
+  @Override
+  public boolean accept(Key k, Value v) {
+    Long threshold = ttls.getObject(k);
+    if (threshold == null)
+      return true;
+    if (currentTime - k.getTimestamp() > threshold)
+      return false;
+    return true;
+  }
+  
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+    super.init(source, options, env);
+    this.ttls = new TTLSet(options);
+    currentTime = System.currentTimeMillis();
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    ColumnAgeOffFilter copy = (ColumnAgeOffFilter) super.deepCopy(env);
+    copy.currentTime = currentTime;
+    copy.ttls = ttls;
+    return copy;
+  }
+  
+  public void overrideCurrentTime(long ts) {
+    this.currentTime = ts;
+  }
+  
+  @Override
+  public IteratorOptions describeOptions() {
+    IteratorOptions io = super.describeOptions();
+    io.setName("colageoff");
+    io.setDescription("ColumnAgeOffFilter ages off columns at different rates given a time to live in milliseconds for each column");
+    io.addUnnamedOption("<col fam>[:<col qual>] <Long> (escape non-alphanum chars using %<hex>)");
+    return io;
+  }
+  
+  @Override
+  public boolean validateOptions(Map<String,String> options) {
+    super.validateOptions(options);
+    this.ttls = new TTLSet(options);
+    return true;
+  }
+  
+  /**
+   * A convenience method for adding or changing an age off threshold for a column.
+   * 
+   * @param is
+   *          IteratorSetting object to configure.
+   * @param column
+   *          column to encode as a parameter name.
+   * @param ttl
+   *          age off threshold in milliseconds.
+   */
+  public static void addTTL(IteratorSetting is, IteratorSetting.Column column, Long ttl) {
+    is.addOption(ColumnSet.encodeColumns(column.getFirst(), column.getSecond()), Long.toString(ttl));
+  }
+  
+  /**
+   * A convenience method for removing an age off threshold for a column.
+   * 
+   * @param is
+   *          IteratorSetting object to configure.
+   * @param column
+   *          column to encode as a parameter name.
+   */
+  public static void removeTTL(IteratorSetting is, IteratorSetting.Column column) {
+    is.removeOption(ColumnSet.encodeColumns(column.getFirst(), column.getSecond()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7bdbfccb/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java
----------------------------------------------------------------------
diff --git a/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java
new file mode 100644
index 0000000..ab8ca84
--- /dev/null
+++ b/1.5/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java
@@ -0,0 +1,103 @@
+/*
+ * 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.user;
+
+import java.io.IOException;
+import java.util.Arrays;
+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.SortedKeyValueIterator;
+
+/**
+ * This iterator provides exact string matching. It searches both the Key and Value for the string. The string to match is specified by the "term" option.
+ */
+public class GrepIterator extends Filter {
+  
+  private byte term[];
+  
+  @Override
+  public boolean accept(Key k, Value v) {
+    return match(v.get()) || match(k.getRowData()) || match(k.getColumnFamilyData()) || match(k.getColumnQualifierData());
+  }
+  
+  private boolean match(ByteSequence bs) {
+    return indexOf(bs.getBackingArray(), bs.offset(), bs.length(), term) >= 0;
+  }
+  
+  private boolean match(byte[] ba) {
+    return indexOf(ba, 0, ba.length, term) >= 0;
+  }
+  
+  // copied code below from java string and modified
+  
+  private static int indexOf(byte[] source, int sourceOffset, int sourceCount, byte[] target) {
+    byte first = target[0];
+    int targetCount = target.length;
+    int max = sourceOffset + (sourceCount - targetCount);
+    
+    for (int i = sourceOffset; i <= max; i++) {
+      /* Look for first character. */
+      if (source[i] != first) {
+        while (++i <= max && source[i] != first)
+          continue;
+      }
+      
+      /* Found first character, now look at the rest of v2 */
+      if (i <= max) {
+        int j = i + 1;
+        int end = j + targetCount - 1;
+        for (int k = 1; j < end && source[j] == target[k]; j++, k++)
+          continue;
+        
+        if (j == end) {
+          /* Found whole string. */
+          return i - sourceOffset;
+        }
+      }
+    }
+    return -1;
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    GrepIterator copy = (GrepIterator) super.deepCopy(env);
+    copy.term = Arrays.copyOf(term, term.length);
+    return copy;
+  }
+  
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+    super.init(source, options, env);
+    term = options.get("term").getBytes();
+  }
+  
+  /**
+   * Encode the grep term as an option for a ScanIterator
+   * 
+   * @param cfg
+   * @param term
+   */
+  public static void setTerm(IteratorSetting cfg, String term) {
+    cfg.addOption("term", term);
+  }
+}


Mime
View raw message