You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2011/12/27 19:19:44 UTC

svn commit: r1224966 [4/10] - in /incubator/accumulo/branches/1.4: ./ contrib/accumulo_sample/ contrib/accumulo_sample/ingest/src/main/java/aggregator/ contrib/accumulo_sample/ingest/src/main/java/ingest/ contrib/accumulo_sample/ingest/src/test/java/ag...

Modified: incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicTreeNode.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicTreeNode.java?rev=1224966&r1=1224965&r2=1224966&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicTreeNode.java (original)
+++ incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/BooleanLogicTreeNode.java Tue Dec 27 18:19:43 2011
@@ -1,19 +1,19 @@
 /*
-* 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 iterator;
 
 import java.io.IOException;
@@ -25,499 +25,499 @@ import java.util.Iterator;
 import javax.swing.tree.DefaultMutableTreeNode;
 import javax.swing.tree.TreeNode;
 
+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.iterators.SortedKeyValueIterator;
 import org.apache.commons.jexl2.parser.ParserTreeConstants;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
 import parser.JexlOperatorConstants;
-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.iterators.SortedKeyValueIterator;
 
 /**
  *
  * 
  */
 public class BooleanLogicTreeNode extends DefaultMutableTreeNode {
-
-	private static final long serialVersionUID = 1L;
-	protected static final Logger log = Logger.getLogger(BooleanLogicTreeNode.class);
-    private Key myTopKey = null;
-    private Key advanceKey = null;
-    private Text fValue = null;
-    private Text fName = null;
-    private boolean negated = false;
-    private int type;
-    private boolean done = false;
-    private boolean valid = false;
-    private boolean rollUp = false;
-    private String fOperator = null;
-    private boolean childrenAllNegated = false;
-    private HashSet<Key> uids;
-    private Text upperBound;
-    private Text lowerBound;
-    private boolean rangeNode;
-
-    public BooleanLogicTreeNode() {
-        super();
-        uids = new HashSet<Key>();
-    }
-
-    public BooleanLogicTreeNode(int type) {
-        super();
-        this.type = type;
-        uids = new HashSet<Key>();
-        setOperator();
-    }
-
-    public BooleanLogicTreeNode(int type, boolean negate) {
-        super();
-        this.type = type;
-        this.negated = negate;
-        uids = new HashSet<Key>();
-        setOperator();
-    }
-
-    public BooleanLogicTreeNode(int type, String fieldName, String fieldValue) {
-        super();
-        this.type = type;
-        if (fieldValue != null) {
-            this.fValue = new Text(fieldValue);
-        }
-        if (fieldName != null) {
-            this.fName = new Text(fieldName);
-        }
-        uids = new HashSet<Key>();
-        setOperator();
-    }
-
-    public BooleanLogicTreeNode(int type, String fieldName, String fieldValue, boolean negated) {
-        super();
-        this.type = type;
-        if (fieldValue != null) {
-            this.fValue = new Text(fieldValue);
+  
+  private static final long serialVersionUID = 1L;
+  protected static final Logger log = Logger.getLogger(BooleanLogicTreeNode.class);
+  private Key myTopKey = null;
+  private Key advanceKey = null;
+  private Text fValue = null;
+  private Text fName = null;
+  private boolean negated = false;
+  private int type;
+  private boolean done = false;
+  private boolean valid = false;
+  private boolean rollUp = false;
+  private String fOperator = null;
+  private boolean childrenAllNegated = false;
+  private HashSet<Key> uids;
+  private Text upperBound;
+  private Text lowerBound;
+  private boolean rangeNode;
+  
+  public BooleanLogicTreeNode() {
+    super();
+    uids = new HashSet<Key>();
+  }
+  
+  public BooleanLogicTreeNode(int type) {
+    super();
+    this.type = type;
+    uids = new HashSet<Key>();
+    setOperator();
+  }
+  
+  public BooleanLogicTreeNode(int type, boolean negate) {
+    super();
+    this.type = type;
+    this.negated = negate;
+    uids = new HashSet<Key>();
+    setOperator();
+  }
+  
+  public BooleanLogicTreeNode(int type, String fieldName, String fieldValue) {
+    super();
+    this.type = type;
+    if (fieldValue != null) {
+      this.fValue = new Text(fieldValue);
+    }
+    if (fieldName != null) {
+      this.fName = new Text(fieldName);
+    }
+    uids = new HashSet<Key>();
+    setOperator();
+  }
+  
+  public BooleanLogicTreeNode(int type, String fieldName, String fieldValue, boolean negated) {
+    super();
+    this.type = type;
+    if (fieldValue != null) {
+      this.fValue = new Text(fieldValue);
+    }
+    if (fieldName != null) {
+      this.fName = new Text(fieldName);
+    }
+    uids = new HashSet<Key>();
+    this.negated = negated;
+    setOperator();
+  }
+  
+  public void setValid(boolean b) {
+    this.valid = b;
+  }
+  
+  public boolean isValid() {
+    return this.valid;
+  }
+  
+  public void setType(int t) {
+    this.type = t;
+  }
+  
+  public int getType() {
+    return this.type;
+  }
+  
+  public void setChildrenAllNegated(boolean childrenAllNegated) {
+    this.childrenAllNegated = childrenAllNegated;
+  }
+  
+  public boolean isChildrenAllNegated() {
+    return childrenAllNegated;
+  }
+  
+  public void setAdvanceKey(Key advanceKey) {
+    this.advanceKey = advanceKey;
+  }
+  
+  public Key getAdvanceKey() {
+    return advanceKey;
+  }
+  
+  public void setNegated(boolean b) {
+    this.negated = b;
+  }
+  
+  public boolean isNegated() {
+    return negated;
+  }
+  
+  public void setTopKey(Key id) {
+    this.myTopKey = id;
+  }
+  
+  public Key getTopKey() {
+    return myTopKey;
+  }
+  
+  public void setDone(boolean done) {
+    this.done = done;
+  }
+  
+  public boolean isDone() {
+    return done;
+  }
+  
+  public void setRollUp(boolean rollUp) {
+    this.rollUp = rollUp;
+  }
+  
+  public boolean isRollUp() {
+    return rollUp;
+  }
+  
+  public Text getFieldValue() {
+    return fValue;
+  }
+  
+  public void setFieldValue(Text term) {
+    this.fValue = term;
+  }
+  
+  public Text getFieldName() {
+    return fName;
+  }
+  
+  public void setFieldName(Text dataLocation) {
+    this.fName = dataLocation;
+  }
+  
+  public String getFieldOperator() {
+    return fOperator;
+  }
+  
+  private void setOperator() {
+    this.fOperator = JexlOperatorConstants.getOperator(type);
+    if (negated && this.fOperator.equals("!=")) {
+      this.fOperator = JexlOperatorConstants.getOperator(JexlOperatorConstants.JJTEQNODE);
+    }
+  }
+  
+  public Text getLowerBound() {
+    return lowerBound;
+  }
+  
+  public void setLowerBound(Text lowerBound) {
+    this.lowerBound = lowerBound;
+  }
+  
+  public Text getUpperBound() {
+    return upperBound;
+  }
+  
+  public void setUpperBound(Text upperBound) {
+    this.upperBound = upperBound;
+  }
+  
+  public boolean isRangeNode() {
+    return rangeNode;
+  }
+  
+  public void setRangeNode(boolean rangeNode) {
+    this.rangeNode = rangeNode;
+  }
+  
+  public String getContents() {
+    StringBuilder s = new StringBuilder("[");
+    s.append(toString());
+    
+    if (children != null) {
+      Enumeration<?> e = this.children();
+      while (e.hasMoreElements()) {
+        BooleanLogicTreeNode n = (BooleanLogicTreeNode) e.nextElement();
+        s.append(",");
+        s.append(n.getContents());
+      }
+    }
+    s.append("]");
+    return s.toString();
+  }
+  
+  public String printNode() {
+    StringBuilder s = new StringBuilder("[");
+    s.append("Full Location & Term = ");
+    if (this.fName != null) {
+      s.append(this.fName.toString());
+    } else {
+      s.append("BlankDataLocation");
+    }
+    s.append("  ");
+    if (this.fValue != null) {
+      s.append(this.fValue.toString());
+    } else {
+      s.append("BlankTerm");
+    }
+    s.append("]");
+    return s.toString();
+  }
+  
+  @Override
+  public String toString() {
+    String uidStr = "none";
+    if (myTopKey != null) {
+      String cf = myTopKey.getColumnFamily().toString();
+      
+      uidStr = cf;
+    }
+    switch (type) {
+      case ParserTreeConstants.JJTEQNODE:
+        return fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
+      case ParserTreeConstants.JJTNENODE:
+        return fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
+      case ParserTreeConstants.JJTERNODE:
+        return fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
+      case ParserTreeConstants.JJTNRNODE:
+        return fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
+      case ParserTreeConstants.JJTLENODE:
+        return "<=:" + fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
+      case ParserTreeConstants.JJTLTNODE:
+        return "<:" + fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
+      case ParserTreeConstants.JJTGENODE:
+        return ">=:" + fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
+      case ParserTreeConstants.JJTGTNODE:
+        return ">:" + fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
+      case ParserTreeConstants.JJTJEXLSCRIPT:
+        return "HEAD" + ":" + uidStr + ":" + isValid();
+      case ParserTreeConstants.JJTANDNODE:
+        return "AND" + ":" + uidStr + ":" + isValid();
+      case ParserTreeConstants.JJTNOTNODE:
+        return "NOT";
+      case ParserTreeConstants.JJTORNODE:
+        return "OR" + ":" + uidStr + ":" + isValid();
+      default:
+        System.out.println("Problem in BLTNODE.toString()");
+        return null;
+    }
+  }
+  
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+    
+    // always start fresh
+    this.setTopKey(null);
+    this.setDone(false);
+    
+    // get my user object which should be an iterator
+    SortedKeyValueIterator<?,?> iter = (SortedKeyValueIterator<?,?>) this.getUserObject();
+    if (iter != null) {
+      
+      iter.seek(range, columnFamilies, inclusive);
+      
+      if (iter.hasTop()) {
+        Key key = (Key) iter.getTopKey();
+        key = buildKey(key);
+        
+        this.setTopKey(key);
+        if (log.isDebugEnabled()) {
+          log.debug("BLTNODE.seek() -> found: " + this.getTopKey());
         }
-        if (fieldName != null) {
-            this.fName = new Text(fieldName);
+      } else {
+        if (log.isDebugEnabled()) {
+          log.debug("BLTNODE.seek() -> hasTop::false");
         }
-        uids = new HashSet<Key>();
-        this.negated = negated;
-        setOperator();
-    }
-
-    public void setValid(boolean b) {
-        this.valid = b;
-    }
-
-    public boolean isValid() {
-        return this.valid;
-    }
-
-    public void setType(int t) {
-        this.type = t;
-    }
-
-    public int getType() {
-        return this.type;
-    }
-
-    public void setChildrenAllNegated(boolean childrenAllNegated) {
-        this.childrenAllNegated = childrenAllNegated;
-    }
-
-    public boolean isChildrenAllNegated() {
-        return childrenAllNegated;
-    }
-
-    public void setAdvanceKey(Key advanceKey) {
-        this.advanceKey = advanceKey;
-    }
-
-    public Key getAdvanceKey() {
-        return advanceKey;
-    }
-
-    public void setNegated(boolean b) {
-        this.negated = b;
-    }
-
-    public boolean isNegated() {
-        return negated;
-    }
-
-    public void setTopKey(Key id) {
-        this.myTopKey = id;
-    }
-
-    public Key getTopKey() {
-        return myTopKey;
-    }
-
-    public void setDone(boolean done) {
-        this.done = done;
-    }
-
-    public boolean isDone() {
-        return done;
-    }
-
-    public void setRollUp(boolean rollUp) {
-        this.rollUp = rollUp;
-    }
-
-    public boolean isRollUp() {
-        return rollUp;
-    }
+        this.setDone(true);
+      }
+    } else {
+      if (log.isDebugEnabled()) {
+        log.debug("BLTNODE.seek(), The iterator was null!");
+      }
+      this.setTopKey(null);
+    }
+  }
+  
+  public String buildTreePathString(TreeNode[] path) {
+    StringBuilder s = new StringBuilder("[");
+    for (TreeNode p : path) {
+      s.append(p.toString());
+      s.append(",");
+    }
+    s.deleteCharAt(s.length() - 1);
+    s.append("]");
+    return s.toString();
+  }
+  
+  public void next() throws IOException {
     
-    public Text getFieldValue() {
-        return fValue;
-    }
-
-    public void setFieldValue(Text term) {
-        this.fValue = term;
-    }
-
-    public Text getFieldName() {
-        return fName;
-    }
-
-    public void setFieldName(Text dataLocation) {
-        this.fName = dataLocation;
-    }
-
-    public String getFieldOperator() {
-        return fOperator;
-    }
-
-    private void setOperator() {
-        this.fOperator = JexlOperatorConstants.getOperator(type);
-        if (negated && this.fOperator.equals("!=")) {
-            this.fOperator = JexlOperatorConstants.getOperator(JexlOperatorConstants.JJTEQNODE);
-        }
-    }
-
-    public Text getLowerBound() {
-        return lowerBound;
-    }
-
-    public void setLowerBound(Text lowerBound) {
-        this.lowerBound = lowerBound;
-    }
-
-    public Text getUpperBound() {
-        return upperBound;
-    }
-
-    public void setUpperBound(Text upperBound) {
-        this.upperBound = upperBound;
-    }
-
-    public boolean isRangeNode() {
-        return rangeNode;
-    }
-
-    public void setRangeNode(boolean rangeNode) {
-        this.rangeNode = rangeNode;
+    // always start fresh
+    this.setTopKey(null);
+    
+    if (log.isDebugEnabled()) {
+      TreeNode[] path = this.getPath();
+      log.debug("BLTNODE.next() path-> " + this.buildTreePathString(path));
     }
     
-    public String getContents() {
-        StringBuilder s = new StringBuilder("[");
-        s.append(toString());
-
-        if (children != null) {
-            Enumeration<?> e = this.children();
-            while (e.hasMoreElements()) {
-                BooleanLogicTreeNode n = (BooleanLogicTreeNode) e.nextElement();
-                s.append(",");
-                s.append(n.getContents());
-            }
-        }
-        s.append("]");
-        return s.toString();
+    // have I been marked as done?
+    if (this.isDone()) {
+      if (log.isDebugEnabled()) {
+        log.debug("I've been marked as done, returning");
+      }
+      return;
     }
-
-    public String printNode() {
-        StringBuilder s = new StringBuilder("[");
-        s.append("Full Location & Term = ");
-        if (this.fName != null) {
-            s.append(this.fName.toString());
-        } else {
-            s.append("BlankDataLocation");
-        }
-        s.append("  ");
-        if (this.fValue != null) {
-            s.append(this.fValue.toString());
-        } else {
-            s.append("BlankTerm");
-        }
-        s.append("]");
-        return s.toString();
+    
+    SortedKeyValueIterator<?,?> iter = (SortedKeyValueIterator<?,?>) this.getUserObject();
+    iter.next();
+    
+    if (iter.hasTop()) {
+      Key key = (Key) iter.getTopKey();
+      
+      // I have a valid topKey, pull out the piece I want
+      key = buildKey(key);
+      this.setTopKey(key);
+      
+      if (log.isDebugEnabled()) {
+        log.debug("BLTNODE.next() -> found: " + this.getTopKey());
+      }
+    } else {
+      // no top value has been returned, I'm done.
+      if (log.isDebugEnabled()) {
+        log.debug("BLTNODE.next() -> Nothing found");
+      }
+      this.setTopKey(null);
+      this.setDone(true);
     }
-
-    @Override
-    public String toString() {
-        String uidStr = "none";
-        if (myTopKey != null) {
-            String cf = myTopKey.getColumnFamily().toString();
-
-            uidStr = cf;
+    
+  }
+  
+  public boolean jump(Key jumpKey) throws IOException {
+    boolean ok = true;
+    if (this.getType() == ParserTreeConstants.JJTEQNODE) {
+      FieldIndexIterator iter = (FieldIndexIterator) this.getUserObject();
+      ok = iter.jump(jumpKey);
+      if (iter.hasTop()) {
+        Key key = (Key) iter.getTopKey();
+        key = buildKey(key);
+        
+        this.setTopKey(key);
+        if (log.isDebugEnabled()) {
+          log.debug("BLTNODE.jump() -> found: " + this.getTopKey());
         }
-        switch (type) {
-            case ParserTreeConstants.JJTEQNODE:
-                return fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
-            case ParserTreeConstants.JJTNENODE:
-                return fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
-            case ParserTreeConstants.JJTERNODE:
-                return fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
-            case ParserTreeConstants.JJTNRNODE:
-                return fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
-            case ParserTreeConstants.JJTLENODE:
-                return "<=:"+fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
-            case ParserTreeConstants.JJTLTNODE:
-                return "<:"+fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
-            case ParserTreeConstants.JJTGENODE:
-                return ">=:"+fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
-            case ParserTreeConstants.JJTGTNODE:
-                return ">:"+fName.toString() + ":" + fValue.toString() + ", uid=" + uidStr + " , negation=" + this.isNegated();
-            case ParserTreeConstants.JJTJEXLSCRIPT:
-                return "HEAD" + ":" + uidStr + ":" + isValid();
-            case ParserTreeConstants.JJTANDNODE:
-                return "AND" + ":" + uidStr + ":" + isValid();
-            case ParserTreeConstants.JJTNOTNODE:
-                return "NOT";
-            case ParserTreeConstants.JJTORNODE:
-                return "OR" + ":" + uidStr + ":" + isValid();
-            default:
-                System.out.println("Problem in BLTNODE.toString()");
-                return null;
+      } else {
+        if (log.isDebugEnabled()) {
+          log.debug("FieldIndexIteratorJexl does not have top after jump, marking done.");
         }
-    }
-
-    public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
-
-        // always start fresh
         this.setTopKey(null);
-        this.setDone(false);
-
-        // get my user object which should be an iterator
-        SortedKeyValueIterator<?,?> iter = (SortedKeyValueIterator<?,?>) this.getUserObject();
-        if (iter != null) {
-
-            iter.seek(range, columnFamilies, inclusive);
-
-            if (iter.hasTop()) {
-                Key key = (Key) iter.getTopKey();
-                key = buildKey(key);
-
-                this.setTopKey(key);
-                if (log.isDebugEnabled()) {
-                    log.debug("BLTNODE.seek() -> found: " + this.getTopKey());
-                }
-            } else {
-                if (log.isDebugEnabled()) {
-                    log.debug("BLTNODE.seek() -> hasTop::false");
-                }
-                this.setDone(true);
-            }
-        } else {
-            if (log.isDebugEnabled()) {
-                log.debug("BLTNODE.seek(), The iterator was null!");
-            }
-            this.setTopKey(null);
+        this.setDone(true);
+      }
+      
+    } else if (this.getType() == ParserTreeConstants.JJTANDNODE) {
+      AndIterator iter = (AndIterator) this.getUserObject();
+      ok = iter.jump(jumpKey);
+      if (iter.hasTop()) {
+        Key key = (Key) iter.getTopKey();
+        key = buildKey(key);
+        
+        this.setTopKey(key);
+        if (log.isDebugEnabled()) {
+          log.debug("BLTNODE.jump() -> found: " + this.getTopKey());
         }
-    }
-
-    public String buildTreePathString(TreeNode[] path) {
-        StringBuilder s = new StringBuilder("[");
-        for (TreeNode p : path) {
-            s.append(p.toString());
-            s.append(",");
+      } else {
+        if (log.isDebugEnabled()) {
+          log.debug("IntersectingIteratorJexl does not have top after jump, marking done.");
         }
-        s.deleteCharAt(s.length() - 1);
-        s.append("]");
-        return s.toString();
-    }
-
-    public void next() throws IOException {
-
-        // always start fresh
         this.setTopKey(null);
-
+        this.setDone(true);
+      }
+    } else if (this.getType() == ParserTreeConstants.JJTORNODE) {
+      OrIterator iter = (OrIterator) this.getUserObject();
+      ok = iter.jump(jumpKey);
+      if (iter.hasTop()) {
+        Key key = (Key) iter.getTopKey();
+        key = buildKey(key);
+        
+        this.setTopKey(key);
         if (log.isDebugEnabled()) {
-            TreeNode[] path = this.getPath();
-            log.debug("BLTNODE.next() path-> " + this.buildTreePathString(path));
-        }
-
-        // have I been marked as done?
-        if (this.isDone()) {
-            if(log.isDebugEnabled()){
-                log.debug("I've been marked as done, returning");
-            }
-            return;
-        }
-
-        SortedKeyValueIterator<?,?> iter = (SortedKeyValueIterator<?,?>) this.getUserObject();
-        iter.next();
-
-        if (iter.hasTop()) {
-            Key key = (Key) iter.getTopKey();
-
-            // I have a valid topKey, pull out the piece I want
-            key = buildKey(key);
-            this.setTopKey(key);
-
-            if (log.isDebugEnabled()) {
-                log.debug("BLTNODE.next() -> found: " + this.getTopKey());
-            }
-        } else {
-            // no top value has been returned, I'm done.
-            if (log.isDebugEnabled()) {
-                log.debug("BLTNODE.next() -> Nothing found");
-            }
-            this.setTopKey(null);
-            this.setDone(true);
+          log.debug("BLTNODE.jump() -> found: " + this.getTopKey());
         }
-
-    }
-
-    public boolean jump(Key jumpKey) throws IOException {
-        boolean ok = true;
-        if(this.getType() == ParserTreeConstants.JJTEQNODE){
-            FieldIndexIterator iter = (FieldIndexIterator) this.getUserObject();
-            ok = iter.jump(jumpKey);
-            if(iter.hasTop()){
-                Key key = (Key) iter.getTopKey();
-                key = buildKey(key);
-
-                this.setTopKey(key);
-                if (log.isDebugEnabled()) {
-                    log.debug("BLTNODE.jump() -> found: " + this.getTopKey());
-                }
-            }else{
-                if(log.isDebugEnabled()){
-                    log.debug("FieldIndexIteratorJexl does not have top after jump, marking done.");
-                }
-                this.setTopKey(null);
-                this.setDone(true);
-            }
-
-        }else if(this.getType() == ParserTreeConstants.JJTANDNODE){
-            AndIterator iter = (AndIterator) this.getUserObject();
-            ok = iter.jump(jumpKey);
-            if(iter.hasTop()){
-                Key key = (Key) iter.getTopKey();
-                key = buildKey(key);
-
-                this.setTopKey(key);
-                if (log.isDebugEnabled()) {
-                    log.debug("BLTNODE.jump() -> found: " + this.getTopKey());
-                }
-            }else{
-                if(log.isDebugEnabled()){
-                    log.debug("IntersectingIteratorJexl does not have top after jump, marking done.");
-                }
-                this.setTopKey(null);
-                this.setDone(true);
-            }
-        }else if(this.getType() == ParserTreeConstants.JJTORNODE){
-            OrIterator iter = (OrIterator) this.getUserObject();
-            ok = iter.jump(jumpKey);
-            if(iter.hasTop()){
-                Key key = (Key) iter.getTopKey();
-                key = buildKey(key);
-
-                this.setTopKey(key);
-                if (log.isDebugEnabled()) {
-                    log.debug("BLTNODE.jump() -> found: " + this.getTopKey());
-                }
-            }else{
-                if(log.isDebugEnabled()){
-                    log.debug("OrIteratorJexl does not have top after jump, marking done.");
-                }
-                this.setTopKey(null);
-                this.setDone(true);
-            }
-        }
-        return ok;
-    }
-
-    public void addToSet(Key i) {
-        uids.add(i);
-    }
-
-    public void reSet() {
-        uids = new HashSet<Key>();
-    }
-
-    public boolean inSet(Key t) {
-        return uids.contains(t);
-    }
-
-    public Iterator<Key> getSetIterator() {
-        return uids.iterator();
-    }
-
-    public HashSet<Key> getIntersection(HashSet<Key> h) {
-        h.retainAll(uids);
-        return h;
-    }
-
-    public Key getMinUniqueID() {
-        Iterator iter = uids.iterator();
-        Key min = null;
-        while (iter.hasNext()) {
-            Key t = (Key) iter.next();
-            if (log.isDebugEnabled()) {
-                log.debug("OR set member: " + t);
-            }
-            if (t != null) {
-                if (min == null) {
-                    min = t;
-                } else if (t.compareTo(min) < 0) {
-                    min = t;
-                }
-            }
+      } else {
+        if (log.isDebugEnabled()) {
+          log.debug("OrIteratorJexl does not have top after jump, marking done.");
         }
-        return min;
+        this.setTopKey(null);
+        this.setDone(true);
+      }
     }
-
-    public boolean hasTop() {
-        // This part really needs to be cleaned up.
-        // It was created before I knew what was being passed back.
-        if (this.getType() == ParserTreeConstants.JJTORNODE) {
-            // Are you a Logical OR or an OR Iterator
-            if(!this.isLeaf()){ // logical construct
-                //I have a set of keys
-                return this.uids.size() > 0;
-            } else { // or iterator, you only have possible key
-                if (this.getTopKey() == null) {
-                    return false;
-                } else {
-                    return true;
-                }
-            }
+    return ok;
+  }
+  
+  public void addToSet(Key i) {
+    uids.add(i);
+  }
+  
+  public void reSet() {
+    uids = new HashSet<Key>();
+  }
+  
+  public boolean inSet(Key t) {
+    return uids.contains(t);
+  }
+  
+  public Iterator<Key> getSetIterator() {
+    return uids.iterator();
+  }
+  
+  public HashSet<Key> getIntersection(HashSet<Key> h) {
+    h.retainAll(uids);
+    return h;
+  }
+  
+  public Key getMinUniqueID() {
+    Iterator<Key> iter = uids.iterator();
+    Key min = null;
+    while (iter.hasNext()) {
+      Key t = (Key) iter.next();
+      if (log.isDebugEnabled()) {
+        log.debug("OR set member: " + t);
+      }
+      if (t != null) {
+        if (min == null) {
+          min = t;
+        } else if (t.compareTo(min) < 0) {
+          min = t;
+        }
+      }
+    }
+    return min;
+  }
+  
+  public boolean hasTop() {
+    // This part really needs to be cleaned up.
+    // It was created before I knew what was being passed back.
+    if (this.getType() == ParserTreeConstants.JJTORNODE) {
+      // Are you a Logical OR or an OR Iterator
+      if (!this.isLeaf()) { // logical construct
+        // I have a set of keys
+        return this.uids.size() > 0;
+      } else { // or iterator, you only have possible key
+        if (this.getTopKey() == null) {
+          return false;
         } else {
-            return this.getTopKey() != null;
+          return true;
         }
+      }
+    } else {
+      return this.getTopKey() != null;
+    }
+  }
+  
+  public static Key buildKey(Key key) {
+    if (key == null) {
+      log.error("Problem in BooleanLogicTreeNodeJexl.buildKey");
+      return null;
+    }
+    // Build Key(Text row, Text colfam) where colFam is dataype\0uid
+    String[] cq = key.getColumnQualifier().toString().split("\0");
+    Text uuid = new Text(cq[cq.length - 2] + "\0" + cq[cq.length - 1]);
+    Text row = key.getRow();
+    if (log.isDebugEnabled()) {
+      log.debug("Key-> r:" + row + "  fam:" + uuid);
     }
-
-    public static Key buildKey(Key key) {
-        if (key == null) {
-            log.error("Problem in BooleanLogicTreeNodeJexl.buildKey");
-            return null;
-        }
-        // Build Key(Text row, Text colfam) where colFam is dataype\0uid
-        String[] cq = key.getColumnQualifier().toString().split("\0");
-        Text uuid = new Text(cq[cq.length - 2] + "\0" + cq[cq.length - 1]);
-        Text row = key.getRow();
-        if(log.isDebugEnabled()){
-            log.debug("Key-> r:"+row+"  fam:"+uuid);
-        }
-        //System.out.println("Key-> r:"+row+"  fam:"+uuid);
-
-        Key k = new Key(row, uuid);
-        //System.out.println("Key: "+k);
-        return k;
-    }
+    // System.out.println("Key-> r:"+row+"  fam:"+uuid);
+    
+    Key k = new Key(row, uuid);
+    // System.out.println("Key: "+k);
+    return k;
+  }
 }

Modified: incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/DefaultIteratorEnvironment.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/DefaultIteratorEnvironment.java?rev=1224966&r1=1224965&r2=1224966&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/DefaultIteratorEnvironment.java (original)
+++ incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/DefaultIteratorEnvironment.java Tue Dec 27 18:19:43 2011
@@ -1,19 +1,19 @@
 /*
-* 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 iterator;
 
 import java.io.IOException;
@@ -32,30 +32,27 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.util.CachedConfiguration;
 
 public class DefaultIteratorEnvironment implements IteratorEnvironment {
-
-	public Reader reserveMapFileReader(String mapFileName)
-			throws IOException {
-		Configuration conf = CachedConfiguration.getInstance();
-		FileSystem fs = FileSystem.get(conf);
-		return new MyMapFile.Reader(fs,mapFileName,conf);
-	}
-
-	public AccumuloConfiguration getConfig() {
-		return AccumuloConfiguration.getDefaultConfiguration();
-	}
-
-	public IteratorScope getIteratorScope() {
-		throw new UnsupportedOperationException();
-	}
-
-	public boolean isFullMajorCompaction() {
-		throw new UnsupportedOperationException();
-	}
-
-	public void registerSideChannel(SortedKeyValueIterator<Key, Value> iter) {
-		throw new UnsupportedOperationException();		
-	}
-	
-	
-
+  
+  public Reader reserveMapFileReader(String mapFileName) throws IOException {
+    Configuration conf = CachedConfiguration.getInstance();
+    FileSystem fs = FileSystem.get(conf);
+    return new MyMapFile.Reader(fs, mapFileName, conf);
+  }
+  
+  public AccumuloConfiguration getConfig() {
+    return AccumuloConfiguration.getDefaultConfiguration();
+  }
+  
+  public IteratorScope getIteratorScope() {
+    throw new UnsupportedOperationException();
+  }
+  
+  public boolean isFullMajorCompaction() {
+    throw new UnsupportedOperationException();
+  }
+  
+  public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {
+    throw new UnsupportedOperationException();
+  }
+  
 }

Modified: incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/EvaluatingIterator.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/EvaluatingIterator.java?rev=1224966&r1=1224965&r2=1224966&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/EvaluatingIterator.java (original)
+++ incubator/accumulo/branches/1.4/contrib/accumulo_sample/query/src/main/java/iterator/EvaluatingIterator.java Tue Dec 27 18:19:43 2011
@@ -1,88 +1,100 @@
 /*
-* 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 iterator;
 
-import parser.EventFields;
-import parser.EventFields.FieldValue;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.hadoop.io.Text;
 
-public class EvaluatingIterator extends AbstractEvaluatingIterator {
-
-	public static final String NULL_BYTE_STRING = "\u0000";
-
-	public EvaluatingIterator() {
-		super();
-	}
-
-	public EvaluatingIterator(AbstractEvaluatingIterator other, IteratorEnvironment env) {
-		super(other, env);
-	}
-
-	public SortedKeyValueIterator<Key, Value> deepCopy(IteratorEnvironment env) {
-		return new EvaluatingIterator(this, env);
-	}
-
-	@Override
-	public PartialKey getKeyComparator() {
-		return PartialKey.ROW_COLFAM;
-	}
-
-	@Override
-	public Key getReturnKey(Key k) {
-		//If we were using column visibility, then we would get the merged visibility here and use it in the key.
-		//Remove the COLQ from the key and use the combined visibility
-		Key r = new Key(k.getRowData().getBackingArray(), k.getColumnFamilyData().getBackingArray(),
-				NULL_BYTE, k.getColumnVisibility().getBytes(), k.getTimestamp(), k.isDeleted(), false);
-		return r;
-	}
-
-	@Override
-	public void fillMap(EventFields event, Key key, Value value) {
-		//If we were using column visibility, we would have to merge them here.
-
-		//Pull the datatype from the colf in case we need to do anything datatype specific.
-		//String colf = key.getColumnFamily().toString();
-		//String datatype = colf.substring(0, colf.indexOf(NULL_BYTE_STRING));
-
-		//For the partitioned table, the field name and field value are stored in the column qualifier
-		//separated by a \0.
-		String colq = key.getColumnQualifier().toString();//.toLowerCase();
-		int idx = colq.indexOf(NULL_BYTE_STRING);
-		String fieldName = colq.substring(0,idx);
-		String fieldValue = colq.substring(idx+1);
-
-		event.put(fieldName, new FieldValue(new ColumnVisibility(key.getColumnVisibility().getBytes()), fieldValue.getBytes()));
-	}
-
-
-	/**
-	 * Don't accept this key if the colf starts with 'fi'
-	 */
-	@Override
-	public boolean isKeyAccepted(Key key) {
-		if (key.getColumnFamily().toString().startsWith("fi")) {
-			return false;
-		}
-		return true;
-	}
+import parser.EventFields;
+import parser.EventFields.FieldValue;
 
+public class EvaluatingIterator extends AbstractEvaluatingIterator {
+  
+  public static final String NULL_BYTE_STRING = "\u0000";
+  
+  public EvaluatingIterator() {
+    super();
+  }
+  
+  public EvaluatingIterator(AbstractEvaluatingIterator other, IteratorEnvironment env) {
+    super(other, env);
+  }
+  
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    return new EvaluatingIterator(this, env);
+  }
+  
+  @Override
+  public PartialKey getKeyComparator() {
+    return PartialKey.ROW_COLFAM;
+  }
+  
+  @Override
+  public Key getReturnKey(Key k) {
+    // If we were using column visibility, then we would get the merged visibility here and use it in the key.
+    // Remove the COLQ from the key and use the combined visibility
+    Key r = new Key(k.getRowData().getBackingArray(), k.getColumnFamilyData().getBackingArray(), NULL_BYTE, k.getColumnVisibility().getBytes(),
+        k.getTimestamp(), k.isDeleted(), false);
+    return r;
+  }
+  
+  @Override
+  public void fillMap(EventFields event, Key key, Value value) {
+    // If we were using column visibility, we would have to merge them here.
+    
+    // Pull the datatype from the colf in case we need to do anything datatype specific.
+    // String colf = key.getColumnFamily().toString();
+    // String datatype = colf.substring(0, colf.indexOf(NULL_BYTE_STRING));
+    
+    // For the partitioned table, the field name and field value are stored in the column qualifier
+    // separated by a \0.
+    String colq = key.getColumnQualifier().toString();// .toLowerCase();
+    int idx = colq.indexOf(NULL_BYTE_STRING);
+    String fieldName = colq.substring(0, idx);
+    String fieldValue = colq.substring(idx + 1);
+    
+    event.put(fieldName, new FieldValue(new ColumnVisibility(key.getColumnVisibility().getBytes()), fieldValue.getBytes()));
+  }
+  
+  /**
+   * Don't accept this key if the colf starts with 'fi'
+   */
+  @Override
+  public boolean isKeyAccepted(Key key) throws IOException {
+    if (key.getColumnFamily().toString().startsWith("fi")) {
+      Key copy = new Key(key.getRow(), new Text("fi\01"));
+      Collection<ByteSequence> columnFamilies = Collections.emptyList();
+      this.iterator.seek(new Range(copy, copy), columnFamilies, true);
+      if (this.iterator.hasTop())
+        return isKeyAccepted(this.iterator.getTopKey());
+      return true;
+    }
+    return true;
+  }
+  
 }