You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2011/11/28 22:24:30 UTC

svn commit: r1207621 - in /incubator/accumulo/branches/1.4/src: core/src/main/java/org/apache/accumulo/core/iterators/ core/src/main/java/org/apache/accumulo/core/iterators/user/ core/src/test/java/org/apache/accumulo/core/client/impl/ core/src/test/ja...

Author: kturner
Date: Mon Nov 28 21:24:26 2011
New Revision: 1207621

URL: http://svn.apache.org/viewvc?rev=1207621&view=rev
Log:
ACCUMULO-155 moved some more iterators intended for users to the user package

Added:
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java
      - copied, changed from r1207611, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java
      - copied, changed from r1207611, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java
    incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java
      - copied, changed from r1207611, incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/RowDeletingIteratorTest.java
    incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/WholeRowIteratorTest.java
      - copied, changed from r1207611, incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/WholeRowIteratorTest.java
Removed:
    incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/RowDeletingIteratorTest.java
    incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/WholeRowIteratorTest.java
Modified:
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java
    incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java
    incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerOptionsTest.java
    incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
    incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/LargeRowFilterTest.java
    incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
    incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java
    incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/RowDeleteTest.java

Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java?rev=1207621&r1=1207620&r2=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java Mon Nov 28 21:24:26 2011
@@ -1,4 +1,4 @@
-/*
+/**
  * 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.
@@ -16,161 +16,12 @@
  */
 package org.apache.accumulo.core.iterators;
 
-import java.io.IOException;
-import java.util.Collection;
-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.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-
 /**
- * An iterator for deleting whole rows.
- * 
- * After setting this iterator up for your table, to delete a row insert a row with empty column family, empty column qualifier, empty column visibility, and a
- * value of DEL_ROW. Do not use empty columns for anything else when using this iterator.
- * 
- * When using this iterator the locality group containing the row deletes will always be read. The locality group containing the empty column family will
- * contain row deletes. Always reading this locality group can have an impact on performance.
- * 
- * For example assume there are two locality groups, one containing large images and one containing small metadata about the images. If row deletes are in the
- * same locality group as the images, then this will significantly slow down scans and major compactions that are only reading the metadata locality group.
- * Therefore, you would want to put the empty column family in the locality group that contains the metadata. Another option is to put the empty column in its
- * own locality group. Which is best depends on your data.
+ * This class remains here for backwards compatibility.
  * 
+ * @deprecated since 1.4
+ * @see org.apache.accumulo.core.iterators.user.RowDeletingIterator
  */
-
-public class RowDeletingIterator implements SortedKeyValueIterator<Key,Value> {
-  
-  public static final Value DELETE_ROW_VALUE = new Value("DEL_ROW".getBytes());
-  private SortedKeyValueIterator<Key,Value> source;
-  private boolean propogateDeletes;
-  private ByteSequence currentRow;
-  private boolean currentRowDeleted;
-  private long deleteTS;
-  
-  private boolean dropEmptyColFams;
-  
-  private static final ByteSequence EMPTY = new ArrayByteSequence(new byte[] {});
-  
-  private RowDeletingIterator(SortedKeyValueIterator<Key,Value> source, boolean propogateDeletes2) {
-    this.source = source;
-    this.propogateDeletes = propogateDeletes2;
-  }
-  
-  public RowDeletingIterator() {}
-  
-  @Override
-  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    return new RowDeletingIterator(source.deepCopy(env), propogateDeletes);
-  }
-  
-  @Override
-  public Key getTopKey() {
-    return source.getTopKey();
-  }
-  
-  @Override
-  public Value getTopValue() {
-    return source.getTopValue();
-  }
-  
-  @Override
-  public boolean hasTop() {
-    return source.hasTop();
-  }
-  
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
-    this.source = source;
-    this.propogateDeletes = (env.getIteratorScope() == IteratorScope.majc && !env.isFullMajorCompaction()) || env.getIteratorScope() == IteratorScope.minc;
-  }
-  
-  @Override
-  public void next() throws IOException {
-    source.next();
-    consumeDeleted();
-    consumeEmptyColFams();
-  }
-  
-  private void consumeEmptyColFams() throws IOException {
-    while (dropEmptyColFams && source.hasTop() && source.getTopKey().getColumnFamilyData().length() == 0) {
-      source.next();
-      consumeDeleted();
-    }
-  }
-  
-  private boolean isDeleteMarker(Key key, Value val) {
-    return key.getColumnFamilyData().length() == 0 && key.getColumnQualifierData().length() == 0 && key.getColumnVisibilityData().length() == 0
-        && val.equals(DELETE_ROW_VALUE);
-  }
-  
-  private void consumeDeleted() throws IOException {
-    // this method tries to do as little work as possible when nothing is deleted
-    while (source.hasTop()) {
-      if (currentRowDeleted) {
-        while (source.hasTop() && currentRow.equals(source.getTopKey().getRowData()) && source.getTopKey().getTimestamp() <= deleteTS) {
-          source.next();
-        }
-        
-        if (source.hasTop() && !currentRow.equals(source.getTopKey().getRowData())) {
-          currentRowDeleted = false;
-        }
-      }
-      
-      if (!currentRowDeleted && source.hasTop() && isDeleteMarker(source.getTopKey(), source.getTopValue())) {
-        currentRow = source.getTopKey().getRowData();
-        currentRowDeleted = true;
-        deleteTS = source.getTopKey().getTimestamp();
-        
-        if (propogateDeletes)
-          break;
-      } else {
-        break;
-      }
-    }
-    
-  }
-  
-  @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;
-    }
-    
-    currentRowDeleted = false;
-    
-    if (range.getStartKey() != null) {
-      // seek to beginning of row
-      Range newRange = new Range(new Key(range.getStartKey().getRow()), true, range.getEndKey(), range.isEndKeyInclusive());
-      source.seek(newRange, columnFamilies, inclusive);
-      consumeDeleted();
-      consumeEmptyColFams();
-      
-      if (source.hasTop() && range.beforeStartKey(source.getTopKey())) {
-        source.seek(range, columnFamilies, inclusive);
-        consumeDeleted();
-        consumeEmptyColFams();
-      }
-    } else {
-      source.seek(range, columnFamilies, inclusive);
-      consumeDeleted();
-      consumeEmptyColFams();
-    }
-    
-  }
+public class RowDeletingIterator extends org.apache.accumulo.core.iterators.user.RowDeletingIterator {
   
 }

Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java?rev=1207621&r1=1207620&r2=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java Mon Nov 28 21:24:26 2011
@@ -1,4 +1,4 @@
-/*
+/**
  * 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.
@@ -16,227 +16,12 @@
  */
 package org.apache.accumulo.core.iterators;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-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.hadoop.io.Text;
-
 /**
+ * This class remains here for backwards compatibility.
  * 
- * The WholeRowIterator is designed to provide row-isolation so that queries see mutations as atomic. It does so by encapsulating an entire row of key/value
- * pairs into a single key/value pair, which is returned through the client as an atomic operation.
- * 
- * <p>
- * One caveat is that when seeking in the WholeRowIterator using a range that starts at a non-inclusive first key in a row, (e.g. seek(new Range(new Key(new
- * Text("row")),false,...),...)) this iterator will skip to the next row. This is done in order to prevent repeated scanning of the same row when system
- * automatically creates ranges of that form, which happens in the case of the client calling continueScan, or in the case of the tablet server continuing a
- * scan after swapping out sources.
- * 
- * <p>
- * To regain the original key/value pairs of the row, call the decodeRow function on the key/value pair that this iterator returned.
- * 
+ * @deprecated since 1.4
+ * @see org.apache.accumulo.core.iterators.user.WholeRowIterator
  */
-public class WholeRowIterator implements SortedKeyValueIterator<Key,Value> {
-  
-  private SortedKeyValueIterator<Key,Value> sourceIter;
-  private Key topKey = null;
-  private Value topValue = null;
-  
-  public WholeRowIterator() {
-    
-  }
-  
-  WholeRowIterator(SortedKeyValueIterator<Key,Value> source) {
-    this.sourceIter = source;
-  }
-  
-  // decode a bunch of key value pairs that have been encoded into a single value
-  public static final SortedMap<Key,Value> decodeRow(Key rowKey, Value rowValue) throws IOException {
-    SortedMap<Key,Value> map = new TreeMap<Key,Value>();
-    ByteArrayInputStream in = new ByteArrayInputStream(rowValue.get());
-    DataInputStream din = new DataInputStream(in);
-    int numKeys = din.readInt();
-    for (int i = 0; i < numKeys; i++) {
-      byte[] cf;
-      byte[] cq;
-      byte[] cv;
-      byte[] valBytes;
-      // read the col fam
-      {
-        int len = din.readInt();
-        cf = new byte[len];
-        din.read(cf);
-      }
-      // read the col qual
-      {
-        int len = din.readInt();
-        cq = new byte[len];
-        din.read(cq);
-      }
-      // read the col visibility
-      {
-        int len = din.readInt();
-        cv = new byte[len];
-        din.read(cv);
-      }
-      // read the timestamp
-      long timestamp = din.readLong();
-      // read the value
-      {
-        int len = din.readInt();
-        valBytes = new byte[len];
-        din.read(valBytes);
-      }
-      map.put(new Key(rowKey.getRowData().toArray(), cf, cq, cv, timestamp, false, false), new Value(valBytes, false));
-    }
-    return map;
-  }
-  
-  // take a stream of keys and values and output a value that encodes everything but their row
-  // keys and values must be paired one for one
-  public static final Value encodeRow(List<Key> keys, List<Value> values) throws IOException {
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    DataOutputStream dout = new DataOutputStream(out);
-    dout.writeInt(keys.size());
-    for (int i = 0; i < keys.size(); i++) {
-      Key k = keys.get(i);
-      Value v = values.get(i);
-      // write the colfam
-      {
-        ByteSequence bs = k.getColumnFamilyData();
-        dout.writeInt(bs.length());
-        dout.write(bs.getBackingArray(), bs.offset(), bs.length());
-      }
-      // write the colqual
-      {
-        ByteSequence bs = k.getColumnQualifierData();
-        dout.writeInt(bs.length());
-        dout.write(bs.getBackingArray(), bs.offset(), bs.length());
-      }
-      // write the column visibility
-      {
-        ByteSequence bs = k.getColumnVisibilityData();
-        dout.writeInt(bs.length());
-        dout.write(bs.getBackingArray(), bs.offset(), bs.length());
-      }
-      // write the timestamp
-      dout.writeLong(k.getTimestamp());
-      // write the value
-      byte[] valBytes = v.get();
-      dout.writeInt(valBytes.length);
-      dout.write(valBytes);
-    }
-    
-    return new Value(out.toByteArray());
-  }
-  
-  List<Key> keys = new ArrayList<Key>();
-  List<Value> values = new ArrayList<Value>();
-  
-  private void prepKeys() throws IOException {
-    if (topKey != null)
-      return;
-    Text currentRow;
-    do {
-      if (sourceIter.hasTop() == false)
-        return;
-      currentRow = new Text(sourceIter.getTopKey().getRow());
-      keys.clear();
-      values.clear();
-      while (sourceIter.hasTop() && sourceIter.getTopKey().getRow().equals(currentRow)) {
-        keys.add(new Key(sourceIter.getTopKey()));
-        values.add(new Value(sourceIter.getTopValue()));
-        sourceIter.next();
-      }
-    } while (!filter(currentRow, keys, values));
-    
-    topKey = new Key(currentRow);
-    topValue = encodeRow(keys, values);
-    
-  }
-  
-  /**
-   * 
-   * @param currentRow
-   *          All keys have this in their row portion (do not modify!).
-   * @param keys
-   *          One key for each key in the row, ordered as they are given by the source iterator (do not modify!).
-   * @param values
-   *          One value for each key in keys, ordered to correspond to the ordering in keys (do not modify!).
-   * @return true if we want to keep the row, false if we want to skip it
-   */
-  protected boolean filter(Text currentRow, List<Key> keys, List<Value> values) {
-    return true;
-  }
-  
-  @Override
-  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    if (sourceIter != null)
-      return new WholeRowIterator(sourceIter.deepCopy(env));
-    return new WholeRowIterator();
-  }
-  
-  @Override
-  public Key getTopKey() {
-    return topKey;
-  }
-  
-  @Override
-  public Value getTopValue() {
-    return topValue;
-  }
-  
-  @Override
-  public boolean hasTop() {
-    return topKey != null || sourceIter.hasTop();
-  }
-  
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
-    sourceIter = source;
-  }
-  
-  @Override
-  public void next() throws IOException {
-    topKey = null;
-    topValue = null;
-    prepKeys();
-  }
-  
-  @Override
-  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
-    topKey = null;
-    topValue = null;
-    
-    Key sk = range.getStartKey();
-    
-    if (sk != null && sk.getColumnFamilyData().length() == 0 && sk.getColumnQualifierData().length() == 0 && sk.getColumnVisibilityData().length() == 0
-        && sk.getTimestamp() == Long.MAX_VALUE && !range.isStartKeyInclusive()) {
-      // assuming that we are seeking using a key previously returned by this iterator
-      // therefore go to the next row
-      Key followingRowKey = sk.followingKey(PartialKey.ROW);
-      if (range.getEndKey() != null && followingRowKey.compareTo(range.getEndKey()) > 0)
-        return;
-      
-      range = new Range(sk.followingKey(PartialKey.ROW), true, range.getEndKey(), range.isEndKeyInclusive());
-    }
-    
-    sourceIter.seek(range, columnFamilies, inclusive);
-    prepKeys();
-  }
+public class WholeRowIterator extends org.apache.accumulo.core.iterators.user.WholeRowIterator {
   
 }

Copied: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java (from r1207611, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java)
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java?p2=incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java&p1=incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java&r1=1207611&r2=1207621&rev=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/RowDeletingIterator.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java Mon Nov 28 21:24:26 2011
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.core.iterators;
+package org.apache.accumulo.core.iterators.user;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -26,6 +26,8 @@ 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.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 
 /**

Copied: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java (from r1207611, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java)
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java?p2=incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java&p1=incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java&r1=1207611&r2=1207621&rev=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/WholeRowIterator.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/WholeRowIterator.java Mon Nov 28 21:24:26 2011
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.core.iterators;
+package org.apache.accumulo.core.iterators.user;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -33,6 +33,8 @@ 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;
 import org.apache.hadoop.io.Text;
 
 /**

Modified: incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerOptionsTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerOptionsTest.java?rev=1207621&r1=1207620&r2=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerOptionsTest.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerOptionsTest.java Mon Nov 28 21:24:26 2011
@@ -19,7 +19,7 @@ package org.apache.accumulo.core.client.
 import static org.junit.Assert.assertEquals;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.iterators.WholeRowIterator;
+import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.junit.Test;
 
 /**

Modified: incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java?rev=1207621&r1=1207620&r2=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java Mon Nov 28 21:24:26 2011
@@ -33,7 +33,7 @@ import org.apache.accumulo.core.client.m
 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.WholeRowIterator;
+import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
@@ -121,7 +121,7 @@ public class AccumuloInputFormatTest {
     // Walk the list and make sure our settings are correct
     AccumuloIterator setting = list.get(0);
     assertEquals(1, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
+    assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator", setting.getIteratorClass());
     assertEquals("WholeRow", setting.getIteratorName());
     
     setting = list.get(1);

Modified: incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/LargeRowFilterTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/LargeRowFilterTest.java?rev=1207621&r1=1207620&r2=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/LargeRowFilterTest.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/LargeRowFilterTest.java Mon Nov 28 21:24:26 2011
@@ -30,7 +30,6 @@ 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.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.iterators.RowDeletingIteratorTest;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
 import org.apache.accumulo.core.util.LocalityGroupUtil;

Copied: incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java (from r1207611, incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/RowDeletingIteratorTest.java)
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java?p2=incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java&p1=incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/RowDeletingIteratorTest.java&r1=1207611&r2=1207621&rev=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/RowDeletingIteratorTest.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java Mon Nov 28 21:24:26 2011
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.core.iterators;
+package org.apache.accumulo.core.iterators.user;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -29,8 +29,13 @@ 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.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+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.RowDeletingIterator;
 import org.apache.hadoop.io.Text;
 
 public class RowDeletingIteratorTest extends TestCase {

Copied: incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/WholeRowIteratorTest.java (from r1207611, incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/WholeRowIteratorTest.java)
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/WholeRowIteratorTest.java?p2=incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/WholeRowIteratorTest.java&p1=incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/WholeRowIteratorTest.java&r1=1207611&r2=1207621&rev=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/WholeRowIteratorTest.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/WholeRowIteratorTest.java Mon Nov 28 21:24:26 2011
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.core.iterators;
+package org.apache.accumulo.core.iterators.user;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -33,8 +33,8 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
-import org.apache.accumulo.core.iterators.WholeRowIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
+import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.hadoop.io.Text;
 
 public class WholeRowIteratorTest extends TestCase {

Modified: incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java?rev=1207621&r1=1207620&r2=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java (original)
+++ incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java Mon Nov 28 21:24:26 2011
@@ -37,7 +37,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.WholeRowIterator;
+import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.security.SecurityConstants;

Modified: incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java?rev=1207621&r1=1207620&r2=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java (original)
+++ incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java Mon Nov 28 21:24:26 2011
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SkippingIterator;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.WholeRowIterator;
+import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.server.util.AddressUtil;
 import org.apache.commons.codec.binary.Base64;

Modified: incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/RowDeleteTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/RowDeleteTest.java?rev=1207621&r1=1207620&r2=1207621&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/RowDeleteTest.java (original)
+++ incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/RowDeleteTest.java Mon Nov 28 21:24:26 2011
@@ -28,8 +28,8 @@ import org.apache.accumulo.core.client.S
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.RowDeletingIterator;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.user.RowDeletingIterator;
 import org.apache.accumulo.core.util.UtilWaitThread;
 
 public class RowDeleteTest extends FunctionalTest {