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 2013/11/26 16:49:56 UTC
[08/40] ACCUMULO-600 removed wikisearch from trunk
http://git-wip-us.apache.org/repos/asf/accumulo/blob/8db62992/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/BooleanLogicTreeNode.java
----------------------------------------------------------------------
diff --git a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/BooleanLogicTreeNode.java b/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/BooleanLogicTreeNode.java
deleted file mode 100644
index ccc5669..0000000
--- a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/BooleanLogicTreeNode.java
+++ /dev/null
@@ -1,523 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.examples.wikisearch.iterator;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Enumeration;
-import java.util.HashSet;
-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.accumulo.examples.wikisearch.parser.JexlOperatorConstants;
-import org.apache.commons.jexl2.parser.ParserTreeConstants;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Logger;
-
-
-/**
- *
- *
- */
-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);
- }
- 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());
- }
- } 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);
- }
- }
-
- 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 {
-
- // always start fresh
- this.setTopKey(null);
-
- 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);
- }
-
- }
-
- 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<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 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);
- }
- // System.out.println("Key-> r:"+row+" fam:"+uuid);
-
- Key k = new Key(row, uuid);
- // System.out.println("Key: "+k);
- return k;
- }
-}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/8db62992/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/DefaultIteratorEnvironment.java
----------------------------------------------------------------------
diff --git a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/DefaultIteratorEnvironment.java b/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/DefaultIteratorEnvironment.java
deleted file mode 100644
index 353ce79..0000000
--- a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/DefaultIteratorEnvironment.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.examples.wikisearch.iterator;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.map.MyMapFile;
-import org.apache.accumulo.core.file.map.MyMapFile.Reader;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-
-@SuppressWarnings("deprecation")
-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();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/8db62992/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/EvaluatingIterator.java
----------------------------------------------------------------------
diff --git a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/EvaluatingIterator.java b/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/EvaluatingIterator.java
deleted file mode 100644
index d51023c..0000000
--- a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/EvaluatingIterator.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.examples.wikisearch.iterator;
-
-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.accumulo.examples.wikisearch.parser.EventFields;
-import org.apache.accumulo.examples.wikisearch.parser.EventFields.FieldValue;
-import org.apache.commons.collections.map.LRUMap;
-import org.apache.hadoop.io.Text;
-
-
-public class EvaluatingIterator extends AbstractEvaluatingIterator {
-
- public static final String NULL_BYTE_STRING = "\u0000";
- LRUMap visibilityMap = new LRUMap();
-
- 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(getColumnVisibility(key), fieldValue.getBytes()));
- }
-
- /**
- * @param key
- * @return
- */
- public ColumnVisibility getColumnVisibility(Key key) {
- ColumnVisibility result = (ColumnVisibility) visibilityMap.get(key.getColumnVisibility());
- if (result != null)
- return result;
- result = new ColumnVisibility(key.getColumnVisibility().getBytes());
- visibilityMap.put(key.getColumnVisibility(), result);
- return result;
- }
-
- /**
- * 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;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/8db62992/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/FieldIndexIterator.java
----------------------------------------------------------------------
diff --git a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/FieldIndexIterator.java b/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/FieldIndexIterator.java
deleted file mode 100644
index ad39ab3..0000000
--- a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/FieldIndexIterator.java
+++ /dev/null
@@ -1,736 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.examples.wikisearch.iterator;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.WrappingIterator;
-import org.apache.accumulo.examples.wikisearch.function.QueryFunctions;
-import org.apache.accumulo.examples.wikisearch.util.FieldIndexKeyParser;
-import org.apache.commons.jexl2.Expression;
-import org.apache.commons.jexl2.JexlContext;
-import org.apache.commons.jexl2.JexlEngine;
-import org.apache.commons.jexl2.MapContext;
-import org.apache.commons.jexl2.parser.ParserTreeConstants;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * This iterator should only return keys from the fi\0{fieldName}:{fieldValue} part of the shard table. Expect topKey to be CF, {datatype}\0{UID}
- */
-public class FieldIndexIterator extends WrappingIterator {
-
- private Key topKey = null;
- private Value topValue = null;
- private Range range = null;
- private Text currentRow;
- private Text fName = null;
- private String fNameString = null;
- private Text fValue = null;
- private String fOperator = null;
- private Expression expr = null;
- private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<ByteSequence>();
- protected static final Logger log = Logger.getLogger(FieldIndexIterator.class);
- private boolean negated = false;
- private int type;
- private static final String NULL_BYTE = "\0";
- private static final String ONE_BYTE = "\1";
- // According to the JEXL 2.0 docs, the engine is thread-safe. Let's create 1 engine per VM and
- // cache 128 expressions
- private static JexlEngine engine = new JexlEngine();
- private Range parentRange;
- private Text parentEndRow = null;
- private FieldIndexKeyParser keyParser;
-
- static {
- engine.setCache(128);
- Map<String,Object> functions = new HashMap<String,Object>();
- functions.put("f", QueryFunctions.class);
- engine.setFunctions(functions);
- }
-
- public static void setLogLevel(Level l) {
- log.setLevel(l);
- }
-
- // -------------------------------------------------------------------------
- // ------------- Constructors
- public FieldIndexIterator() {}
-
- public FieldIndexIterator(int type, Text rowId, Text fieldName, Text fieldValue, String operator) {
- this.fName = fieldName;
- this.fNameString = fName.toString().substring(3);
- this.fValue = fieldValue;
- this.fOperator = operator;
- this.range = buildRange(rowId);
- this.negated = false;
- this.type = type;
-
- // Create the Jexl expression, we need to add the ' around the field value
- StringBuilder buf = new StringBuilder();
- buf.append(fNameString).append(" ").append(this.fOperator).append(" ").append("'").append(fValue.toString()).append("'");
- this.expr = engine.createExpression(buf.toString());
-
- // Set a default KeyParser
- keyParser = createDefaultKeyParser();
- }
-
- public FieldIndexIterator(int type, Text rowId, Text fieldName, Text fieldValue, boolean neg, String operator) {
- this.fName = fieldName;
- this.fNameString = fName.toString().substring(3);
- this.fValue = fieldValue;
- this.fOperator = operator;
- this.range = buildRange(rowId);
- this.negated = neg;
- this.type = type;
-
- // Create the Jexl expression, we need to add the ' around the field value
- StringBuilder buf = new StringBuilder();
- buf.append(fNameString).append(" ").append(this.fOperator).append(" ").append("'").append(fValue.toString()).append("'");
- this.expr = engine.createExpression(buf.toString());
- // Set a default KeyParser
- keyParser = createDefaultKeyParser();
- }
-
- public FieldIndexIterator(FieldIndexIterator other, IteratorEnvironment env) {
- setSource(other.getSource().deepCopy(env));
- // Set a default KeyParser
- keyParser = createDefaultKeyParser();
- }
-
- private FieldIndexKeyParser createDefaultKeyParser() {
- FieldIndexKeyParser parser = new FieldIndexKeyParser();
- return parser;
- }
-
- @Override
- public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new FieldIndexIterator(this, env);
- }
-
- @Override
- public Key getTopKey() {
- return topKey;
- }
-
- @Override
- public Value getTopValue() {
- return topValue;
- }
-
- @Override
- public boolean hasTop() {
- return (topKey != null);
- }
-
- @Override
- public void next() throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("next()");
- }
- if (this.hasTop()) {
- currentRow = topKey.getRow();
- }
-
- getSource().next();
- while (true) {
- log.debug("next(), Range: " + range);
- if (getSource().hasTop()) {
- Key k = getSource().getTopKey();
- if (range.contains(k)) {
- if (matches(k)) {
- topKey = k;
- topValue = getSource().getTopValue();
- return;
- } else {
- getSource().next();
- }
-
- } else {
-
- if (parentEndRow != null) { // need to check it
- if (k.getRow().equals(currentRow)) {
- currentRow = getNextRow();
- } else if (currentRow == null || k.getRow().compareTo(currentRow) > 0) {
- currentRow = k.getRow();
- }
-
- if (currentRow == null || parentEndRow.compareTo(currentRow) < 0) {
- // you're done
- topKey = null;
- topValue = null;
- return;
- }
-
- } else { // we can go to end of the tablet
- if (k.getRow().equals(currentRow)) {
- currentRow = getNextRow();
- if (currentRow == null) {
- topKey = null;
- topValue = null;
- return;
- }
- } else if (currentRow == null || (k.getRow().compareTo(currentRow) > 0)) {
- currentRow = k.getRow();
- }
- }
-
- // construct new range and seek the source
- range = buildRange(currentRow);
- if (log.isDebugEnabled()) {
- log.debug("next, range: " + range);
- }
- getSource().seek(range, EMPTY_COL_FAMS, false);
- }
- } else {
- topKey = null;
- topValue = null;
- return;
- }
- }
- }
-
- /*
- * NOTE: there is some special magic here with range modification. If it's negated, assume the range is explicitly set and don't mess with it (this is how
- * it's called by the BooleanLogicIterator) Otherwise, modify the range to start at the beginning and set an explicit end point.
- *
- * In the future, maybe all we need to do is look for an endKey and modifying that.
- */
- @Override
- public void seek(Range r, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
- parentRange = r;
- if (log.isDebugEnabled()) {
- log.debug("begin seek, range: " + r);
- }
- if (parentRange.getEndKey() != null) {
- if (parentRange.getEndKey().getRow() != null) {
- parentEndRow = parentRange.getEndKey().getRow();
- if (log.isDebugEnabled()) {
- log.debug("begin seek, parentEndRow: " + parentEndRow);
- }
- }
- }
-
- try {
- if (isNegated()) {
- range = r;
- if (log.isDebugEnabled()) {
- log.debug("seek, negation, skipping range modification.");
- }
- } else {
- if (r.getStartKey() != null) {
- if (r.getStartKey().getRow() == null || r.getStartKey().getRow().toString().isEmpty()) {
- currentRow = getFirstRow();
- } else {
- currentRow = r.getStartKey().getRow();
- }
- this.range = buildRange(currentRow);
- } else {
- currentRow = getFirstRow();
- this.range = buildRange(currentRow);
- }
- }
-
- setTopKey(null);
- setTopValue(null);
-
- if (log.isDebugEnabled()) {
- log.debug("seek, incoming range: " + range);
- }
- getSource().seek(range, columnFamilies, inclusive);
-
- while (topKey == null) {
- if (getSource().hasTop()) {
- if (log.isDebugEnabled()) {
- log.debug("seek, source has top: " + getSource().getTopKey());
- }
- Key k = getSource().getTopKey();
- if (range.contains(k)) {
- if (matches(k)) {
- topKey = k;
- topValue = getSource().getTopValue();
- if (log.isDebugEnabled()) {
- log.debug("seek, source has top in valid range");
- }
- } else {
- getSource().next();
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug("seek, top out of range");
- String pEndRow = "empty";
- if (parentEndRow != null) {
- pEndRow = parentEndRow.toString();
- }
- log.debug("source.topKey.row: " + k.getRow() + "\t currentRow: " + currentRow + "\t parentEndRow: " + pEndRow);
- }
- if (isNegated()) {
- topKey = null;
- topValue = null;
- return;
- }
-
- if (parentEndRow != null) {
- // check it
- if (k.getRow().equals(currentRow)) {
- currentRow = getNextRow();
- }
-
- if (currentRow == null || parentEndRow.compareTo(currentRow) < 0) {
- // you're done
- topKey = null;
- topValue = null;
- return;
- }
-
- } else { // can go to end of the tablet
- if (k.getRow().equals(currentRow)) {
- currentRow = getNextRow();
- if (currentRow == null) {
- topKey = null;
- topValue = null;
- return;
- }
- }
- }
-
- // construct new range and seek the source
- range = buildRange(currentRow);
- if (log.isDebugEnabled()) {
- log.debug("currentRow: " + currentRow);
- log.debug("seek, range: " + range);
- }
- getSource().seek(range, columnFamilies, inclusive);
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug("seek, underlying source had no top key.");
- }
- topKey = null;
- topValue = null;
- return;
- }
- }
- if (log.isDebugEnabled()) {
- log.debug("seek, topKey found: " + topKey);
- }
- } catch (IOException e) {
- topKey = null;
- topValue = null;
- throw new IOException();
- }
- }
-
- // -------------------------------------------------------------------------
- // ------------- Public stuff
- public boolean isNegated() {
- return negated;
- }
-
- public Text getCurrentRow() {
- return currentRow;
- }
-
- public Text getfName() {
- return fName;
- }
-
- public Text getfValue() {
- return fValue;
- }
-
- // works like seek, but we need to avoid range issues.
- public boolean jump(Key jumpKey) throws IOException {
- if (log.isDebugEnabled()) {
- String pEndRow = "empty";
- if (parentEndRow != null) {
- pEndRow = parentEndRow.toString();
- }
- log.debug("jump, current range: " + range + " parentEndRow is: " + pEndRow);
-
- }
-
- if (parentEndRow != null && jumpKey.getRow().compareTo(parentEndRow) > 0) {
- // can't go there.
- if (log.isDebugEnabled()) {
- log.debug("jumpRow: " + jumpKey.getRow() + " is greater than my parentEndRow: " + parentEndRow);
- }
- return false;
- }
-
- int comp;
- if (!this.hasTop()) {
- if (log.isDebugEnabled()) {
- log.debug("current row: " + this.currentRow);
- }
-
- /*
- * if I don't have a top, then I should be out of my range for my current row. Need to check parent range to see if I'm supposed to continue to next row
- * or not. Current row can be null because maybe I never found anything in this row.
- */
-
- if (parentEndRow != null) {
- // if jumpKey row is greater than parentEndRow, stop
- if (jumpKey.getRow().compareTo(parentEndRow) > 0) {
- if (log.isDebugEnabled()) {
- log.debug("jumpKey row is greater than my parentEndRow, done");
- }
- return false;
- }
-
- // if my current row is null, I must have hit the end of the tablet
- if (currentRow == null) {
- if (log.isDebugEnabled()) {
- log.debug("I have parentEndRow, but no current row, must have hit end of tablet, done");
- }
- return false;
- }
-
- // if my current row is greater than jump row stop, a seek will be
- // called to get me going again. If my row is equal, but i don't
- // have a topkey, i'm done
- if (currentRow.compareTo(jumpKey.getRow()) >= 0) {
- if (log.isDebugEnabled()) {
- log.debug("I have parentEndRow, but topKey, and my currentRow is >= jumpRow, done");
- }
- return false;
- }
-
- } else { // we're allowed to go to the end of the tablet
- // if my current row is null, I must have hit the end of the tablet
- if (currentRow == null) {
- if (log.isDebugEnabled()) {
- log.debug("no parentEndRow and current Row is null, must have hit end of tablet, done");
- }
- return false;
- }
-
- if (currentRow.compareTo(jumpKey.getRow()) >= 0) {
- // i'm past or equal to the jump point and have no top,
- // jumping's not going to help
- if (log.isDebugEnabled()) {
- log.debug("no parentEndRow, no topKey, and currentRow is >= jumpRow, done");
- }
- return false;
- }
- }
-
- // ok, jumpKey is ahead of me I'll mark it and allow the normal
- // flow to jump there and see if I have top.
- if (log.isDebugEnabled()) {
- log.debug("no topKey, but jumpRow is ahead and I'm allowed to go to it, marking");
- }
- comp = -1;
-
- } else { // I have a topKey, I can do the normal comparisons
- if (log.isDebugEnabled()) {
- log.debug("have top, can do normal comparisons");
- }
- comp = this.topKey.getRow().compareTo(jumpKey.getRow());
- }
-
- // ------------------
- // compare rows
- if (comp > 0) { // my row is ahead of jump key
- if (canBeInNextRow()) {
- if (log.isDebugEnabled()) {
- log.debug("I'm ahead of jump row & it's ok.");
- log.debug("jumpRow: " + jumpKey.getRow() + " myRow: " + topKey.getRow() + " parentEndRow: " + parentEndRow);
- }
- return true;
- } else {
- if (log.isDebugEnabled()) {
- log.debug("I'm ahead of jump row & can't be here, or at end of tablet.");
- }
- topKey = null;
- topValue = null;
- return false;
- }
-
- } else if (comp < 0) { // a row behind jump key, need to move forward
- if (log.isDebugEnabled()) {
- String myRow = "";
- if (hasTop()) {
- myRow = topKey.getRow().toString();
- } else if (currentRow != null) {
- myRow = currentRow.toString();
- }
- log.debug("My row " + myRow + " is less than jump row: " + jumpKey.getRow() + " seeking");
- }
- range = buildRange(jumpKey.getRow());
- // this.seek(range, EMPTY_COL_FAMS, false);
-
- boolean success = jumpSeek(range);
- if (log.isDebugEnabled() && success) {
- log.debug("uid forced jump, found topKey: " + topKey);
- }
-
- if (!this.hasTop()) {
- log.debug("seeked with new row and had no top");
- topKey = null;
- topValue = null;
- return false;
- } else if (parentEndRow != null && currentRow.compareTo(parentEndRow) > 0) {
- if (log.isDebugEnabled()) {
- log.debug("myRow: " + getTopKey().getRow() + " is past parentEndRow: " + parentEndRow);
- }
- topKey = null;
- topValue = null;
- return false;
- }
- if (log.isDebugEnabled()) {
- log.debug("jumped, valid top: " + getTopKey());
- }
-
- return true;
-
- } else { // rows are equal, check the uid!
-
- keyParser.parse(topKey);
- String myUid = keyParser.getUid();
- keyParser.parse(jumpKey);
- String jumpUid = keyParser.getUid();
-
- int ucomp = myUid.compareTo(jumpUid);
- if (log.isDebugEnabled()) {
- log.debug("topKeyUid: " + myUid + " jumpUid: " + jumpUid + " myUid.compareTo(jumpUid)->" + ucomp);
- }
- if (ucomp < 0) { // need to move up
- log.debug("my uid is less than jumpUid, topUid: " + myUid + " jumpUid: " + jumpUid);
-
- Text cq = jumpKey.getColumnQualifier();
- int index = cq.find(NULL_BYTE);
- if (0 <= index) {
- cq.set(cq.getBytes(), index + 1, cq.getLength() - index - 1);
- } else {
- log.error("Expected a NULL separator in the column qualifier");
- this.topKey = null;
- this.topValue = null;
- return false;
- }
-
- // note my internal range stays the same, I just need to move forward
- Key startKey = new Key(topKey.getRow(), fName, new Text(fValue + NULL_BYTE + cq));
- Key endKey = new Key(topKey.getRow(), fName, new Text(fValue + ONE_BYTE));
- range = new Range(startKey, true, endKey, false);
- log.debug("Using range: " + range + " to seek");
- // source.seek(range, EMPTY_COL_FAMS, false);
- boolean success = jumpSeek(range);
- if (log.isDebugEnabled() && success) {
- log.debug("uid forced jump, found topKey: " + topKey);
- }
-
- return success;
-
- } else { // else do nothing
- log.debug("my uid is greater than jumpUid, topKey: " + topKey + " jumpKey: " + jumpKey);
- log.debug("doing nothing");
- }
- }
-
- return hasTop();
- }
-
- // -------------------------------------------------------------------------
- // ------------- Private stuff, KEEP OUT!!
- private void setTopKey(Key key) {
- topKey = key;
- }
-
- private void setTopValue(Value v) {
- this.topValue = v;
- }
-
- private boolean canBeInNextRow() {
- if (parentEndRow == null) {
- return true;
- } else if (currentRow == null) {
- return false;
- } else if (currentRow.compareTo(parentEndRow) <= 0) {
- return true;
- } else {
- return false;
- }
- }
-
- private Range buildRange(Text rowId) {
- if (type == ParserTreeConstants.JJTGTNODE || type == ParserTreeConstants.JJTGENODE || type == ParserTreeConstants.JJTLTNODE
- || type == ParserTreeConstants.JJTLENODE || type == ParserTreeConstants.JJTERNODE || type == ParserTreeConstants.JJTNRNODE) {
- Key startKey = new Key(rowId, fName);
- Key endKey = new Key(rowId, new Text(fName + NULL_BYTE));
- return (new Range(startKey, true, endKey, false));
- } else {
- // construct new range
- Key startKey = new Key(rowId, fName, new Text(fValue + NULL_BYTE));
- Key endKey = new Key(rowId, fName, new Text(fValue + ONE_BYTE));
- return (new Range(startKey, true, endKey, false));
- }
- }
-
- // need to build a range starting at the end of current row and seek the
- // source to it. If we get an IOException, that means we hit the end of the tablet.
- private Text getNextRow() throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("getNextRow()");
- }
- Key fakeKey = new Key(new Text(currentRow + NULL_BYTE));
- Range fakeRange = new Range(fakeKey, fakeKey);
- getSource().seek(fakeRange, EMPTY_COL_FAMS, false);
- if (getSource().hasTop()) {
- return getSource().getTopKey().getRow();
- } else {
- return null;
- }
- }
-
- private Text getFirstRow() throws IOException {
- getSource().seek(new Range(), EMPTY_COL_FAMS, false);
- if (getSource().hasTop()) {
- return getSource().getTopKey().getRow();
- } else {
- throw new IOException();
- }
- }
-
- private boolean matches(Key k) {
- if (log.isDebugEnabled()) {
- log.debug("You've reached the match function!");
- }
- JexlContext ctx = new MapContext();
- // Add the field value from the key to the context
- // String fieldValue = k.getColumnQualifier().toString().split("\0")[0];
- // String fieldValue = getFieldValueFromKey(k);
- keyParser.parse(k);
- String fieldValue = keyParser.getFieldValue();
-
- ctx.set(fNameString, fieldValue);
- Object o = expr.evaluate(ctx);
- if (o instanceof Boolean && (((Boolean) o) == true)) {
- if (log.isDebugEnabled()) {
- log.debug("matches:: fName: " + fName + " , fValue: " + fieldValue + " , operator: " + fOperator + " , key: " + k);
- }
- return true;
- } else {
- if (log.isDebugEnabled()) {
- log.debug("NO MATCH:: fName: " + fName + " , fValue: " + fieldValue + " , operator: " + fOperator + " , key: " + k);
- }
- return false;
- }
- }
-
- private boolean jumpSeek(Range r) throws IOException {
- range = r;
- setTopKey(null);
- setTopValue(null);
- getSource().seek(range, EMPTY_COL_FAMS, false);
- while (topKey == null) {
- if (getSource().hasTop()) {
- if (log.isDebugEnabled()) {
- log.debug("jump, source has top: " + getSource().getTopKey());
- }
- Key k = getSource().getTopKey();
- if (range.contains(k)) {
- if (matches(k)) {
- topKey = k;
- topValue = getSource().getTopValue();
- if (log.isDebugEnabled()) {
- log.debug("jump, source has top in valid range");
- }
- } else {
- getSource().next();
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug("jump, top out of range");
- String pEndRow = "empty";
- if (parentEndRow != null) {
- pEndRow = parentEndRow.toString();
- }
- log.debug("source.topKey.row: " + k.getRow() + "\t currentRow: " + currentRow + "\t parentEndRow: " + pEndRow);
- }
-
- if (parentEndRow != null) {
-
- if (currentRow == null) {
- topKey = null;
- topValue = null;
- return false;
- }
- // check it
- if (k.getRow().equals(currentRow)) {
- currentRow = getNextRow();
- } else if (k.getRow().compareTo(currentRow) > 0) {
- currentRow = k.getRow();
- }
-
- if (currentRow == null || parentEndRow.compareTo(currentRow) < 0) {
- // you're done
- topKey = null;
- topValue = null;
- return false;
- }
-
- } else { // can go to end of the tablet
- if (currentRow == null || k.getRow() == null) {
- topKey = null;
- topValue = null;
- return false;
- }
-
- if (k.getRow().equals(currentRow)) {
- currentRow = getNextRow();
- if (currentRow == null) {
- topKey = null;
- topValue = null;
- return false;
- }
- } else if (k.getRow().compareTo(currentRow) > 0) {
- currentRow = k.getRow();
- }
- }
-
- // construct new range and seek the source
- range = buildRange(currentRow);
- if (log.isDebugEnabled()) {
- log.debug("jump, new build range: " + range);
- }
- getSource().seek(range, EMPTY_COL_FAMS, false);
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug("jump, underlying source had no top key.");
- }
- topKey = null;
- topValue = null;
- return false;
- }
- }// end while
-
- return hasTop();
- }
-}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/8db62992/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/OptimizedQueryIterator.java
----------------------------------------------------------------------
diff --git a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/OptimizedQueryIterator.java b/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/OptimizedQueryIterator.java
deleted file mode 100644
index c8fa5ad..0000000
--- a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/OptimizedQueryIterator.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.examples.wikisearch.iterator;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.OptionDescriber;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.log4j.Logger;
-
-/**
- * This iterator internally uses the BooleanLogicIterator to find event UIDs in the field index portion of the partition and uses the EvaluatingIterator to
- * evaluate the events against an expression. The key and value that are emitted from this iterator are the key and value that come from the EvaluatingIterator.
- */
-public class OptimizedQueryIterator implements SortedKeyValueIterator<Key,Value>, OptionDescriber {
-
- private static Logger log = Logger.getLogger(OptimizedQueryIterator.class);
- private EvaluatingIterator event = null;
- private SortedKeyValueIterator<Key,Value> index = null;
- private Key key = null;
- private Value value = null;
- private boolean eventSpecificRange = false;
-
- public IteratorOptions describeOptions() {
- Map<String,String> options = new HashMap<String,String>();
- options.put(EvaluatingIterator.QUERY_OPTION, "full query expression");
- options.put(BooleanLogicIterator.FIELD_INDEX_QUERY, "modified query for the field index query portion");
- options.put(ReadAheadIterator.QUEUE_SIZE, "parallel queue size");
- options.put(ReadAheadIterator.TIMEOUT, "parallel iterator timeout");
- return new IteratorOptions(getClass().getSimpleName(), "evaluates event objects against an expression using the field index", options, null);
- }
-
- public boolean validateOptions(Map<String,String> options) {
- if (options.containsKey(EvaluatingIterator.QUERY_OPTION) && options.containsKey(BooleanLogicIterator.FIELD_INDEX_QUERY)) {
- return true;
- }
- return false;
- }
-
- public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
- if (!validateOptions(options)) {
- throw new IllegalArgumentException("Invalid options");
- }
-
- // Setup the EvaluatingIterator
- event = new EvaluatingIterator();
- event.init(source.deepCopy(env), options, env);
-
- // if queue size and timeout are set, then use the read ahead iterator
- if (options.containsKey(ReadAheadIterator.QUEUE_SIZE) && options.containsKey(ReadAheadIterator.TIMEOUT)) {
- BooleanLogicIterator bli = new BooleanLogicIterator();
- bli.init(source, options, env);
- index = new ReadAheadIterator();
- index.init(bli, options, env);
- } else {
- index = new BooleanLogicIterator();
- // index.setDebug(Level.DEBUG);
- index.init(source, options, env);
- }
-
- }
-
- public OptimizedQueryIterator() {}
-
- public OptimizedQueryIterator(OptimizedQueryIterator other, IteratorEnvironment env) {
- this.event = other.event;
- this.index = other.index;
- }
-
- public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new OptimizedQueryIterator(this, env);
- }
-
- public Key getTopKey() {
- if (log.isDebugEnabled()) {
- log.debug("getTopKey: " + key);
- }
- return key;
- }
-
- public Value getTopValue() {
- if (log.isDebugEnabled()) {
- log.debug("getTopValue: " + value);
- }
- return value;
- }
-
- public boolean hasTop() {
- if (log.isDebugEnabled()) {
- log.debug("hasTop: returned: " + (key != null));
- }
- return (key != null);
- }
-
- public void next() throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("next");
- }
- if (key != null) {
- key = null;
- value = null;
- }
-
- if (eventSpecificRange) {
- // Then this will probably return nothing
- event.next();
- if (event.hasTop()) {
- key = event.getTopKey();
- value = event.getTopValue();
- }
- } else {
-
- do {
- index.next();
- // If the index has a match, then seek the event to the key
- if (index.hasTop()) {
- Key eventKey = index.getTopKey();
- Key endKey = eventKey.followingKey(PartialKey.ROW_COLFAM);
- Key startKey = new Key(eventKey.getRow(), eventKey.getColumnFamily());
- Range eventRange = new Range(startKey, endKey);
- HashSet<ByteSequence> cf = new HashSet<ByteSequence>();
- cf.add(eventKey.getColumnFamilyData());
- event.seek(eventRange, cf, true);
- if (event.hasTop()) {
- key = event.getTopKey();
- value = event.getTopValue();
- }
- }
- } while (key == null && index.hasTop());
- }
- // Sanity check. Make sure both returnValue and returnKey are null or both are not null
- if (!((key == null && value == null) || (key != null && value != null))) {
- log.warn("Key: " + ((key == null) ? "null" : key.toString()));
- log.warn("Value: " + ((value == null) ? "null" : value.toString()));
- throw new IOException("Return values are inconsistent");
- }
-
- }
-
- public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("seek, range:" + range);
- }
- // Test the range to see if it is event specific.
- if (null != range.getEndKey() && range.getEndKey().getColumnFamily() != null && range.getEndKey().getColumnFamily().getLength() != 0) {
- if (log.isDebugEnabled()) {
- log.debug("Jumping straight to the event");
- }
- // Then this range is for a specific event. We don't need to use the index iterator to find it, we can just
- // seek to it with the event iterator and evaluate it.
- eventSpecificRange = true;
- event.seek(range, columnFamilies, inclusive);
- if (event.hasTop()) {
- key = event.getTopKey();
- value = event.getTopValue();
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug("Using BooleanLogicIteratorJexl");
- }
- // Seek the boolean logic iterator
- index.seek(range, columnFamilies, inclusive);
-
- // If the index has a match, then seek the event to the key
- if (index.hasTop()) {
- Key eventKey = index.getTopKey();
- // Range eventRange = new Range(eventKey, eventKey);
- Range eventRange = new Range(eventKey.getRow());
- HashSet<ByteSequence> cf = new HashSet<ByteSequence>();
- cf.add(eventKey.getColumnFamilyData());
- event.seek(eventRange, cf, true);
- if (event.hasTop()) {
- key = event.getTopKey();
- value = event.getTopValue();
- } else {
- next();
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/8db62992/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/OrIterator.java
----------------------------------------------------------------------
diff --git a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/OrIterator.java b/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/OrIterator.java
deleted file mode 100644
index 78c8576..0000000
--- a/src/examples/wikisearch/query/src/main/java/org/apache/accumulo/examples/wikisearch/iterator/OrIterator.java
+++ /dev/null
@@ -1,822 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.examples.wikisearch.iterator;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.PriorityQueue;
-
-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.hadoop.io.Text;
-import org.apache.log4j.Logger;
-
-/**
- * An iterator that handles "OR" query constructs on the server side. This code has been adapted/merged from Heap and Multi Iterators.
- */
-public class OrIterator implements SortedKeyValueIterator<Key,Value> {
-
- private TermSource currentTerm;
- private ArrayList<TermSource> sources;
- private PriorityQueue<TermSource> sorted = new PriorityQueue<TermSource>(5);
- private static final Text nullText = new Text();
- private Key topKey = null;
- private Range overallRange;
- private Collection<ByteSequence> columnFamilies;
- private boolean inclusive;
- protected static final Logger log = Logger.getLogger(OrIterator.class);
- private Text parentEndRow;
-
- protected static class TermSource implements Comparable<TermSource> {
-
- public SortedKeyValueIterator<Key,Value> iter;
- public Text dataLocation;
- public Text term;
- public Text docid;
- public Text fieldTerm;
- public Key topKey;
- public boolean atEnd;
-
- public TermSource(TermSource other) {
- this.iter = other.iter;
- this.term = other.term;
- this.dataLocation = other.dataLocation;
- this.atEnd = other.atEnd;
- }
-
- public TermSource(SortedKeyValueIterator<Key,Value> iter, Text term) {
- this.iter = iter;
- this.term = term;
- this.atEnd = false;
- }
-
- public TermSource(SortedKeyValueIterator<Key,Value> iter, Text dataLocation, Text term) {
- this.iter = iter;
- this.dataLocation = dataLocation;
- this.term = term;
- this.atEnd = false;
- }
-
- public void setNew() {
- if (!this.atEnd && this.iter.hasTop()) {
- this.topKey = this.iter.getTopKey();
-
- if (log.isDebugEnabled()) {
- log.debug("OI.TermSource.setNew TS.iter.topKey >>" + topKey + "<<");
- }
-
- if (this.term == null) {
- this.docid = this.topKey.getColumnQualifier();
- } else {
- String cqString = this.topKey.getColumnQualifier().toString();
-
- int idx = cqString.indexOf("\0");
- this.fieldTerm = new Text(cqString.substring(0, idx));
- this.docid = new Text(cqString.substring(idx + 1));
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug("OI.TermSource.setNew Setting to null...");
- }
-
- // this.term = null;
- // this.dataLocation = null;
- this.topKey = null;
- this.fieldTerm = null;
- this.docid = null;
- }
- }
-
- public int compareTo(TermSource o) {
- // NOTE: If your implementation can have more than one row in a tablet,
- // you must compare row key here first, then column qualifier.
- // NOTE2: A null check is not needed because things are only added to the
- // sorted after they have been determined to be valid.
- // return this.docid.compareTo(o.docid);
- // return this.topKey.compareTo(o.topKey);
-
- // NOTE! We need to compare UID's, not Keys!
- Key k1 = topKey;
- Key k2 = o.topKey;
- // return t1.compareTo(t2);
- String uid1 = getUID(k1);
- String uid2 = getUID(k2);
-
- if (uid1 != null && uid2 != null) {
- return uid1.compareTo(uid2);
- } else if (uid1 == null && uid2 == null) {
- return 0;
- } else if (uid1 == null) {
- return 1;
- } else {
- return -1;
- }
-
- }
-
- @Override
- public String toString() {
- return "TermSource: " + this.dataLocation + " " + this.term;
- }
-
- public boolean hasTop() {
- return this.topKey != null;
- }
- }
-
- /**
- * Returns the given key's row
- *
- * @param key
- * @return The given key's row
- */
- protected Text getPartition(Key key) {
- return key.getRow();
- }
-
- /**
- * Returns the given key's dataLocation
- *
- * @param key
- * @return The given key's dataLocation
- */
- protected Text getDataLocation(Key key) {
- return key.getColumnFamily();
- }
-
- /**
- * Returns the given key's term
- *
- * @param key
- * @return The given key's term
- */
- protected Text getTerm(Key key) {
- int idx = 0;
- String sKey = key.getColumnQualifier().toString();
-
- idx = sKey.indexOf("\0");
- return new Text(sKey.substring(0, idx));
- }
-
- /**
- * Returns the given key's DocID
- *
- * @param key
- * @return The given key's DocID
- */
- protected Text getDocID(Key key) {
- int idx = 0;
- String sKey = key.getColumnQualifier().toString();
-
- idx = sKey.indexOf("\0");
- return new Text(sKey.substring(idx + 1));
- }
-
- /**
- * Returns the given key's UID
- *
- * @param key
- * @return The given key's UID
- */
- static protected String getUID(Key key) {
- try {
- int idx = 0;
- String sKey = key.getColumnQualifier().toString();
-
- idx = sKey.indexOf("\0");
- return sKey.substring(idx + 1);
- } catch (Exception e) {
- return null;
- }
- }
-
- public OrIterator() {
- this.sources = new ArrayList<TermSource>();
- }
-
- private OrIterator(OrIterator other, IteratorEnvironment env) {
- this.sources = new ArrayList<TermSource>();
-
- for (TermSource TS : other.sources) {
- this.sources.add(new TermSource(TS.iter.deepCopy(env), TS.dataLocation, TS.term));
- }
- }
-
- public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new OrIterator(this, env);
- }
-
- public void addTerm(SortedKeyValueIterator<Key,Value> source, Text term, IteratorEnvironment env) {
- if (log.isDebugEnabled()) {
- log.debug("OI.addTerm Added source w/o family");
- log.debug("OI.addTerm term >>" + term + "<<");
- }
-
- // Don't deepcopy an iterator
- if (term == null) {
- this.sources.add(new TermSource(source, term));
- } else {
- this.sources.add(new TermSource(source.deepCopy(env), term));
- }
- }
-
- public void addTerm(SortedKeyValueIterator<Key,Value> source, Text dataLocation, Text term, IteratorEnvironment env) {
- if (log.isDebugEnabled()) {
- log.debug("OI.addTerm Added source ");
- log.debug("OI.addTerm family >>" + dataLocation + "<< term >>" + term + "<<");
- }
-
- // Don't deepcopy an iterator
- if (term == null) {
- this.sources.add(new TermSource(source, dataLocation, term));
- } else {
- this.sources.add(new TermSource(source.deepCopy(env), dataLocation, term));
- }
- }
-
- /**
- * Construct the topKey given the current <code>TermSource</code>
- *
- * @param TS
- * @return The top Key for a given TermSource
- */
- protected Key buildTopKey(TermSource TS) {
- if ((TS == null) || (TS.topKey == null)) {
- return null;
- }
-
- if (log.isDebugEnabled()) {
- log.debug("OI.buildTopKey New topKey >>" + new Key(TS.topKey.getRow(), TS.dataLocation, TS.docid) + "<<");
- }
-
- return new Key(TS.topKey.getRow(), TS.topKey.getColumnFamily(), TS.topKey.getColumnQualifier());
- }
-
- final public void next() throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("OI.next Enter: sorted.size = " + sorted.size() + " currentTerm = " + ((currentTerm == null) ? "null" : "not null"));
- }
-
- if (currentTerm == null) {
- if (log.isDebugEnabled()) {
- log.debug("OI.next currentTerm is NULL... returning");
- }
-
- topKey = null;
- return;
- }
-
- // Advance currentTerm
- currentTerm.iter.next();
-
- advanceToMatch(currentTerm);
-
- currentTerm.setNew();
-
- // See if currentTerm is still valid, remove if not
- if (log.isDebugEnabled()) {
- log.debug("OI.next Checks (correct = 0,0,0): " + ((currentTerm.topKey != null) ? "0," : "1,") + ((currentTerm.dataLocation != null) ? "0," : "1,")
- + ((currentTerm.term != null && currentTerm.fieldTerm != null) ? (currentTerm.term.compareTo(currentTerm.fieldTerm)) : "0"));
- }
-
- if (currentTerm.topKey == null || ((currentTerm.dataLocation != null) && (currentTerm.term.compareTo(currentTerm.fieldTerm) != 0))) {
- if (log.isDebugEnabled()) {
- log.debug("OI.next removing entry:" + currentTerm.term);
- }
-
- currentTerm = null;
- }
-
- // optimization.
- // if size == 0, currentTerm is the only item left,
- // OR there are no items left.
- // In either case, we don't need to use the PriorityQueue
- if (sorted.size() > 0) {
- // sort the term back in
- if (currentTerm != null) {
- sorted.add(currentTerm);
- }
- // and get the current top item out.
- currentTerm = sorted.poll();
- }
-
- if (log.isDebugEnabled()) {
- log.debug("OI.next CurrentTerm is " + ((currentTerm == null) ? "null" : currentTerm));
- }
-
- topKey = buildTopKey(currentTerm);
-
- if (hasTop()) {
- if (overallRange != null && !overallRange.contains(topKey)) {
- topKey = null;
- }
- }
- }
-
- public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
-
- overallRange = new Range(range);
- if (log.isDebugEnabled()) {
- log.debug("seek, overallRange: " + overallRange);
- }
-
- if (range.getEndKey() != null && range.getEndKey().getRow() != null) {
- this.parentEndRow = range.getEndKey().getRow();
- }
-
- if (log.isDebugEnabled()) {
- log.debug("OI.seek Entry - sources.size = " + sources.size());
- log.debug("OI.seek Entry - currentTerm = " + ((currentTerm == null) ? "false" : currentTerm.iter.getTopKey()));
- log.debug("OI.seek Entry - Key from Range = " + ((range == null) ? "false" : range.getStartKey()));
- }
-
- // If sources.size is 0, there is nothing to process, so just return.
- if (sources.isEmpty()) {
- currentTerm = null;
- topKey = null;
- return;
- }
-
- this.columnFamilies = columnFamilies;
- this.inclusive = inclusive;
-
- Range newRange = range;
- Key sourceKey = null;
- Key startKey = null;
-
- if (range != null) {
- startKey = range.getStartKey();
- }
-
- // Clear the PriorityQueue so that we can re-populate it.
- sorted.clear();
-
- TermSource TS = null;
- Iterator<TermSource> iter = sources.iterator();
- // For each term, seek forward.
- // if a hit is not found, delete it from future searches.
- int counter = 1;
- while (iter.hasNext()) {
- TS = iter.next();
-
- TS.atEnd = false;
-
- if (sources.size() == 1) {
- currentTerm = TS;
- }
-
- if (log.isDebugEnabled()) {
- log.debug("OI.seek on TS >>" + TS + "<<");
- log.debug("OI.seek seeking source >>" + counter + "<< ");
- }
-
- counter++;
-
- newRange = range;
- sourceKey = null;
-
- if (startKey != null) {
- // Construct the new key for the range
- if (log.isDebugEnabled()) {
- log.debug("OI.seek startKey >>" + startKey + "<<");
- }
-
- if (startKey.getColumnQualifier() != null) {
- sourceKey = new Key(startKey.getRow(), (TS.dataLocation == null) ? nullText : TS.dataLocation, new Text(((TS.term == null) ? "" : TS.term + "\0")
- + range.getStartKey().getColumnQualifier()));
- } else {
- sourceKey = new Key(startKey.getRow(), (TS.dataLocation == null) ? nullText : TS.dataLocation, (TS.term == null) ? nullText : TS.term);
- }
-
- if (log.isDebugEnabled()) {
- log.debug("OI.seek Seeking to the key => " + sourceKey);
- }
-
- newRange = new Range(sourceKey, true, sourceKey.followingKey(PartialKey.ROW), false);
- } else {
- if (log.isDebugEnabled()) {
- log.debug("OI.seek Using the range Seek() argument to seek => " + newRange);
- }
- }
-
- TS.iter.seek(newRange, columnFamilies, inclusive);
-
- TS.setNew();
-
- // Make sure we're on a key with the correct dataLocation and term
- advanceToMatch(TS);
-
- TS.setNew();
-
- if (log.isDebugEnabled()) {
- log.debug("OI.seek sourceKey >>" + sourceKey + "<< ");
- log.debug("OI.seek topKey >>" + ((TS.topKey == null) ? "false" : TS.topKey) + "<< ");
- log.debug("OI.seek TS.fieldTerm == " + TS.fieldTerm);
-
- log.debug("OI.seek Checks (correct = 0,0,0 / 0,1,1): " + ((TS.topKey != null) ? "0," : "1,") + ((TS.dataLocation != null) ? "0," : "1,")
- + (((TS.term != null && TS.fieldTerm != null) && (TS.term.compareTo(TS.fieldTerm) != 0)) ? "0" : "1"));
- }
-
- if ((TS.topKey == null) || ((TS.dataLocation != null) && (TS.term.compareTo(TS.fieldTerm) != 0))) {
- // log.debug("OI.seek Removing " + TS.term);
- // iter.remove();
- } // Optimization if we only have one element
- else if (sources.size() > 0 || iter.hasNext()) {
- // We have more than one source to search for, use the priority queue
- sorted.add(TS);
- } else {
- // Don't need to continue, only had one item to search
- if (log.isDebugEnabled()) {
- log.debug("OI.seek new topKey >>" + ((topKey == null) ? "false" : topKey) + "<< ");
- }
-
- // make sure it is in the range if we have one.
- if (hasTop()) {
- if (overallRange != null && !overallRange.contains(topKey)) {
- if (log.isDebugEnabled()) {
- log.debug("seek, topKey: " + topKey + " is not in the overallRange: " + overallRange);
- }
- topKey = null;
- }
- }
- return;
- }
- }
-
- // And set currentTerm = the next valid key/term.
- currentTerm = sorted.poll();
-
- if (log.isDebugEnabled()) {
- log.debug("OI.seek currentTerm = " + currentTerm);
- }
-
- topKey = buildTopKey(currentTerm);
- if (topKey == null) {
- if (log.isDebugEnabled()) {
- log.debug("OI.seek() topKey is null");
- }
- }
-
- if (log.isDebugEnabled()) {
- log.debug("OI.seek new topKey >>" + ((topKey == null) ? "false" : topKey) + "<< ");
- }
-
- if (hasTop()) {
- if (overallRange != null && !overallRange.contains(topKey)) {
- if (log.isDebugEnabled()) {
- log.debug("seek, topKey: " + topKey + " is not in the overallRange: " + overallRange);
- }
- topKey = null;
- }
- }
-
- }
-
- final public Key getTopKey() {
- if (log.isDebugEnabled()) {
- log.debug("OI.getTopKey key >>" + topKey);
- }
-
- return topKey;
- }
-
- final public Value getTopValue() {
- if (log.isDebugEnabled()) {
- log.debug("OI.getTopValue key >>" + currentTerm.iter.getTopValue());
- }
-
- return currentTerm.iter.getTopValue();
- }
-
- final public boolean hasTop() {
- if (log.isDebugEnabled()) {
- log.debug("OI.hasTop = " + ((topKey == null) ? "false" : "true"));
- }
-
- return topKey != null;
- }
-
- public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Ensures that the current <code>TermSource</code> is pointing to a key with the correct <code>dataLocation</code> and <code>term</code> or sets
- * <code>topKey</code> to null if there is no such key remaining.
- *
- * @param TS
- * The <code>TermSource</code> to advance
- * @throws IOException
- */
- private void advanceToMatch(TermSource TS) throws IOException {
- boolean matched = false;
- while (!matched) {
- if (!TS.iter.hasTop()) {
- TS.topKey = null;
- return;
- }
-
- Key iterTopKey = TS.iter.getTopKey();
-
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch current topKey = " + iterTopKey);
- }
-
- // we should compare the row to the end of the range
- if (overallRange.getEndKey() != null) {
-
- if (overallRange != null && !overallRange.contains(TS.iter.getTopKey())) {
- if (log.isDebugEnabled()) {
- log.debug("overallRange: " + overallRange + " does not contain TS.iter.topKey: " + TS.iter.getTopKey());
- log.debug("OI.advanceToMatch at the end, returning");
- }
-
- TS.atEnd = true;
- TS.topKey = null;
-
- return;
- } else {
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch not at the end");
- }
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch overallRange.getEndKey() == null");
- }
- }
-
- // Advance to the correct dataLocation
- if (log.isDebugEnabled()) {
- log.debug("Comparing dataLocations.");
- log.debug("OI.advanceToMatch dataLocationCompare: " + getDataLocation(iterTopKey) + " == " + TS.dataLocation);
- }
-
- int dataLocationCompare = getDataLocation(iterTopKey).compareTo(TS.dataLocation);
-
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch dataLocationCompare = " + dataLocationCompare);
- }
-
- // Make sure we're at a row for this dataLocation
- if (dataLocationCompare < 0) {
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch seek to desired dataLocation");
- }
-
- Key seekKey = new Key(iterTopKey.getRow(), TS.dataLocation, nullText);
-
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch seeking to => " + seekKey);
- }
-
- TS.iter.seek(new Range(seekKey, true, null, false), columnFamilies, inclusive);
-
- continue;
- } else if (dataLocationCompare > 0) {
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch advanced beyond desired dataLocation, seek to next row");
- }
-
- // Gone past the current dataLocation, seek to the next row
- Key seekKey = iterTopKey.followingKey(PartialKey.ROW);
-
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch seeking to => " + seekKey);
- }
-
- TS.iter.seek(new Range(seekKey, true, null, false), columnFamilies, inclusive);
-
- continue;
- }
-
- // Advance to the correct term
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch termCompare: " + getTerm(iterTopKey) + " == " + TS.term);
- }
-
- int termCompare = getTerm(iterTopKey).compareTo(TS.term);
-
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch termCompare = " + termCompare);
- }
-
- // Make sure we're at a row for this term
- if (termCompare < 0) {
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch seek to desired term");
- }
-
- Key seekKey = new Key(iterTopKey.getRow(), iterTopKey.getColumnFamily(), TS.term);
-
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch seeking to => " + seekKey);
- }
-
- TS.iter.seek(new Range(seekKey, true, null, false), columnFamilies, inclusive);
-
- continue;
- } else if (termCompare > 0) {
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch advanced beyond desired term, seek to next row");
- }
-
- // Gone past the current term, seek to the next row
- Key seekKey = iterTopKey.followingKey(PartialKey.ROW);
-
- if (log.isDebugEnabled()) {
- log.debug("OI.advanceToMatch seeking to => " + seekKey);
- }
-
- TS.iter.seek(new Range(seekKey, true, null, false), columnFamilies, inclusive);
- continue;
- }
-
- // If we made it here, we found a match
- matched = true;
- }
- }
-
- public boolean jump(Key jumpKey) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("OR jump: " + jumpKey);
- printTopKeysForTermSources();
- }
-
- // is the jumpKey outside my overall range?
- if (parentEndRow != null && parentEndRow.compareTo(jumpKey.getRow()) < 0) {
- // can't go there.
- if (log.isDebugEnabled()) {
- log.debug("jumpRow: " + jumpKey.getRow() + " is greater than my parentEndRow: " + parentEndRow);
- }
- return false;
- }
-
- // Clear the PriorityQueue so that we can re-populate it.
- sorted.clear();
-
- // check each term source and jump it if necessary.
- for (TermSource ts : sources) {
- int comp;
- if (!ts.hasTop()) {
- if (log.isDebugEnabled()) {
- log.debug("jump called, but ts.topKey is null, this one needs to move to next row.");
- }
- Key startKey = new Key(jumpKey.getRow(), ts.dataLocation, new Text(ts.term + "\0" + jumpKey.getColumnFamily()));
- Key endKey = null;
- if (parentEndRow != null) {
- endKey = new Key(parentEndRow);
- }
- Range newRange = new Range(startKey, true, endKey, false);
- ts.iter.seek(newRange, columnFamilies, inclusive);
- ts.setNew();
- advanceToMatch(ts);
- ts.setNew();
-
- } else {
- // check row, then uid
- comp = this.topKey.getRow().compareTo(jumpKey.getRow());
- if (comp > 0) {
- if (log.isDebugEnabled()) {
- log.debug("jump, our row is ahead of jumpKey.");
- log.debug("jumpRow: " + jumpKey.getRow() + " myRow: " + topKey.getRow() + " parentEndRow" + parentEndRow);
- }
- if (ts.hasTop()) {
- sorted.add(ts);
- }
- // do nothing, we're ahead of jumpKey row and have topkey
- } else if (comp < 0) { // a row behind jump key, need to move forward
- if (log.isDebugEnabled()) {
- log.debug("OR jump, row jump");
- }
- Key endKey = null;
- if (parentEndRow != null) {
- endKey = new Key(parentEndRow);
- }
- Key sKey = new Key(jumpKey.getRow());
- Range fake = new Range(sKey, true, endKey, false);
- ts.iter.seek(fake, columnFamilies, inclusive);
- ts.setNew();
- advanceToMatch(ts);
- ts.setNew();
- } else {
- // need to check uid
- String myUid = getUID(ts.topKey);
- String jumpUid = getUID(jumpKey);
-
- if (log.isDebugEnabled()) {
- if (myUid == null) {
- log.debug("myUid is null");
- } else {
- log.debug("myUid: " + myUid);
- }
-
- if (jumpUid == null) {
- log.debug("jumpUid is null");
- } else {
- log.debug("jumpUid: " + jumpUid);
- }
- }
-
- int ucomp = myUid.compareTo(jumpUid);
- if (ucomp < 0) {
- // need to move forward
- // create range and seek it.
- Text row = ts.topKey.getRow();
- Text cf = ts.topKey.getColumnFamily();
- String cq = ts.topKey.getColumnQualifier().toString().replaceAll(myUid, jumpUid);
- Text cq_text = new Text(cq);
- Key sKey = new Key(row, cf, cq_text);
- Key eKey = null;
- if (parentEndRow != null) {
- eKey = new Key(parentEndRow);
- }
- Range fake = new Range(sKey, true, eKey, false);
- if (log.isDebugEnabled()) {
- log.debug("uid jump, new ts.iter.seek range: " + fake);
- }
- ts.iter.seek(fake, columnFamilies, inclusive);
- ts.setNew();
- advanceToMatch(ts);
- ts.setNew();
-
- if (log.isDebugEnabled()) {
- if (ts.iter.hasTop()) {
- log.debug("ts.iter.topkey: " + ts.iter.getTopKey());
- } else {
- log.debug("ts.iter.topKey is null");
- }
- }
- }// else do nothing, we're ahead of jump key
- }
- }
-
- // ts should have moved, validate this particular ts.
- if (ts.hasTop()) {
- if (overallRange != null) {
- if (overallRange.contains(topKey)) {
- // if (topKey.getRow().compareTo(parentEndRow) < 0) {
- sorted.add(ts);
- }
- } else {
- sorted.add(ts);
- }
- }
- }
- // now get the top key from all TermSources.
- currentTerm = sorted.poll();
- if (log.isDebugEnabled()) {
- log.debug("OI.jump currentTerm = " + currentTerm);
- }
-
- topKey = buildTopKey(currentTerm);
- if (log.isDebugEnabled()) {
- log.debug("OI.jump new topKey >>" + ((topKey == null) ? "false" : topKey) + "<< ");
- }
- return hasTop();
- }
-
- private void printTopKeysForTermSources() {
- if (log.isDebugEnabled()) {
- for (TermSource ts : sources) {
- if (ts != null) {
- if (ts.topKey == null) {
- log.debug(ts.toString() + " topKey is null");
- } else {
- log.debug(ts.toString() + " topKey: " + ts.topKey);
- }
- } else {
- log.debug("ts is null");
- }
- }
-
- if (topKey != null) {
- log.debug("OrIterator current topKey: " + topKey);
- } else {
- log.debug("OrIterator current topKey is null");
- }
- }
- }
-}