You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by af...@apache.org on 2012/07/05 22:18:47 UTC

svn commit: r1357868 - in /accumulo/branches/ACCUMULO-652/core/src: main/java/org/apache/accumulo/core/file/rfile/ main/java/org/apache/accumulo/core/iterators/ main/java/org/apache/accumulo/core/iterators/predicates/ main/java/org/apache/accumulo/core...

Author: afuchs
Date: Thu Jul  5 20:18:46 2012
New Revision: 1357868

URL: http://svn.apache.org/viewvc?rev=1357868&view=rev
Log:
ACCUMULO-652 fixed bug in initial seek to index with filtering turned on, added more test cases, changed predicate objects to be grabbed through accessors, removed Filterer interface on WrappingIterator and added Filterer interface to Filter, removed generic filtering capability from RFile and moved it to GenericFilterer

Added:
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/GenericFilterer.java
Modified:
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockStats.java
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/ColumnVisibilityPredicate.java
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/TimestampRangePredicate.java
    accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
    accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java
    accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
    accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java
    accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java

Modified: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockStats.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockStats.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockStats.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockStats.java Thu Jul  5 20:18:46 2012
@@ -99,7 +99,6 @@ public class BlockStats implements Writa
       else {
         byte[] visibility = minimumVisibility.getExpression();
         if (visibility.length > maxVisibilityLength) {
-          System.out.println("expression too large: "+toString());
           out.writeInt(0);
         } else {
           out.writeInt(visibility.length);

Modified: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java Thu Jul  5 20:18:46 2012
@@ -514,9 +514,9 @@ public class MultiLevelIndex {
       }
       
       private final boolean checkFilterIndexEntry(IndexEntry ie) {
-        if(timestampFilter != null && (ie.blockStats.maxTimestamp < timestampFilter.startTimestamp || ie.blockStats.minTimestamp > timestampFilter.endTimestamp))
+        if(timestampFilter != null && (ie.blockStats.maxTimestamp < timestampFilter.getStartTimestamp() || ie.blockStats.minTimestamp > timestampFilter.getEndTimestamp()))
           return false;
-        if(columnVisibilityPredicate != null && ie.blockStats.minimumVisibility != null && ie.blockStats.minimumVisibility.evaluate(columnVisibilityPredicate.auths) == false)
+        if(columnVisibilityPredicate != null && ie.blockStats.minimumVisibility != null && ie.blockStats.minimumVisibility.evaluate(columnVisibilityPredicate.getAuthorizations()) == false)
           return false;
         return true;
       }
@@ -568,7 +568,7 @@ public class MultiLevelIndex {
             return;
           } else {
             if (top.block.level == 0) {
-              // found a matching index entry
+              // found a matching index entry -- set the pointer to be just before this location
               top.offset = pos - 1;
               return;
             } else {
@@ -596,12 +596,17 @@ public class MultiLevelIndex {
           } else {
             if (top.block.level == 0) {
               // success!
-              return;
+              break;
             }
             // go down
             position.add(new StackEntry(getIndexBlock(index.get(top.offset)), -1));
           }
         }
+        if (position.isEmpty())
+          return;
+        StackEntry e = position.peek();
+        nextEntry = e.block.getIndex().get(e.offset);
+        nextIndex = e.block.getOffset() + e.offset;
       }
       
       IndexEntry nextEntry = null;
@@ -616,11 +621,6 @@ public class MultiLevelIndex {
           } catch (IOException e) {
             throw new RuntimeException(e);
           }
-          if (position.isEmpty())
-            return;
-          StackEntry e = position.peek();
-          nextEntry = e.block.getIndex().get(e.offset);
-          nextIndex = e.block.getOffset() + e.offset;
         }
       }
       

Modified: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java Thu Jul  5 20:18:46 2012
@@ -794,7 +794,7 @@ public class RFile {
         TimestampRangePredicate p = (TimestampRangePredicate)filter;
         // intersect with previous timestampRange
         if(timestampRange != null)
-          timestampRange = new TimestampRangePredicate(Math.max(p.startTimestamp, timestampRange.startTimestamp), Math.min(p.endTimestamp, timestampRange.endTimestamp));
+          timestampRange = new TimestampRangePredicate(Math.max(p.getStartTimestamp(), timestampRange.getStartTimestamp()), Math.min(p.getEndTimestamp(), timestampRange.getEndTimestamp()));
         else
           timestampRange = p;
         index.setTimestampRange(timestampRange);
@@ -982,9 +982,6 @@ public class RFile {
     @Override
     public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
       
-      topKey = null;
-      topValue = null;
-      
       clear();
       
       numLGSeeked = 0;
@@ -1090,91 +1087,16 @@ public class RFile {
       }
     }
     
-    ArrayList<Predicate<Key,Value>> filters = new ArrayList<Predicate<Key,Value>>();
-    
     TimestampRangePredicate timestampFilter = null;
     ColumnVisibilityPredicate columnVisibilityPredicate = null;
     
-    Key topKey;
-    Value topValue;
-    
-    /* (non-Javadoc)
-     * @see org.apache.accumulo.core.iterators.system.HeapIterator#hasTop()
-     */
-    @Override
-    public boolean hasTop() {
-      if(topKey == null)
-      {
-        while(super.hasTop())
-        {
-          topKey = super.getTopKey();
-          topValue = super.getTopValue();
-          // check all the filters to see if we found a valid key/value pair
-          boolean keep = true;
-          for(Predicate<Key,Value> filter: filters)
-          {
-            if(!filter.evaluate(topKey, topValue))
-            {
-              keep = false;
-              try {
-                super.next();
-              } catch (IOException e) {
-                throw new RuntimeException(e);
-              }
-              break;
-            }
-          }
-          if(keep == true)
-            return true;
-        }
-        // ran out of key/value pairs
-        topKey = null;
-        topValue = null;
-        return false;
-      }
-      else
-      {
-        return true;
-      }
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.accumulo.core.iterators.system.HeapIterator#next()
-     */
-    @Override
-    public void next() throws IOException {
-      topKey = null;
-      topValue = null;
-      super.next();
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.accumulo.core.iterators.system.HeapIterator#getTopKey()
-     */
-    @Override
-    public Key getTopKey() {
-      if(topKey == null)
-        hasTop();
-      return topKey;
-    }
-    
-    /* (non-Javadoc)
-     * @see org.apache.accumulo.core.iterators.system.HeapIterator#getTopValue()
-     */
-    @Override
-    public Value getTopValue() {
-      if(topValue == null)
-        hasTop();
-      return topValue;
-    }
-    
     /* (non-Javadoc)
      * @see org.apache.accumulo.core.iterators.Filterer#applyFilter(org.apache.accumulo.core.iterators.Predicate)
      */
     @Override
     public void applyFilter(Predicate<Key,Value> filter, boolean required) {
       if(required)
-        filters.add(filter);
+        throw new IllegalArgumentException("RFile cannot guarantee filtering");
       // the HeapIterator will pass this filter on to its children, a collection of LocalityGroupReaders
       if(filter instanceof TimestampRangePredicate)
         this.timestampFilter = (TimestampRangePredicate)filter;
@@ -1191,9 +1113,6 @@ public class RFile {
     int max_cf = 10;
     int max_cq = 10;
     
-    // FSDataOutputStream fsout = fs.create(new Path("/tmp/test.rf"));
-    
-    // RFile.Writer w = new RFile.Writer(fsout, 1000, "gz", conf);
     CachableBlockFile.Writer _cbw = new CachableBlockFile.Writer(fs, new Path("/tmp/test.rf"), "gz", conf);
     RFile.Writer w = new RFile.Writer(_cbw, 100000);
     
@@ -1213,9 +1132,7 @@ public class RFile {
     }
     
     w.close();
-    // fsout.close();
-    
-    // Logger.getLogger("accumulo.core.file.rfile").setLevel(Level.DEBUG);
+
     long t1 = System.currentTimeMillis();
     FSDataInputStream fsin = fs.open(new Path("/tmp/test.rf"));
     long t2 = System.currentTimeMillis();

Modified: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java Thu Jul  5 20:18:46 2012
@@ -36,7 +36,7 @@ import org.apache.accumulo.core.data.Val
  * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
  * filter, thus iterating over entries that do not match its filter.
  */
-public abstract class Filter extends WrappingIterator implements OptionDescriber {
+public abstract class Filter extends WrappingIterator implements OptionDescriber, Filterer<Key,Value> {
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
     Filter newInstance;
@@ -117,4 +117,13 @@ public abstract class Filter extends Wra
   public static void setNegate(IteratorSetting is, boolean negate) {
     is.addOption(NEGATE, Boolean.toString(negate));
   }
+  
+  @SuppressWarnings("unchecked")
+  @Override
+  public void applyFilter(Predicate<Key,Value> filter, boolean required) {
+    if(getSource() instanceof Filterer)
+      ((Filterer<Key,Value>)getSource()).applyFilter(filter, required);
+    else if(required)
+      throw new IllegalArgumentException("Cannot require filter of underlying iterator");
+  }
 }

Modified: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java Thu Jul  5 20:18:46 2012
@@ -25,7 +25,7 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 
-public abstract class WrappingIterator implements SortedKeyValueIterator<Key,Value>, Filterer<Key,Value> {
+public abstract class WrappingIterator implements SortedKeyValueIterator<Key,Value> {
   
   private SortedKeyValueIterator<Key,Value> source = null;
   boolean seenSeek = false;
@@ -93,13 +93,4 @@ public abstract class WrappingIterator i
     seenSeek = true;
   }
   
-  @SuppressWarnings("unchecked")
-  @Override
-  public void applyFilter(Predicate<Key,Value> filter, boolean required) {
-    if(source instanceof Filterer)
-      ((Filterer<Key,Value>)source).applyFilter(filter, required);
-    else if(required)
-      throw new IllegalArgumentException("Cannot require filter of underlying iterator");
-  }
-
 }

Modified: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/ColumnVisibilityPredicate.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/ColumnVisibilityPredicate.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/ColumnVisibilityPredicate.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/ColumnVisibilityPredicate.java Thu Jul  5 20:18:46 2012
@@ -6,21 +6,25 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 
-public final class ColumnVisibilityPredicate implements Predicate<Key, Value> {
-	public final Authorizations auths;
-
-	public ColumnVisibilityPredicate(Authorizations auths)
-	{
-		this.auths = auths;
-	}
-	
-	@Override
-	public boolean evaluate(Key k, Value v) {
-		return new ColumnVisibility(k.getColumnVisibility()).evaluate(auths);
-	}
-	
-	@Override
-	public String toString() {
-	  return "{"+auths+"}";
-	}
+public final class ColumnVisibilityPredicate implements Predicate<Key,Value> {
+  
+  private final Authorizations auths;
+  
+  public Authorizations getAuthorizations() {
+    return auths;
+  }
+  
+  public ColumnVisibilityPredicate(Authorizations auths) {
+    this.auths = auths;
+  }
+  
+  @Override
+  public boolean evaluate(Key k, Value v) {
+    return new ColumnVisibility(k.getColumnVisibility()).evaluate(auths);
+  }
+  
+  @Override
+  public String toString() {
+    return "{" + auths + "}";
+  }
 }

Modified: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/TimestampRangePredicate.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/TimestampRangePredicate.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/TimestampRangePredicate.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/predicates/TimestampRangePredicate.java Thu Jul  5 20:18:46 2012
@@ -24,24 +24,30 @@ import org.apache.accumulo.core.iterator
  * TimestampRangeFilter is used to determine whether a Key/Value pair falls within a timestamp range
  */
 public class TimestampRangePredicate implements Predicate<Key,Value> {
-
-  public final long startTimestamp;
-  public final long endTimestamp;
   
+  private final long startTimestamp;
+  private final long endTimestamp;
+  
+  public long getStartTimestamp() {
+    return startTimestamp;
+  }
+  
+  public long getEndTimestamp() {
+    return endTimestamp;
+  }
   
   /**
-   * @param startTimestamp - inclusive first allowable timestamp
-   * @param endTimestamp - inclusive last allowable timestamp
+   * @param startTimestamp
+   *          - inclusive first allowable timestamp
+   * @param endTimestamp
+   *          - inclusive last allowable timestamp
    */
   public TimestampRangePredicate(long startTimestamp, long endTimestamp) {
     super();
     this.startTimestamp = startTimestamp;
     this.endTimestamp = endTimestamp;
   }
-
-  /* (non-Javadoc)
-   * @see org.apache.accumulo.core.iterators.Predicate#evaluate(java.lang.Object, java.lang.Object)
-   */
+  
   /**
    * return true IFF the key falls within the timestamp range
    */
@@ -53,6 +59,6 @@ public class TimestampRangePredicate imp
   
   @Override
   public String toString() {
-    return "{"+startTimestamp+"-"+endTimestamp+"}";
+    return "{" + startTimestamp + "-" + endTimestamp + "}";
   }
 }

Added: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/GenericFilterer.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/GenericFilterer.java?rev=1357868&view=auto
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/GenericFilterer.java (added)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/GenericFilterer.java Thu Jul  5 20:18:46 2012
@@ -0,0 +1,104 @@
+package org.apache.accumulo.core.iterators.system;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.Filterer;
+import org.apache.accumulo.core.iterators.Predicate;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+
+public class GenericFilterer extends WrappingIterator implements Filterer<Key,Value> {
+  
+  private ArrayList<Predicate<Key,Value>> filters = new ArrayList<Predicate<Key,Value>>();
+  
+  private Key topKey;
+  private Value topValue;
+  
+  public GenericFilterer(SortedKeyValueIterator<Key,Value> source) {
+    setSource(source);
+  }
+
+  public GenericFilterer() {
+  }
+  
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    topKey = null;
+    topValue = null;
+    super.seek(range, columnFamilies, inclusive);
+  }
+
+  @Override
+  public void next() throws IOException {
+    topKey = null;
+    topValue = null;
+    super.next();
+  }
+
+  @Override
+  public Key getTopKey() {
+    if(topKey == null)
+      hasTop();
+    return topKey;
+  }
+  
+  @Override
+  public Value getTopValue() {
+    if(topValue == null)
+      hasTop();
+    return topValue;
+  }
+  
+  @Override
+  public boolean hasTop() {
+    if(topKey == null)
+    {
+      while(super.hasTop())
+      {
+        topKey = super.getTopKey();
+        topValue = super.getTopValue();
+        // check all the filters to see if we found a valid key/value pair
+        boolean keep = true;
+        for(Predicate<Key,Value> filter: filters)
+        {
+          if(!filter.evaluate(topKey, topValue))
+          {
+            keep = false;
+            try {
+              super.next();
+            } catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+            break;
+          }
+        }
+        if(keep == true)
+          return true;
+      }
+      // ran out of key/value pairs
+      topKey = null;
+      topValue = null;
+      return false;
+    }
+    else
+    {
+      return true;
+    }
+  }
+  
+  @Override
+  public void applyFilter(Predicate<Key,Value> filter, boolean required) {
+    filters.add(filter);
+    if(getSource() instanceof Filterer)
+    {
+      Filterer<Key,Value> source = (Filterer<Key,Value>)getSource();
+      source.applyFilter(filter, false);
+    }
+  }
+}

Modified: accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java Thu Jul  5 20:18:46 2012
@@ -42,18 +42,20 @@ public class VisibilityFilter extends Fi
   private static final Logger log = Logger.getLogger(VisibilityFilter.class);
   
   public VisibilityFilter(SortedKeyValueIterator<Key,Value> iterator, Authorizations authorizations, byte[] defaultVisibility) {
-    setSource(iterator);
     this.auths = authorizations;
     this.defaultVisibility = new Text(defaultVisibility);
     this.cache = new LRUMap(1000);
     this.tmpVis = new Text();
+    if(iterator instanceof Filterer)
+      ((Filterer<Key,Value>)iterator).applyFilter(new ColumnVisibilityPredicate(auths), false);
+    else
+      throw new IllegalArgumentException("expected to get a "+Filterer.class.getSimpleName());
+    setSource(iterator);
   }
 
   @Override
   public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
-    super.init(source, options, env);
-    if(source instanceof Filterer)
-      ((Filterer<Key,Value>)source).applyFilter(new ColumnVisibilityPredicate(auths), false);
+    throw new UnsupportedOperationException();
   }
 
   @Override

Modified: accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java Thu Jul  5 20:18:46 2012
@@ -35,6 +35,7 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.predicates.ColumnVisibilityPredicate;
 import org.apache.accumulo.core.iterators.predicates.TimestampRangePredicate;
 import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.iterators.system.VisibilityFilter;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.hadoop.conf.Configuration;
@@ -97,12 +98,12 @@ public class AuthorizationFilterTest {
     CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length, conf);
     RFile.Reader reader = new RFile.Reader(_cbr);
     int count = 0;
-    reader.applyFilter(columnVisibilityPredicate,true);
-    reader.seek(new Range(), Collections.EMPTY_SET, false);
-    while (reader.hasTop()) {
+    VisibilityFilter vf = new VisibilityFilter(reader, auths, new byte[0]);
+    vf.seek(new Range(), Collections.EMPTY_SET, false);
+    while (vf.hasTop()) {
       count++;
-      assertTrue(columnVisibilityPredicate.evaluate(reader.getTopKey(), reader.getTopValue()));
-      reader.next();
+      assertTrue(columnVisibilityPredicate.evaluate(vf.getTopKey(), vf.getTopValue()));
+      vf.next();
     }
     assertEquals(expected, count);
   }

Modified: accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java Thu Jul  5 20:18:46 2012
@@ -20,8 +20,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.Random;
 
-import junit.framework.TestCase;
-
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.file.blockfile.ABlockWriter;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
@@ -32,14 +30,20 @@ import org.apache.accumulo.core.file.rfi
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.Reader.IndexIterator;
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.Writer;
 import org.apache.accumulo.core.file.rfile.RFileTest.SeekableByteArrayInputStream;
+import org.apache.accumulo.core.iterators.predicates.ColumnVisibilityPredicate;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.junit.Test;
+
+import static junit.framework.Assert.*;
 
-public class MultiLevelIndexTest extends TestCase {
+public class MultiLevelIndexTest {
   
+  @Test
   public void test1() throws Exception {
     
     runTest(500, 1);
@@ -111,4 +115,65 @@ public class MultiLevelIndexTest extends
     
   }
   
+  /**
+   * Test the behavior of seeking to a spot that the high-level index blocks
+   * say passes a filter, but the low level index blocks do not agree, forcing
+   * an index scan beyond the end of the first low-level block.
+   * @throws IOException
+   */
+  @Test
+  public void testIndexScanWithFilter() throws IOException
+  {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    FSDataOutputStream dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a"));
+    CachableBlockFile.Writer _cbw = new CachableBlockFile.Writer(dos, "gz", CachedConfiguration.getInstance());
+    
+    BufferedWriter mliw = new BufferedWriter(new Writer(_cbw, 1));
+    
+    // throw in a block stat with a visibility that is too big to serialize
+    mliw.add(new Key(String.format("%05d000", 0),"cf","cq","a",0), new BlockStats(0,0,new ColumnVisibility("abcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyz"),0), 0, 0, 0, RFile.RINDEX_VER_7);
+    for (int i = 1; i < 100; i++)
+    {
+      mliw.add(new Key(String.format("%05d000", i),"cf","cq","a",0), new BlockStats(0,0,new ColumnVisibility("a"),i), 0, 0, 0, RFile.RINDEX_VER_7);
+    }
+    mliw.add(new Key(String.format("%05d000", 100),"cf","cq","",0), new BlockStats(0,0,new ColumnVisibility(""),100), 0, 0, 0, RFile.RINDEX_VER_7);
+    for (int i = 101; i < 200; i++)
+    {
+      mliw.add(new Key(String.format("%05d000", i),"cf","cq","a",0), new BlockStats(0,0,new ColumnVisibility("a"),i), 0, 0, 0, RFile.RINDEX_VER_7);
+    }
+    mliw.addLast(new Key(String.format("%05d000", 200),"cf","cq","a",0), new BlockStats(0,0,new ColumnVisibility("a"),200), 0, 0, 0, RFile.RINDEX_VER_7);
+
+    ABlockWriter root = _cbw.prepareMetaBlock("root");
+    mliw.close(root);
+    root.close();
+    
+    _cbw.close();
+    dos.close();
+    baos.close();
+    
+    byte[] data = baos.toByteArray();
+    SeekableByteArrayInputStream bais = new SeekableByteArrayInputStream(data);
+    FSDataInputStream in = new FSDataInputStream(bais);
+    CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length, CachedConfiguration.getInstance());
+    
+    Reader reader = new Reader(_cbr, RFile.RINDEX_VER_7);
+    BlockRead rootIn = _cbr.getMetaBlock("root");
+    reader.readFields(rootIn);
+    rootIn.close();
+    reader.setColumnVisibilityPredicate(new ColumnVisibilityPredicate(new Authorizations()));
+    // seek past the block stat with the visibility that is too big to serialize, but not past the block that can be seen
+    IndexIterator liter = reader.lookup(new Key("000010"));
+    int count = 0;
+    while (liter.hasNext()) {
+      assertEquals(100, liter.nextIndex());
+      assertEquals(100, liter.peek().getNumEntries());
+      assertEquals(100, liter.next().getNumEntries());
+      count++;
+    }
+    
+    assertEquals(1, count);
+    
+    _cbr.close();
+  }
+  
 }

Modified: accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java Thu Jul  5 20:18:46 2012
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.file.rfi
 import org.apache.accumulo.core.iterators.Predicate;
 import org.apache.accumulo.core.iterators.predicates.TimestampRangePredicate;
 import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.iterators.system.GenericFilterer;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -82,14 +83,15 @@ public class TimestampFilterTest {
     FSDataInputStream in = new FSDataInputStream(bais);
     CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length, conf);
     RFile.Reader reader = new RFile.Reader(_cbr);
+    GenericFilterer filterer = new GenericFilterer(reader);
     int count = 0;
-    reader.applyFilter(timeRange,true);
-    reader.seek(new Range(), Collections.EMPTY_SET, false);
-    while(reader.hasTop())
+    filterer.applyFilter(timeRange,true);
+    filterer.seek(new Range(), Collections.EMPTY_SET, false);
+    while(filterer.hasTop())
     {
       count++;
-      assertTrue(timeRange.evaluate(reader.getTopKey(),reader.getTopValue()));
-      reader.next();
+      assertTrue(timeRange.evaluate(filterer.getTopKey(),filterer.getTopValue()));
+      filterer.next();
     }
     assertEquals(expected, count);
   }

Modified: accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java?rev=1357868&r1=1357867&r2=1357868&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java (original)
+++ accumulo/branches/ACCUMULO-652/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java Thu Jul  5 20:18:46 2012
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter;
+import org.apache.accumulo.core.iterators.system.GenericFilterer;
 import org.apache.accumulo.core.iterators.system.VisibilityFilter;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
@@ -305,7 +306,8 @@ public class FilterTest {
     }
     assertTrue(tm.size() == 1000);
     
-    VisibilityFilter a = new VisibilityFilter(new SortedMapIterator(tm), auths, le2.getExpression());
+    SortedKeyValueIterator<Key,Value> iter = new GenericFilterer(new SortedMapIterator(tm));
+    VisibilityFilter a = new VisibilityFilter(iter, auths, le2.getExpression());
     a.seek(new Range(), EMPTY_COL_FAMS, false);
     int size = size(a);
     assertTrue("size was " + size, size == 750);