You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bi...@apache.org on 2011/11/17 23:31:37 UTC

svn commit: r1203397 - in /incubator/accumulo/trunk: ./ src/core/src/main/java/org/apache/accumulo/core/client/admin/ src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/ src/core/src/main/java/org/apache/accumulo/core/client/mock/ src/cor...

Author: billie
Date: Thu Nov 17 22:31:36 2011
New Revision: 1203397

URL: http://svn.apache.org/viewvc?rev=1203397&view=rev
Log:
ACCUMULO-155 merged to trunk

Added:
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java
      - copied unchanged from r1203389, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java
      - copied unchanged from r1203389, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java
      - copied unchanged from r1203389, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java
      - copied unchanged from r1203389, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java
Modified:
    incubator/accumulo/trunk/   (props changed)
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java
    incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java
    incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
    incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java

Propchange: incubator/accumulo/trunk/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov 17 22:31:36 2011
@@ -1,2 +1,2 @@
 /incubator/accumulo/branches/1.3:1190280,1190413,1190420,1190427,1190500,1195622,1195625,1195629,1195635,1196044,1196054,1196057,1196071-1196072,1196106,1197066,1198935,1199383
-/incubator/accumulo/branches/1.4:1201902-1203347
+/incubator/accumulo/branches/1.4:1201902-1203389

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Thu Nov 17 22:31:36 2011
@@ -59,7 +59,6 @@ import org.apache.accumulo.core.data.Byt
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.IteratorUtil;
-import org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.master.thrift.TableOperation;
@@ -182,8 +181,7 @@ public class TableOperationsImpl extends
     
     Map<String,String> opts;
     if (limitVersion) {
-      List<PerColumnIteratorConfig> emptyArgs = Collections.emptyList();
-      opts = IteratorUtil.generateInitialTableProperties(emptyArgs);
+      opts = IteratorUtil.generateInitialTableProperties();
     } else
       opts = Collections.emptyMap();
     
@@ -1039,13 +1037,14 @@ public class TableOperationsImpl extends
    * @deprecated since 1.4 {@link #attachIterator(String, IteratorSetting)}
    */
   @Override
-  public void addAggregators(String tableName, List<? extends PerColumnIteratorConfig> aggregators) throws AccumuloSecurityException, TableNotFoundException,
+  public void addAggregators(String tableName, List<? extends org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig> aggregators)
+      throws AccumuloSecurityException, TableNotFoundException,
       AccumuloException {
     ArgumentChecker.notNull(tableName, aggregators);
     MasterClientService.Iface client = null;
     try {
       client = MasterClient.getConnection(instance);
-      for (Entry<String,String> entry : IteratorUtil.generateInitialTableProperties(aggregators).entrySet()) {
+      for (Entry<String,String> entry : IteratorUtil.generateAggTableProperties(aggregators).entrySet()) {
         client.setTableProperty(null, credentials, tableName, entry.getKey(), entry.getValue());
       }
     } catch (ThriftSecurityException e) {

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java Thu Nov 17 22:31:36 2011
@@ -1,12 +1,18 @@
 /*
- * 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.
+ * 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.client.mapreduce;
 
@@ -55,7 +61,7 @@ import org.apache.accumulo.core.data.Par
 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.VersioningIterator;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.AuthInfo;

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java Thu Nov 17 22:31:36 2011
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
@@ -25,6 +24,7 @@ import java.util.Map.Entry;
 import java.util.SortedMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 
+import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -33,7 +33,6 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil;
-import org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig;
 import org.apache.accumulo.core.security.TablePermission;
 
 public class MockTable {
@@ -88,24 +87,7 @@ public class MockTable {
   
   MockTable(boolean useVersions, TimeType timeType) {
     this.timeType = timeType;
-    List<PerColumnIteratorConfig> aggs = Collections.emptyList();
-    settings = IteratorUtil.generateInitialTableProperties(aggs);
-    for (Entry<String,String> entry : AccumuloConfiguration.getDefaultConfiguration()) {
-      String key = entry.getKey();
-      if (key.startsWith(Property.TABLE_PREFIX.getKey()))
-        settings.put(key, entry.getValue());
-    }
-  }
-  
-  /**
-   * @deprecated since 1.4, use {@link #MockTable(boolean, TimeType)}
-   * @see {@link #addAggregators(List)}
-   * @param aggregators
-   * @param timeType
-   */
-  MockTable(List<? extends PerColumnIteratorConfig> aggregators, TimeType timeType) {
-    this.timeType = timeType;
-    settings = IteratorUtil.generateInitialTableProperties(aggregators);
+    settings = IteratorUtil.generateInitialTableProperties();
     for (Entry<String,String> entry : AccumuloConfiguration.getDefaultConfiguration()) {
       String key = entry.getKey();
       if (key.startsWith(Property.TABLE_PREFIX.getKey()))
@@ -131,8 +113,11 @@ public class MockTable {
     }
   }
   
-  public void addAggregators(List<? extends PerColumnIteratorConfig> aggregators) {
-    for (Entry<String,String> entry : IteratorUtil.generateInitialTableProperties(aggregators).entrySet()) {
+  /**
+   * @deprecated since 1.4 {@link #attachIterator(String, IteratorSetting)}
+   */
+  public void addAggregators(List<? extends org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig> aggregators) {
+    for (Entry<String,String> entry : IteratorUtil.generateAggTableProperties(aggregators).entrySet()) {
       String key = entry.getKey();
       if (key.startsWith(Property.TABLE_PREFIX.getKey()))
         settings.put(key, entry.getValue());

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java Thu Nov 17 22:31:36 2011
@@ -28,6 +28,7 @@ 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.user.IntersectingIterator;
 import org.apache.hadoop.io.Text;
 
 public class FamilyIntersectingIterator extends IntersectingIterator {

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java Thu Nov 17 22:31:36 2011
@@ -16,88 +16,12 @@
  */
 package org.apache.accumulo.core.iterators;
 
-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;
-
-public class GrepIterator extends SkippingIterator {
-  
-  private byte term[];
-  
-  @Override
-  protected void consume() throws IOException {
-    while (getSource().hasTop()) {
-      Key k = getSource().getTopKey();
-      Value v = getSource().getTopValue();
-      
-      if (match(v.get()) || match(k.getRowData()) || match(k.getColumnFamilyData()) || match(k.getColumnQualifierData())) {
-        break;
-      }
-      
-      getSource().next();
-    }
-  }
-  
-  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) {
-    throw new UnsupportedOperationException();
-  }
-  
-  @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();
-  }
+/**
+ * This class remains here for backwards compatibility.
+ * 
+ * @deprecated since 1.4
+ * @see org.apache.accumulo.core.iterators.user.GrepIterator
+ */
+public class GrepIterator extends org.apache.accumulo.core.iterators.user.GrepIterator {
   
-  /**
-   * 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);
-  }
 }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java Thu Nov 17 22:31:36 2011
@@ -16,493 +16,12 @@
  */
 package org.apache.accumulo.core.iterators;
 
-import java.io.IOException;
-import java.util.Collection;
-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.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.TextUtil;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Logger;
-
-public class IntersectingIterator implements SortedKeyValueIterator<Key,Value> {
-  
-  protected Text nullText = new Text();
-  
-  protected Text getPartition(Key key) {
-    return key.getRow();
-  }
-  
-  protected Text getTerm(Key key) {
-    return key.getColumnFamily();
-  }
-  
-  protected Text getDocID(Key key) {
-    return key.getColumnQualifier();
-  }
-  
-  protected Key buildKey(Text partition, Text term) {
-    return new Key(partition, (term == null) ? nullText : term);
-  }
-  
-  protected Key buildKey(Text partition, Text term, Text docID) {
-    return new Key(partition, (term == null) ? nullText : term, docID);
-  }
-  
-  protected Key buildFollowingPartitionKey(Key key) {
-    return key.followingKey(PartialKey.ROW);
-  }
-  
-  protected static final Logger log = Logger.getLogger(IntersectingIterator.class);
-  
-  protected static class TermSource {
-    public SortedKeyValueIterator<Key,Value> iter;
-    public Text term;
-    public boolean notFlag;
-    
-    public TermSource(TermSource other) {
-      this.iter = other.iter;
-      this.term = other.term;
-      this.notFlag = other.notFlag;
-    }
-    
-    public TermSource(SortedKeyValueIterator<Key,Value> iter, Text term) {
-      this.iter = iter;
-      this.term = term;
-      this.notFlag = false;
-    }
-    
-    public TermSource(SortedKeyValueIterator<Key,Value> iter, Text term, boolean notFlag) {
-      this.iter = iter;
-      this.term = term;
-      this.notFlag = notFlag;
-    }
-    
-    public String getTermString() {
-      return (this.term == null) ? new String("Iterator") : this.term.toString();
-    }
-  }
-  
-  TermSource[] sources;
-  int sourcesCount = 0;
-  
-  Range overallRange;
-  
-  // query-time settings
-  Text currentPartition = null;
-  Text currentDocID = new Text(emptyByteArray);
-  static final byte[] emptyByteArray = new byte[0];
-  
-  Key topKey = null;
-  Value value = new Value(emptyByteArray);
-  
-  protected Collection<ByteSequence> seekColumnFamilies;
-  
-  protected boolean inclusive;
-  
-  public IntersectingIterator() {}
-  
-  @Override
-  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    return new IntersectingIterator(this, env);
-  }
-  
-  public IntersectingIterator(IntersectingIterator other, IteratorEnvironment env) {
-    if (other.sources != null) {
-      sourcesCount = other.sourcesCount;
-      sources = new TermSource[sourcesCount];
-      for (int i = 0; i < sourcesCount; i++) {
-        sources[i] = new TermSource(other.sources[i].iter.deepCopy(env), other.sources[i].term);
-      }
-    }
-  }
-  
-  @Override
-  public Key getTopKey() {
-    return topKey;
-  }
-  
-  @Override
-  public Value getTopValue() {
-    // we don't really care about values
-    return value;
-  }
-  
-  @Override
-  public boolean hasTop() {
-    return currentPartition != null;
-  }
-  
-  // precondition: currentRow is not null
-  private boolean seekOneSource(int sourceID) throws IOException {
-    // find the next key in the appropriate column family that is at or beyond the cursor (currentRow, currentCQ)
-    // advance the cursor if this source goes beyond it
-    // return whether we advanced the cursor
-    
-    // within this loop progress must be made in one of the following forms:
-    // - currentRow or currentCQ must be increased
-    // - the given source must advance its iterator
-    // this loop will end when any of the following criteria are met
-    // - the iterator for the given source is pointing to the key (currentRow, columnFamilies[sourceID], currentCQ)
-    // - the given source is out of data and currentRow is set to null
-    // - the given source has advanced beyond the endRow and currentRow is set to null
-    boolean advancedCursor = false;
-    
-    if (sources[sourceID].notFlag) {
-      while (true) {
-        if (sources[sourceID].iter.hasTop() == false) {
-          // an empty column that you are negating is a valid condition
-          break;
-        }
-        // check if we're past the end key
-        int endCompare = -1;
-        // we should compare the row to the end of the range
-        if (overallRange.getEndKey() != null) {
-          endCompare = overallRange.getEndKey().getRow().compareTo(sources[sourceID].iter.getTopKey().getRow());
-          if ((!overallRange.isEndKeyInclusive() && endCompare <= 0) || endCompare < 0) {
-            // an empty column that you are negating is a valid condition
-            break;
-          }
-        }
-        int partitionCompare = currentPartition.compareTo(getPartition(sources[sourceID].iter.getTopKey()));
-        // check if this source is already at or beyond currentRow
-        // if not, then seek to at least the current row
-        
-        if (partitionCompare > 0) {
-          // seek to at least the currentRow
-          Key seekKey = buildKey(currentPartition, sources[sourceID].term);
-          sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive);
-          continue;
-        }
-        // check if this source has gone beyond currentRow
-        // if so, this is a valid condition for negation
-        if (partitionCompare < 0) {
-          break;
-        }
-        // we have verified that the current source is positioned in currentRow
-        // now we must make sure we're in the right columnFamily in the current row
-        // Note: Iterators are auto-magically set to the correct columnFamily
-        if (sources[sourceID].term != null) {
-          int termCompare = sources[sourceID].term.compareTo(getTerm(sources[sourceID].iter.getTopKey()));
-          // check if this source is already on the right columnFamily
-          // if not, then seek forwards to the right columnFamily
-          if (termCompare > 0) {
-            Key seekKey = buildKey(currentPartition, sources[sourceID].term, currentDocID);
-            sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive);
-            continue;
-          }
-          // check if this source is beyond the right columnFamily
-          // if so, then this is a valid condition for negating
-          if (termCompare < 0) {
-            break;
-          }
-        }
-        
-        // we have verified that we are in currentRow and the correct column family
-        // make sure we are at or beyond columnQualifier
-        Text docID = getDocID(sources[sourceID].iter.getTopKey());
-        int docIDCompare = currentDocID.compareTo(docID);
-        // If we are past the target, this is a valid result
-        if (docIDCompare < 0) {
-          break;
-        }
-        // if this source is not yet at the currentCQ then advance in this source
-        if (docIDCompare > 0) {
-          // seek forwards
-          Key seekKey = buildKey(currentPartition, sources[sourceID].term, currentDocID);
-          sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive);
-          continue;
-        }
-        // if we are equal to the target, this is an invalid result.
-        // Force the entire process to go to the next row.
-        // We are advancing column 0 because we forced that column to not contain a !
-        // when we did the init()
-        if (docIDCompare == 0) {
-          sources[0].iter.next();
-          advancedCursor = true;
-          break;
-        }
-      }
-    } else {
-      while (true) {
-        if (sources[sourceID].iter.hasTop() == false) {
-          currentPartition = null;
-          // setting currentRow to null counts as advancing the cursor
-          return true;
-        }
-        // check if we're past the end key
-        int endCompare = -1;
-        // we should compare the row to the end of the range
-        
-        if (overallRange.getEndKey() != null) {
-          endCompare = overallRange.getEndKey().getRow().compareTo(sources[sourceID].iter.getTopKey().getRow());
-          if ((!overallRange.isEndKeyInclusive() && endCompare <= 0) || endCompare < 0) {
-            currentPartition = null;
-            // setting currentRow to null counts as advancing the cursor
-            return true;
-          }
-        }
-        int partitionCompare = currentPartition.compareTo(getPartition(sources[sourceID].iter.getTopKey()));
-        // check if this source is already at or beyond currentRow
-        // if not, then seek to at least the current row
-        if (partitionCompare > 0) {
-          // seek to at least the currentRow
-          Key seekKey = buildKey(currentPartition, sources[sourceID].term);
-          sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive);
-          continue;
-        }
-        // check if this source has gone beyond currentRow
-        // if so, advance currentRow
-        if (partitionCompare < 0) {
-          currentPartition.set(getPartition(sources[sourceID].iter.getTopKey()));
-          currentDocID.set(emptyByteArray);
-          advancedCursor = true;
-          continue;
-        }
-        // we have verified that the current source is positioned in currentRow
-        // now we must make sure we're in the right columnFamily in the current row
-        // Note: Iterators are auto-magically set to the correct columnFamily
-        
-        if (sources[sourceID].term != null) {
-          int termCompare = sources[sourceID].term.compareTo(getTerm(sources[sourceID].iter.getTopKey()));
-          // check if this source is already on the right columnFamily
-          // if not, then seek forwards to the right columnFamily
-          if (termCompare > 0) {
-            Key seekKey = buildKey(currentPartition, sources[sourceID].term, currentDocID);
-            sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive);
-            continue;
-          }
-          // check if this source is beyond the right columnFamily
-          // if so, then seek to the next row
-          if (termCompare < 0) {
-            // we're out of entries in the current row, so seek to the next one
-            // byte[] currentRowBytes = currentRow.getBytes();
-            // byte[] nextRow = new byte[currentRowBytes.length + 1];
-            // System.arraycopy(currentRowBytes, 0, nextRow, 0, currentRowBytes.length);
-            // nextRow[currentRowBytes.length] = (byte)0;
-            // // we should reuse text objects here
-            // sources[sourceID].seek(new Key(new Text(nextRow),columnFamilies[sourceID]));
-            if (endCompare == 0) {
-              // we're done
-              currentPartition = null;
-              // setting currentRow to null counts as advancing the cursor
-              return true;
-            }
-            Key seekKey = buildFollowingPartitionKey(sources[sourceID].iter.getTopKey());
-            sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive);
-            continue;
-          }
-        }
-        // we have verified that we are in currentRow and the correct column family
-        // make sure we are at or beyond columnQualifier
-        Text docID = getDocID(sources[sourceID].iter.getTopKey());
-        int docIDCompare = currentDocID.compareTo(docID);
-        // if this source has advanced beyond the current column qualifier then advance currentCQ and return true
-        if (docIDCompare < 0) {
-          currentDocID.set(docID);
-          advancedCursor = true;
-          break;
-        }
-        // if this source is not yet at the currentCQ then seek in this source
-        if (docIDCompare > 0) {
-          // seek forwards
-          Key seekKey = buildKey(currentPartition, sources[sourceID].term, currentDocID);
-          sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive);
-          continue;
-        }
-        // this source is at the current row, in its column family, and at currentCQ
-        break;
-      }
-    }
-    return advancedCursor;
-  }
-  
-  @Override
-  public void next() throws IOException {
-    if (currentPartition == null) {
-      return;
-    }
-    // precondition: the current row is set up and the sources all have the same column qualifier
-    // while we don't have a match, seek in the source with the smallest column qualifier
-    sources[0].iter.next();
-    advanceToIntersection();
-  }
-  
-  protected void advanceToIntersection() throws IOException {
-    boolean cursorChanged = true;
-    while (cursorChanged) {
-      // seek all of the sources to at least the highest seen column qualifier in the current row
-      cursorChanged = false;
-      for (int i = 0; i < sourcesCount; i++) {
-        if (currentPartition == null) {
-          topKey = null;
-          return;
-        }
-        if (seekOneSource(i)) {
-          cursorChanged = true;
-          break;
-        }
-      }
-    }
-    topKey = buildKey(currentPartition, nullText, currentDocID);
-  }
-  
-  public static String stringTopKey(SortedKeyValueIterator<Key,Value> iter) {
-    if (iter.hasTop())
-      return iter.getTopKey().toString();
-    return "";
-  }
-  
-  public static final String columnFamiliesOptionName = "columnFamilies";
-  public static final String notFlagOptionName = "notFlag";
-  
-  // to be made private
-  // @see setColumnFamilies
-  public static String encodeColumns(Text[] columns) {
-    StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < columns.length; i++) {
-      sb.append(new String(Base64.encodeBase64(TextUtil.getBytes(columns[i]))));
-      sb.append('\n');
-    }
-    return sb.toString();
-  }
-  
-  public static String encodeBooleans(boolean[] flags) {
-    byte[] bytes = new byte[flags.length];
-    for (int i = 0; i < flags.length; i++) {
-      if (flags[i])
-        bytes[i] = 1;
-      else
-        bytes[i] = 0;
-    }
-    return new String(Base64.encodeBase64(bytes));
-  }
-  
-  private static Text[] decodeColumns(String columns) {
-    String[] columnStrings = columns.split("\n");
-    Text[] columnTexts = new Text[columnStrings.length];
-    for (int i = 0; i < columnStrings.length; i++) {
-      columnTexts[i] = new Text(Base64.decodeBase64(columnStrings[i].getBytes()));
-    }
-    return columnTexts;
-  }
-  
-  public static boolean[] decodeBooleans(String flags) {
-    // return null of there were no flags
-    if (flags == null)
-      return null;
-    
-    byte[] bytes = Base64.decodeBase64(flags.getBytes());
-    boolean[] bFlags = new boolean[bytes.length];
-    for (int i = 0; i < bytes.length; i++) {
-      if (bytes[i] == 1)
-        bFlags[i] = true;
-      else
-        bFlags[i] = false;
-    }
-    return bFlags;
-  }
-  
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
-    Text[] terms = decodeColumns(options.get(columnFamiliesOptionName));
-    boolean[] notFlag = decodeBooleans(options.get(notFlagOptionName));
-    
-    if (terms.length < 2) {
-      throw new IllegalArgumentException("IntersectionIterator requires two or more columns families");
-    }
-    
-    // Scan the not flags.
-    // There must be at least one term that isn't negated
-    // And we are going to re-order such that the first term is not a ! term
-    if (notFlag == null) {
-      notFlag = new boolean[terms.length];
-      for (int i = 0; i < terms.length; i++)
-        notFlag[i] = false;
-    }
-    if (notFlag[0]) {
-      for (int i = 1; i < notFlag.length; i++) {
-        if (notFlag[i] == false) {
-          Text swapFamily = new Text(terms[0]);
-          terms[0].set(terms[i]);
-          terms[i].set(swapFamily);
-          notFlag[0] = false;
-          notFlag[i] = true;
-          break;
-        }
-      }
-      if (notFlag[0]) {
-        throw new IllegalArgumentException("IntersectionIterator requires at lest one column family without not");
-      }
-    }
-    
-    sources = new TermSource[terms.length];
-    sources[0] = new TermSource(source, terms[0]);
-    for (int i = 1; i < terms.length; i++) {
-      sources[i] = new TermSource(source.deepCopy(env), terms[i], notFlag[i]);
-    }
-    sourcesCount = terms.length;
-  }
-  
-  @Override
-  public void seek(Range range, Collection<ByteSequence> seekColumnFamilies, boolean inclusive) throws IOException {
-    overallRange = new Range(range);
-    currentPartition = new Text();
-    currentDocID.set(emptyByteArray);
-    
-    this.seekColumnFamilies = seekColumnFamilies;
-    this.inclusive = inclusive;
-    
-    // seek each of the sources to the right column family within the row given by key
-    for (int i = 0; i < sourcesCount; i++) {
-      Key sourceKey;
-      if (range.getStartKey() != null) {
-        if (range.getStartKey().getColumnQualifier() != null) {
-          sourceKey = buildKey(getPartition(range.getStartKey()), sources[i].term, range.getStartKey().getColumnQualifier());
-        } else {
-          sourceKey = buildKey(getPartition(range.getStartKey()), sources[i].term);
-        }
-        sources[i].iter.seek(new Range(sourceKey, true, null, false), seekColumnFamilies, inclusive);
-      } else {
-        sources[i].iter.seek(range, seekColumnFamilies, inclusive);
-      }
-    }
-    advanceToIntersection();
-  }
-  
-  public void addSource(SortedKeyValueIterator<Key,Value> source, IteratorEnvironment env, Text term, boolean notFlag) {
-    // Check if we have space for the added Source
-    if (sources == null) {
-      sources = new TermSource[1];
-    } else {
-      // allocate space for node, and copy current tree.
-      // TODO: Should we change this to an ArrayList so that we can just add() ?
-      TermSource[] localSources = new TermSource[sources.length + 1];
-      int currSource = 0;
-      for (TermSource myTerm : sources) {
-        // TODO: Do I need to call new here? or can I just re-use the term?
-        localSources[currSource] = new TermSource(myTerm);
-        currSource++;
-      }
-      sources = localSources;
-    }
-    sources[sourcesCount] = new TermSource(source.deepCopy(env), term, notFlag);
-    sourcesCount++;
-  }
+/**
+ * This class remains here for backwards compatibility.
+ * 
+ * @deprecated since 1.4
+ * @see org.apache.accumulo.core.iterators.user.IntersectingIterator
+ */
+public class IntersectingIterator extends org.apache.accumulo.core.iterators.user.IntersectingIterator {
   
-  /**
-   * Encode the columns to be used when iterating.
-   * 
-   * @param cfg
-   * @param columns
-   */
-  public static void setColumnFamilies(IteratorSetting cfg, Text[] columns) {
-    cfg.addOption(IntersectingIterator.columnFamiliesOptionName, IntersectingIterator.encodeColumns(columns));
-  }
 }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java Thu Nov 17 22:31:36 2011
@@ -27,6 +27,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -34,6 +36,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.start.classloader.AccumuloClassLoader;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -62,17 +65,14 @@ public class IteratorUtil {
    * 
    * @see {@link TableOperations#attachIterators(String, IteratorSetting)}
    */
-  public static Map<String,String> generateInitialTableProperties(List<? extends PerColumnIteratorConfig> aggregators) {
+  public static Map<String,String> generateAggTableProperties(List<? extends PerColumnIteratorConfig> aggregators) {
     
-    TreeMap<String,String> props = new TreeMap<String,String>();
+    Map<String,String> props = new TreeMap<String,String>();
     
     for (IteratorScope iterScope : IteratorScope.values()) {
       if (aggregators.size() > 0) {
         props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".agg", "10," + AggregatingIterator.class.getName());
       }
-      
-      props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers", "20," + VersioningIterator.class.getName());
-      props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers.opt.maxVersions", "1");
     }
     
     for (PerColumnIteratorConfig ac : aggregators) {
@@ -84,6 +84,16 @@ public class IteratorUtil {
     return props;
   }
   
+  public static Map<String,String> generateInitialTableProperties() {
+    TreeMap<String,String> props = new TreeMap<String,String>();
+    
+    for (IteratorScope iterScope : IteratorScope.values()) {
+      props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers", "20," + VersioningIterator.class.getName());
+      props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers.opt.maxVersions", "1");
+    }
+    return props;
+  }
+  
   public static int getMaxPriority(IteratorScope scope, AccumuloConfiguration conf) {
     List<IterInfo> iters = new ArrayList<IterInfo>();
     parseIterConf(scope, iters, new HashMap<String,Map<String,String>>(), conf);

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java Thu Nov 17 22:31:36 2011
@@ -16,263 +16,12 @@
  */
 package org.apache.accumulo.core.iterators;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map;
-
-import org.apache.accumulo.core.data.ArrayByteSequence;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-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.iterators.IteratorUtil.IteratorScope;
-import org.apache.hadoop.io.Text;
-
 /**
- * This iterator suppresses rows that exceed a specified number of columns. Once a row exceeds the threshold, a marker is emitted and the row is always
- * suppressed by this iterator after that point in time.
+ * This class remains here for backwards compatibility.
  * 
- * This iterator works in a similar way to the RowDeletingIterator. See its javadoc about locality groups.
+ * @deprecated since 1.4
+ * @see org.apache.accumulo.core.iterators.user.LargeRowFilter
  */
-public class LargeRowFilter implements SortedKeyValueIterator<Key,Value>, OptionDescriber {
-  
-  public static final Value SUPPRESS_ROW_VALUE = new Value("SUPPRESS_ROW".getBytes());
-  
-  private static final ByteSequence EMPTY = new ArrayByteSequence(new byte[] {});
-  
-  /* key into hash map, value refers to the row supression limit (maxColumns) */
-  public static final String MAX_COLUMNS = "max_columns";
-  
-  private SortedKeyValueIterator<Key,Value> source;
-  
-  // a cache of keys
-  private ArrayList<Key> keys = new ArrayList<Key>();
-  private ArrayList<Value> values = new ArrayList<Value>();
-  
-  private int currentPosition;
-  
-  private int maxColumns;
-  
-  private boolean propogateSuppression = false;
-  
-  private Range range;
-  private Collection<ByteSequence> columnFamilies;
-  private boolean inclusive;
-  private boolean dropEmptyColFams;
-  
-  private boolean isSuppressionMarker(Key key, Value val) {
-    return key.getColumnFamilyData().length() == 0 && key.getColumnQualifierData().length() == 0 && key.getColumnVisibilityData().length() == 0
-        && val.equals(SUPPRESS_ROW_VALUE);
-  }
-  
-  private void reseek(Key key) throws IOException {
-    if (range.afterEndKey(key)) {
-      range = new Range(range.getEndKey(), true, range.getEndKey(), range.isEndKeyInclusive());
-      source.seek(range, columnFamilies, inclusive);
-    } else {
-      range = new Range(key, true, range.getEndKey(), range.isEndKeyInclusive());
-      source.seek(range, columnFamilies, inclusive);
-    }
-  }
-  
-  private void consumeRow(ByteSequence row) throws IOException {
-    // try reading a few and if still not to next row, then seek
-    int count = 0;
-    
-    while (source.hasTop() && source.getTopKey().getRowData().equals(row)) {
-      source.next();
-      count++;
-      if (count >= 10) {
-        Key nextRowStart = new Key(new Text(row.toArray())).followingKey(PartialKey.ROW);
-        reseek(nextRowStart);
-        count = 0;
-      }
-    }
-  }
-  
-  private void addKeyValue(Key k, Value v) {
-    if (dropEmptyColFams && k.getColumnFamilyData().equals(EMPTY)) {
-      return;
-    }
-    keys.add(new Key(k));
-    values.add(new Value(v));
-  }
-  
-  private void bufferNextRow() throws IOException {
-    
-    keys.clear();
-    values.clear();
-    currentPosition = 0;
-    
-    while (source.hasTop() && keys.size() == 0) {
-      
-      addKeyValue(source.getTopKey(), source.getTopValue());
-      
-      if (isSuppressionMarker(source.getTopKey(), source.getTopValue())) {
-        
-        consumeRow(source.getTopKey().getRowData());
-        
-      } else {
-        
-        ByteSequence currentRow = keys.get(0).getRowData();
-        source.next();
-        
-        while (source.hasTop() && source.getTopKey().getRowData().equals(currentRow)) {
-          
-          addKeyValue(source.getTopKey(), source.getTopValue());
-          
-          if (keys.size() > maxColumns) {
-            keys.clear();
-            values.clear();
-            
-            // when the row is to big, just emit a suppression
-            // marker
-            addKeyValue(new Key(new Text(currentRow.toArray())), SUPPRESS_ROW_VALUE);
-            consumeRow(currentRow);
-          } else {
-            source.next();
-          }
-        }
-      }
-      
-    }
-  }
-  
-  private void readNextRow() throws IOException {
-    
-    bufferNextRow();
-    
-    while (!propogateSuppression && currentPosition < keys.size() && isSuppressionMarker(keys.get(0), values.get(0))) {
-      bufferNextRow();
-    }
-  }
-  
-  private LargeRowFilter(SortedKeyValueIterator<Key,Value> source, boolean propogateSuppression, int maxColumns) {
-    this.source = source;
-    this.propogateSuppression = propogateSuppression;
-    this.maxColumns = maxColumns;
-  }
-  
-  public LargeRowFilter() {}
-  
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
-    this.source = source;
-    this.maxColumns = Integer.parseInt(options.get(MAX_COLUMNS));
-    this.propogateSuppression = env.getIteratorScope() != IteratorScope.scan;
-  }
-  
-  @Override
-  public boolean hasTop() {
-    return currentPosition < keys.size();
-  }
-  
-  @Override
-  public void next() throws IOException {
-    
-    if (currentPosition >= keys.size()) {
-      throw new IllegalStateException("Called next() when hasTop() is false");
-    }
-    
-    currentPosition++;
-    
-    if (currentPosition == keys.size()) {
-      readNextRow();
-    }
-  }
-  
-  @Override
-  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
-    
-    if (inclusive && !columnFamilies.contains(EMPTY)) {
-      columnFamilies = new HashSet<ByteSequence>(columnFamilies);
-      columnFamilies.add(EMPTY);
-      dropEmptyColFams = true;
-    } else if (!inclusive && columnFamilies.contains(EMPTY)) {
-      columnFamilies = new HashSet<ByteSequence>(columnFamilies);
-      columnFamilies.remove(EMPTY);
-      dropEmptyColFams = true;
-    } else {
-      dropEmptyColFams = false;
-    }
-    
-    this.range = range;
-    this.columnFamilies = columnFamilies;
-    this.inclusive = inclusive;
-    
-    if (range.getStartKey() != null) {
-      // seek to beginning of row to see if there is a suppression marker
-      Range newRange = new Range(new Key(range.getStartKey().getRow()), true, range.getEndKey(), range.isEndKeyInclusive());
-      source.seek(newRange, columnFamilies, inclusive);
-      
-      readNextRow();
-      
-      // it is possible that all or some of the data read for the current
-      // row is before the start of the range
-      while (currentPosition < keys.size() && range.beforeStartKey(keys.get(currentPosition)))
-        currentPosition++;
-      
-      if (currentPosition == keys.size())
-        readNextRow();
-      
-    } else {
-      source.seek(range, columnFamilies, inclusive);
-      readNextRow();
-    }
-    
-  }
-  
-  @Override
-  public Key getTopKey() {
-    return keys.get(currentPosition);
-  }
-  
-  @Override
-  public Value getTopValue() {
-    return values.get(currentPosition);
-  }
-  
-  @Override
-  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    return new LargeRowFilter(source.deepCopy(env), propogateSuppression, maxColumns);
-  }
-  
-  @Override
-  public IteratorOptions describeOptions() {
-    String description = "This iterator suppresses rows that exceed a specified number of columns. Once\n"
-        + "a row exceeds the threshold, a marker is emitted and the row is always\n" + "suppressed by this iterator after that point in time.\n"
-        + " This iterator works in a similar way to the RowDeletingIterator. See its\n" + " javadoc about locality groups.\n";
-    return new IteratorOptions(this.getClass().getSimpleName(), description, Collections.singletonMap(MAX_COLUMNS, "Number Of Columns To Begin Suppression"),
-        null);
-  }
-  
-  @Override
-  public boolean validateOptions(Map<String,String> options) {
-    if (options == null || options.size() < 1) {
-      System.out.println("Bad # of options, must supply: " + MAX_COLUMNS + " as value");
-      return false;
-    }
-    
-    if (options.containsKey(MAX_COLUMNS)) {
-      try {
-        maxColumns = Integer.parseInt(options.get(MAX_COLUMNS));
-      } catch (NumberFormatException e) {
-        e.printStackTrace();
-        return false;
-      }
-    } else {
-      System.out.println("Need to have " + MAX_COLUMNS);
-      return false;
-    }
-    
-    return true;
-  }
+public class LargeRowFilter extends org.apache.accumulo.core.iterators.user.LargeRowFilter {
   
 }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java Thu Nov 17 22:31:36 2011
@@ -16,141 +16,19 @@
  */
 package org.apache.accumulo.core.iterators;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-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.PartialKey;
-import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 
-public class VersioningIterator extends WrappingIterator implements OptionDescriber {
-  
-  private Key currentKey = new Key();
-  private int numVersions;
-  private int maxVersions;
-  
-  @Override
-  public VersioningIterator deepCopy(IteratorEnvironment env) {
-    return new VersioningIterator(this, env);
-  }
-  
-  private VersioningIterator(VersioningIterator other, IteratorEnvironment env) {
-    setSource(other.getSource().deepCopy(env));
-    maxVersions = other.maxVersions;
-  }
-  
+/**
+ * This class remains here for backwards compatibility.
+ * 
+ * @deprecated since 1.4
+ * @see org.apache.accumulo.core.iterators.user.VersioningIterator
+ */
+public class VersioningIterator extends org.apache.accumulo.core.iterators.user.VersioningIterator {
   public VersioningIterator() {}
   
   public VersioningIterator(SortedKeyValueIterator<Key,Value> iterator, int maxVersions) {
-    if (maxVersions < 1)
-      throw new IllegalArgumentException("maxVersions for versioning iterator must be >= 1");
-    this.setSource(iterator);
-    this.maxVersions = maxVersions;
-  }
-  
-  @Override
-  public void next() throws IOException {
-    if (numVersions >= maxVersions) {
-      skipRowColumn();
-      resetVersionCount();
-      return;
-    }
-    
-    super.next();
-    if (getSource().hasTop()) {
-      if (getSource().getTopKey().equals(currentKey, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) {
-        numVersions++;
-      } else {
-        resetVersionCount();
-      }
-    }
-  }
-  
-  @Override
-  public boolean hasTop() {
-    return super.hasTop();
-  }
-  
-  @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);
-    resetVersionCount();
-    
-    if (range.getStartKey() != null) {
-      while (getSource().hasTop() && getSource().getTopKey().compareTo(range.getStartKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME) < 0) {
-        // the value has a more recent time stamp, so
-        // pass it up
-        // log.debug("skipping "+getTopKey());
-        next();
-      }
-      
-      while (hasTop() && range.beforeStartKey(getTopKey())) {
-        next();
-      }
-    }
-  }
-  
-  private void resetVersionCount() {
-    if (super.hasTop())
-      currentKey.set(getSource().getTopKey());
-    numVersions = 1;
-  }
-  
-  private void skipRowColumn() throws IOException {
-    Key keyToSkip = currentKey;
-    super.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) throws IOException {
-    super.init(source, options, env);
-    this.numVersions = 0;
-    
-    String maxVerString = options.get("maxVersions");
-    if (maxVerString != null)
-      this.maxVersions = Integer.parseInt(maxVerString);
-    else
-      this.maxVersions = 1;
-    
-    if (maxVersions < 1)
-      throw new IllegalArgumentException("maxVersions for versioning iterator must be >= 1");
-  }
-  
-  @Override
-  public IteratorOptions describeOptions() {
-    return new IteratorOptions("vers", "The VersioningIterator keeps a fixed number of versions for each key", Collections.singletonMap("maxVersions",
-        "number of versions to keep for a particular key (with differing timestamps)"), null);
-  }
-  
-  private static final String MAXVERSIONS_OPT = "maxVersions";
-  
-  @Override
-  public boolean validateOptions(Map<String,String> options) {
-    int i = Integer.parseInt(options.get(MAXVERSIONS_OPT));
-    if (i < 1)
-      throw new IllegalArgumentException(MAXVERSIONS_OPT + " for versioning iterator must be >= 1");
-    return true;
-  }
-  
-  /**
-   * Encode the maximum number of versions to return onto the ScanIterator
-   * 
-   * @param cfg
-   * @param maxVersions
-   */
-  public static void setMaxVersions(IteratorSetting cfg, int maxVersions) {
-    cfg.addOption(MAXVERSIONS_OPT, Integer.toString(maxVersions));
+    super(iterator, maxVersions);
   }
 }

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java Thu Nov 17 22:31:36 2011
@@ -16,12 +16,10 @@
  */
 package org.apache.accumulo.core.iterators.conf;
 
-import org.apache.accumulo.core.iterators.Combiner;
 import org.apache.hadoop.io.Text;
 
 /**
  * @deprecated since 1.4
- * @see Combiner#addColumn(Text, Text, accumulo.core.client.IteratorSetting)
  */
 public class PerColumnIteratorConfig {
   

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java Thu Nov 17 22:31:36 2011
@@ -19,7 +19,6 @@ package org.apache.accumulo.core.util.sh
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
@@ -136,8 +135,7 @@ public class CreateTableCommand extends 
     // context
     
     if (cl.hasOption(createTableNoDefaultIters.getOpt())) {
-      List<PerColumnIteratorConfig> empty = Collections.emptyList();
-      for (String key : IteratorUtil.generateInitialTableProperties(empty).keySet())
+      for (String key : IteratorUtil.generateInitialTableProperties().keySet())
         shellState.getConnector().tableOperations().removeProperty(tableName, key);
     }
     

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java Thu Nov 17 22:31:36 2011
@@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.A
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.iterators.GrepIterator;
+import org.apache.accumulo.core.iterators.user.GrepIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.commons.cli.CommandLine;

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java Thu Nov 17 22:31:36 2011
@@ -33,11 +33,11 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.OptionDescriber;
 import org.apache.accumulo.core.iterators.OptionDescriber.IteratorOptions;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.VersioningIterator;
 import org.apache.accumulo.core.iterators.aggregation.Aggregator;
 import org.apache.accumulo.core.iterators.user.AgeOffFilter;
 import org.apache.accumulo.core.iterators.user.NoVisFilter;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.accumulo.core.util.shell.ShellCommandException;

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java Thu Nov 17 22:31:36 2011
@@ -35,8 +35,8 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IntersectingIterator;
-import org.apache.accumulo.core.iterators.VersioningIterator;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java Thu Nov 17 22:31:36 2011
@@ -35,10 +35,10 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.file.rfile.RFileTest;
 import org.apache.accumulo.core.file.rfile.RFileTest.TestRFile;
 import org.apache.accumulo.core.iterators.FamilyIntersectingIterator;
-import org.apache.accumulo.core.iterators.IntersectingIterator;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java Thu Nov 17 22:31:36 2011
@@ -31,11 +31,11 @@ import org.apache.accumulo.core.data.Byt
 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.IntersectingIterator;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java Thu Nov 17 22:31:36 2011
@@ -27,10 +27,10 @@ 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.LargeRowFilter;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.iterators.user.LargeRowFilter;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 
 import junit.framework.TestCase;

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java Thu Nov 17 22:31:36 2011
@@ -29,8 +29,8 @@ 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.SortedMapIterator;
-import org.apache.accumulo.core.iterators.VersioningIterator;
 import org.apache.accumulo.core.iterators.aggregation.LongSummation;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.hadoop.io.Text;
 
 public class VersioningIteratorTest extends TestCase {

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java Thu Nov 17 22:31:36 2011
@@ -32,7 +32,6 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IntersectingIterator;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 

Modified: incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java (original)
+++ incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java Thu Nov 17 22:31:36 2011
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.client.Z
 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.IntersectingIterator;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.hadoop.io.Text;
 
 /**

Modified: incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java (original)
+++ incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java Thu Nov 17 22:31:36 2011
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.Z
 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.IntersectingIterator;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.hadoop.io.Text;
 
 /**

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java Thu Nov 17 22:31:36 2011
@@ -32,7 +32,7 @@ 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.GrepIterator;
+import org.apache.accumulo.core.iterators.user.GrepIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.server.ServerConstants;

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java Thu Nov 17 22:31:36 2011
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.client.I
 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.IntersectingIterator;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.server.test.randomwalk.State;
 import org.apache.accumulo.server.test.randomwalk.Test;

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java Thu Nov 17 22:31:36 2011
@@ -30,7 +30,7 @@ import org.apache.accumulo.core.client.S
 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.IntersectingIterator;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.server.test.randomwalk.State;
 import org.apache.accumulo.server.test.randomwalk.Test;
 import org.apache.hadoop.io.Text;

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java Thu Nov 17 22:31:36 2011
@@ -39,9 +39,9 @@ import org.apache.accumulo.core.file.Fil
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.VersioningIterator;
 import org.apache.accumulo.core.iterators.system.DeletingIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.core.util.LocalityGroupUtil;

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java Thu Nov 17 22:31:36 2011
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.FileUtil;
-import org.apache.accumulo.core.iterators.VersioningIterator;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
 import org.apache.accumulo.core.util.CachedConfiguration;

Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java?rev=1203397&r1=1203396&r2=1203397&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java (original)
+++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java Thu Nov 17 22:31:36 2011
@@ -38,11 +38,11 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.VersioningIterator;
 import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter;
 import org.apache.accumulo.core.iterators.system.DeletingIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.accumulo.core.iterators.system.VisibilityFilter;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.TextUtil;