You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by bu...@apache.org on 2009/08/03 05:38:50 UTC

svn commit: r800191 [3/12] - in /lucene/java/trunk: ./ contrib/ contrib/queryparser/ contrib/queryparser/src/ contrib/queryparser/src/java/ contrib/queryparser/src/java/org/ contrib/queryparser/src/java/org/apache/ contrib/queryparser/src/java/org/apac...

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PrefixWildcardQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PrefixWildcardQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PrefixWildcardQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PrefixWildcardQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,57 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+/**
+ * A {@link PrefixWildcardQueryNode} represents wildcardquery that matches abc*
+ * or *. This does not apply to phrases, this is a special case on the original
+ * lucene parser. TODO: refactor the code to remove this special case from the
+ * parser. and probably do it on a Processor
+ */
+public class PrefixWildcardQueryNode extends WildcardQueryNode {
+
+  private static final long serialVersionUID = 6851557641826407515L;
+
+  /**
+   * @param field
+   *          - field name
+   * @param text
+   *          - value including the wildcard
+   * @param begin
+   *          - position in the query string
+   * @param end
+   *          - position in the query string
+   */
+  public PrefixWildcardQueryNode(CharSequence field, CharSequence text,
+      int begin, int end) {
+    super(field, text, begin, end);
+  }
+
+  public String toString() {
+    return "<prefixWildcard field='" + this.field + "' term='" + this.text
+        + "'/>";
+  }
+
+  public PrefixWildcardQueryNode cloneTree() throws CloneNotSupportedException {
+    PrefixWildcardQueryNode clone = (PrefixWildcardQueryNode) super.cloneTree();
+
+    // nothing to do here
+
+    return clone;
+  }
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PrefixWildcardQueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ProximityQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ProximityQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ProximityQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ProximityQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,251 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.lucene.messages.MessageImpl;
+import org.apache.lucene.queryParser.core.QueryNodeError;
+import org.apache.lucene.queryParser.core.messages.QueryParserMessages;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link ProximityQueryNode} represents a query where the terms should meet
+ * specific distance conditions. (a b c) WITHIN [SENTENCE|PARAGRAPH|NUMBER]
+ * [INORDER] ("a" "b" "c") WITHIN [SENTENCE|PARAGRAPH|NUMBER] [INORDER]
+ * 
+ * TODO: Add this to the future original Lucene parser/processor/builder
+ */
+public class ProximityQueryNode extends BooleanQueryNode {
+
+  private static final long serialVersionUID = 9018220596680832916L;
+
+  public enum Type {
+    PARAGRAPH, SENTENCE, NUMBER;
+
+    CharSequence toQueryString() {
+      switch (this) {
+      case PARAGRAPH:
+        return "WITHIN PARAGRAPH";
+      case SENTENCE:
+        return "WITHIN SENTENCE";
+      case NUMBER:
+        return "WITHIN";
+      default:
+        return "WITHIN SENTENCE";
+      }
+    }
+
+    public String toString() {
+      switch (this) {
+      case PARAGRAPH:
+        return "PARAGRAPH";
+      case SENTENCE:
+        return "SENTENCE";
+      case NUMBER:
+        return "NUMBER";
+      default:
+        return "NUMBER";
+      }
+    }
+  }
+
+  // utility class
+  static public class ProximityType {
+    int pDistance = 0;
+
+    Type pType = null;
+
+    public ProximityType(Type type) {
+      this(type, 0);
+    }
+
+    public ProximityType(Type type, int distance) {
+      this.pType = type;
+      this.pDistance = distance;
+    }
+  }
+
+  private Type proximityType = Type.SENTENCE;
+  private int distance = -1;
+  private boolean inorder = false;
+  private CharSequence field = null;
+
+  /**
+   * @param clauses
+   *          - QueryNode children
+   * @param field
+   *          - field name
+   * @param type
+   *          - type of proximity query
+   * @param distance
+   *          - positive integer that specifies the distance
+   * @param inorder
+   *          - true, if the tokens should be matched in the order of the
+   *          clauses
+   */
+  public ProximityQueryNode(List<QueryNode> clauses, CharSequence field,
+      Type type, int distance, boolean inorder) {
+    super(clauses);
+    setLeaf(false);
+    this.proximityType = type;
+    this.inorder = inorder;
+    this.field = field;
+    if (type == Type.NUMBER) {
+      if (distance <= 0) {
+        throw new QueryNodeError(new MessageImpl(
+            QueryParserMessages.PARAMETER_VALUE_NOT_SUPPORTED, "distance",
+            distance));
+
+      } else {
+        this.distance = distance;
+      }
+
+    }
+    clearFields(clauses, field);
+  }
+
+  /**
+   * @param clauses
+   *          - QueryNode children
+   * @param field
+   *          - field name
+   * @param type
+   *          - type of proximity query
+   * @param inorder
+   *          - true, if the tokens should be matched in the order of the
+   *          clauses
+   */
+  public ProximityQueryNode(List<QueryNode> clauses, CharSequence field,
+      Type type, boolean inorder) {
+    this(clauses, field, type, -1, inorder);
+  }
+
+  static private void clearFields(List<QueryNode> nodes, CharSequence field) {
+    if (nodes == null || nodes.size() == 0)
+      return;
+
+    for (QueryNode clause : nodes) {
+
+      if (clause instanceof FieldQueryNode) {
+        ((FieldQueryNode) clause).toQueryStringIgnoreFields = true;
+        ((FieldQueryNode) clause).setField(field);
+      }
+    }
+  }
+
+  public Type getProximityType() {
+    return this.proximityType;
+  }
+
+  public String toString() {
+    String distanceSTR = ((this.distance == -1) ? ("")
+        : (" distance='" + this.distance) + "'");
+
+    if (getChildren() == null || getChildren().size() == 0)
+      return "<proximity field='" + this.field + "' inorder='" + this.inorder
+          + "' type='" + this.proximityType.toString() + "'" + distanceSTR
+          + "/>";
+    StringBuilder sb = new StringBuilder();
+    sb.append("<proximity field='" + this.field + "' inorder='" + this.inorder
+        + "' type='" + this.proximityType.toString() + "'" + distanceSTR + ">");
+    for (QueryNode child : getChildren()) {
+      sb.append("\n");
+      sb.append(child.toString());
+    }
+    sb.append("\n</proximity>");
+    return sb.toString();
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    String withinSTR = this.proximityType.toQueryString()
+        + ((this.distance == -1) ? ("") : (" " + this.distance))
+        + ((this.inorder) ? (" INORDER") : (""));
+
+    StringBuilder sb = new StringBuilder();
+    if (getChildren() == null || getChildren().size() == 0) {
+      // no children case
+    } else {
+      String filler = "";
+      for (QueryNode child : getChildren()) {
+        sb.append(filler).append(child.toQueryString(escapeSyntaxParser));
+        filler = " ";
+      }
+    }
+
+    if (isDefaultField(this.field)) {
+      return "( " + sb.toString() + " ) " + withinSTR;
+    } else {
+      return this.field + ":(( " + sb.toString() + " ) " + withinSTR + ")";
+    }
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    ProximityQueryNode clone = (ProximityQueryNode) super.cloneTree();
+
+    clone.proximityType = this.proximityType;
+    clone.distance = this.distance;
+    clone.field = this.field;
+
+    return clone;
+  }
+
+  /**
+   * @return the distance
+   */
+  public int getDistance() {
+    return this.distance;
+  }
+
+  /**
+   * returns null if the field was not specified in the query string
+   * 
+   * @return the field
+   */
+  public CharSequence getField() {
+    return this.field;
+  }
+
+  /**
+   * returns null if the field was not specified in the query string
+   * 
+   * @return the field
+   */
+  public String getFieldAsString() {
+    if (this.field == null)
+      return null;
+    else
+      return this.field.toString();
+  }
+
+  /**
+   * @param field
+   *          the field to set
+   */
+  public void setField(CharSequence field) {
+    this.field = field;
+  }
+
+  /**
+   * @return terms must be matched in the specified order
+   */
+  public boolean isInOrder() {
+    return this.inorder;
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ProximityQueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,95 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link QueryNode} is a interface implemented by all nodes on a QueryNode
+ * tree.
+ */
+public interface QueryNode extends Serializable {
+
+  /** convert to a query string understood by the query parser */
+  // TODO: this interface might be changed in the future
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser);
+
+  /** for printing */
+  public String toString();
+
+  /** get Children nodes */
+  public List<QueryNode> getChildren();
+
+  /** verify if a node is a Leaf node */
+  public boolean isLeaf();
+
+  /** verify if a node contains a tag */
+  public boolean containsTag(CharSequence tagName);
+
+  /**
+   * @param tagName
+   * @return of stored on under that tag name
+   */
+  public Object getTag(CharSequence tagName);
+
+  public QueryNode getParent();
+
+  /**
+   * Recursive clone the QueryNode tree The tags are not copied to the new tree
+   * when you call the cloneTree() method
+   * 
+   * @return the cloned tree
+   * @throws CloneNotSupportedException
+   */
+  public QueryNode cloneTree() throws CloneNotSupportedException;
+
+  // Below are the methods that can change state of a QueryNode
+  // Write Operations (not Thread Safe)
+
+  // add a new child to a non Leaf node
+  public void add(QueryNode child);
+
+  public void add(List<QueryNode> children);
+
+  // reset the children of a node
+  public void set(List<QueryNode> children);
+
+  /**
+   * Associate the specified value with the specified tagName. If the tagName
+   * already exists, the old value is replaced. The tagName and value cannot be
+   * null. tagName will be converted to lowercase.
+   * 
+   * @param tagName
+   * @param value
+   */
+  public void setTag(CharSequence tagName, Object value);
+
+  /**
+   * Unset a tag. tagName will be converted to lowercase.
+   * 
+   * @param tagName
+   */
+  public void unsetTag(CharSequence tagName);
+
+  public Map<CharSequence, Object> getTags();
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNodeImpl.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNodeImpl.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNodeImpl.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNodeImpl.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,225 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.ResourceBundle;
+
+import org.apache.lucene.messages.NLS;
+import org.apache.lucene.queryParser.core.messages.QueryParserMessages;
+
+/**
+ * A {@link QueryNodeImpl} is the default implementation of the interface
+ * {@link QueryNode}
+ */
+public abstract class QueryNodeImpl implements QueryNode, Cloneable {
+
+  private static final long serialVersionUID = 5569870883474845989L;
+
+  /* index default field */
+  // TODO remove PLAINTEXT_FIELD_NAME replacing it with configuration APIs
+  public static final String PLAINTEXT_FIELD_NAME = "_plain";
+
+  private boolean isLeaf = true;
+
+  private Hashtable<CharSequence, Object> tags = new Hashtable<CharSequence, Object>();
+
+  private List<QueryNode> clauses = null;
+
+  protected void allocate() {
+
+    if (this.clauses == null) {
+      this.clauses = new ArrayList<QueryNode>();
+
+    } else {
+      this.clauses.clear();
+    }
+
+  }
+
+  public final void add(QueryNode child) {
+
+    if (isLeaf() || this.clauses == null || child == null) {
+      throw new IllegalArgumentException(NLS
+          .getLocalizedMessage(QueryParserMessages.NODE_ACTION_NOT_SUPPORTED));
+    }
+
+    this.clauses.add(child);
+    ((QueryNodeImpl) child).setParent(this);
+
+  }
+
+  public final void add(List<QueryNode> children) {
+
+    if (isLeaf() || this.clauses == null) {
+      throw new IllegalArgumentException(NLS
+          .getLocalizedMessage(QueryParserMessages.NODE_ACTION_NOT_SUPPORTED));
+    }
+
+    for (QueryNode child : getChildren()) {
+      add(child);
+    }
+
+  }
+
+  public boolean isLeaf() {
+    return this.isLeaf;
+  }
+
+  public final void set(List<QueryNode> children) {
+
+    if (isLeaf() || this.clauses == null) {
+      ResourceBundle bundle = ResourceBundle
+          .getBundle("org.apache.lucene.queryParser.messages.QueryParserMessages");
+      String message = bundle.getObject("Q0008E.NODE_ACTION_NOT_SUPPORTED")
+          .toString();
+
+      throw new IllegalArgumentException(message);
+
+    }
+
+    // reset parent value
+    for (QueryNode child : children) {
+
+      ((QueryNodeImpl) child).setParent(null);
+
+    }
+
+    // allocate new children list
+    allocate();
+
+    // add new childs and set parent
+    for (QueryNode child : children) {
+      add(child);
+    }
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    QueryNodeImpl clone = (QueryNodeImpl) super.clone();
+    clone.isLeaf = this.isLeaf;
+
+    // Reset all tags
+    clone.tags = new Hashtable<CharSequence, Object>();
+
+    // copy children
+    if (this.clauses != null) {
+      List<QueryNode> localClauses = new ArrayList<QueryNode>();
+      for (QueryNode clause : this.clauses) {
+        localClauses.add(clause.cloneTree());
+      }
+      clone.clauses = localClauses;
+    }
+
+    return clone;
+  }
+
+  public Object clone() throws CloneNotSupportedException {
+    return cloneTree();
+  }
+
+  protected void setLeaf(boolean isLeaf) {
+    this.isLeaf = isLeaf;
+  }
+
+  /**
+   * @return a List for QueryNode object. Returns null, for nodes that do not
+   *         contain children. All leaf Nodes return null.
+   */
+  public final List<QueryNode> getChildren() {
+    if (isLeaf() || this.clauses == null) {
+      return null;
+    }
+    return this.clauses;
+  }
+
+  public void setTag(CharSequence tagName, Object value) {
+    this.tags.put(tagName.toString().toLowerCase(), value);
+  }
+
+  public void unsetTag(CharSequence tagName) {
+    this.tags.remove(tagName.toString().toLowerCase());
+  }
+
+  public boolean containsTag(CharSequence tagName) {
+    return this.tags.containsKey(tagName.toString().toLowerCase());
+  }
+
+  public Object getTag(CharSequence tagName) {
+    return this.tags.get(tagName.toString().toLowerCase());
+  }
+
+  private QueryNode parent = null;
+
+  private void setParent(QueryNode parent) {
+    this.parent = parent;
+  }
+
+  public QueryNode getParent() {
+    return this.parent;
+  }
+
+  protected boolean isRoot() {
+    return getParent() == null;
+  }
+
+  /**
+   * If set to true the the method toQueryString will not write field names
+   */
+  protected boolean toQueryStringIgnoreFields = false;
+
+  /**
+   * This method is use toQueryString to detect if fld is the default field
+   * 
+   * @param fld
+   *          - field name
+   * @return true if fld is the default field
+   */
+  protected boolean isDefaultField(CharSequence fld) {
+    if (this.toQueryStringIgnoreFields)
+      return true;
+    if (fld == null)
+      return true;
+    if (QueryNodeImpl.PLAINTEXT_FIELD_NAME.equals(fld.toString()))
+      return true;
+    return false;
+  }
+
+  /**
+   * Every implementation of this class should return pseudo xml like this:
+   * 
+   * For FieldQueryNode: <field start='1' end='2' field='subject' text='foo'/>
+   * 
+   * @see org.apache.lucene.queryParser.core.nodes.QueryNode#toString()
+   */
+  public String toString() {
+    return super.toString();
+  }
+
+  /**
+   * @see org.apache.lucene.queryParser.core.nodes.QueryNode#getTag(CharSequence)
+   * @return a Map with all tags for this QueryNode
+   */
+  @SuppressWarnings( { "unchecked" })
+  public Map<CharSequence, Object> getTags() {
+    return (Map<CharSequence, Object>) this.tags.clone();
+  }
+
+} // end class QueryNodeImpl

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QueryNodeImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QuotedFieldQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QuotedFieldQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QuotedFieldQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QuotedFieldQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,64 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link QuotedFieldQueryNode} represents phrase query. Example:
+ * "life is great"
+ */
+public class QuotedFieldQueryNode extends FieldQueryNode {
+
+  private static final long serialVersionUID = -6675157780051428987L;
+
+  /**
+   * @param field
+   *          - field name
+   * @param text
+   *          - value
+   * @param begin
+   *          - position in the query string
+   * @param end
+   *          - position in the query string
+   */
+  public QuotedFieldQueryNode(CharSequence field, CharSequence text, int begin,
+      int end) {
+    super(field, text, begin, end);
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escaper) {
+    if (isDefaultField(this.field)) {
+      return "\"" + getTermEscapeQuoted(escaper) + "\"";
+    } else {
+      return this.field + ":" + "\"" + getTermEscapeQuoted(escaper) + "\"";
+    }
+  }
+
+  public String toString() {
+    return "<quotedfield start='" + this.begin + "' end='" + this.end
+        + "' field='" + this.field + "' term='" + this.text + "'/>";
+  }
+
+  public QuotedFieldQueryNode cloneTree() throws CloneNotSupportedException {
+    QuotedFieldQueryNode clone = (QuotedFieldQueryNode) super.cloneTree();
+    // nothing to do here
+    return clone;
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/QuotedFieldQueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/SlopQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/SlopQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/SlopQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/SlopQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,116 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.messages.MessageImpl;
+import org.apache.lucene.queryParser.core.QueryNodeError;
+import org.apache.lucene.queryParser.core.messages.QueryParserMessages;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link SlopQueryNode} represents phrase query with a slop.
+ * 
+ * From Lucene FAQ: Is there a way to use a proximity operator (like near or
+ * within) with Lucene? There is a variable called slop that allows you to
+ * perform NEAR/WITHIN-like queries. By default, slop is set to 0 so that only
+ * exact phrases will match. When using TextParser you can use this syntax to
+ * specify the slop: "doug cutting"~2 will find documents that contain
+ * "doug cutting" as well as ones that contain "cutting doug".
+ */
+public class SlopQueryNode extends QueryNodeImpl implements FieldableNode {
+
+  private static final long serialVersionUID = 0L;
+
+  private int value = 0;
+
+  /**
+   * @param query
+   *          - QueryNode Tree with the phrase
+   * @param value
+   *          - slop value
+   */
+  public SlopQueryNode(QueryNode query, int value) {
+    if (query == null) {
+      throw new QueryNodeError(new MessageImpl(
+          QueryParserMessages.NODE_ACTION_NOT_SUPPORTED, "query", "null"));
+    }
+
+    this.value = value;
+    setLeaf(false);
+    allocate();
+    add(query);
+  }
+
+  public QueryNode getChild() {
+    return getChildren().get(0);
+  }
+
+  public int getValue() {
+    return this.value;
+  }
+
+  private CharSequence getValueString() {
+    Float f = new Float(this.value);
+    if (f == f.longValue())
+      return "" + f.longValue();
+    else
+      return "" + f;
+
+  }
+
+  public String toString() {
+    return "<slop value='" + getValueString() + "'>" + "\n"
+        + getChild().toString() + "\n</slop>";
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    if (getChild() == null)
+      return "";
+    return getChild().toQueryString(escapeSyntaxParser) + "~"
+        + getValueString();
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    SlopQueryNode clone = (SlopQueryNode) super.cloneTree();
+
+    clone.value = this.value;
+
+    return clone;
+  }
+
+  public CharSequence getField() {
+    QueryNode child = getChild();
+
+    if (child instanceof FieldableNode) {
+      return ((FieldableNode) child).getField();
+    }
+
+    return null;
+
+  }
+
+  public void setField(CharSequence fieldName) {
+    QueryNode child = getChild();
+
+    if (child instanceof FieldableNode) {
+      ((FieldableNode) child).setField(fieldName);
+    }
+
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/SlopQueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TextableQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TextableQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TextableQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TextableQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,26 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+public interface TextableQueryNode {
+
+  CharSequence getText();
+
+  void setText(CharSequence text);
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TextableQueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TokenizedPhraseQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TokenizedPhraseQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TokenizedPhraseQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TokenizedPhraseQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,104 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link TokenizedPhraseQueryNode} represents a node created by a code that
+ * tokenizes/lemmatizes/analizes.
+ */
+public class TokenizedPhraseQueryNode extends QueryNodeImpl implements
+    FieldableNode {
+
+  private static final long serialVersionUID = -7185108320787917541L;
+
+  public TokenizedPhraseQueryNode() {
+    setLeaf(false);
+    allocate();
+  }
+
+  public String toString() {
+    if (getChildren() == null || getChildren().size() == 0)
+      return "<tokenizedphrase/>";
+    StringBuilder sb = new StringBuilder();
+    sb.append("<tokenizedtphrase>");
+    for (QueryNode child : getChildren()) {
+      sb.append("\n");
+      sb.append(child.toString());
+    }
+    sb.append("\n</tokenizedphrase>");
+    return sb.toString();
+  }
+
+  // This text representation is not re-parseable
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    if (getChildren() == null || getChildren().size() == 0)
+      return "";
+
+    StringBuilder sb = new StringBuilder();
+    String filler = "";
+    for (QueryNode child : getChildren()) {
+      sb.append(filler).append(child.toQueryString(escapeSyntaxParser));
+      filler = ",";
+    }
+
+    return "[TP[" + sb.toString() + "]]";
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    TokenizedPhraseQueryNode clone = (TokenizedPhraseQueryNode) super
+        .cloneTree();
+
+    // nothing to do
+
+    return clone;
+  }
+
+  public CharSequence getField() {
+    List<QueryNode> children = getChildren();
+
+    if (children == null || children.size() == 0) {
+      return null;
+
+    } else {
+      return ((FieldableNode) children.get(0)).getField();
+    }
+
+  }
+
+  public void setField(CharSequence fieldName) {
+    List<QueryNode> children = getChildren();
+
+    if (children != null) {
+
+      for (QueryNode child : getChildren()) {
+
+        if (child instanceof FieldableNode) {
+          ((FieldableNode) child).setField(fieldName);
+        }
+
+      }
+
+    }
+
+  }
+
+} // end class MultitermQueryNode

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/TokenizedPhraseQueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/WildcardQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/WildcardQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/WildcardQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/WildcardQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,63 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link WildcardQueryNode} represents wildcard query This does not apply to
+ * phrases. Examples: a*b*c Fl?w? m?ke*g
+ */
+public class WildcardQueryNode extends FieldQueryNode {
+  private static final long serialVersionUID = 0L;
+
+  /**
+   * @param field
+   *          - field name
+   * @param text
+   *          - value that contains one or more wild card characters (? or *)
+   * @param begin
+   *          - position in the query string
+   * @param end
+   *          - position in the query string
+   */
+  public WildcardQueryNode(CharSequence field, CharSequence text, int begin,
+      int end) {
+    super(field, text, begin, end);
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escaper) {
+    if (isDefaultField(this.field)) {
+      return getTermEscaped(escaper);
+    } else {
+      return this.field + ":" + getTermEscaped(escaper);
+    }
+  }
+
+  public String toString() {
+    return "<wildcard field='" + this.field + "' term='" + this.text + "'/>";
+  }
+
+  public WildcardQueryNode cloneTree() throws CloneNotSupportedException {
+    WildcardQueryNode clone = (WildcardQueryNode) super.cloneTree();
+
+    // nothing to do here
+
+    return clone;
+  }
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/WildcardQueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html Mon Aug  3 03:38:44 2009
@@ -0,0 +1,89 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+
+Contains query nodes that are commonly used by query parser implementations 
+
+<h2>Query Nodes</h2>
+<p>
+The package <tt>org.apache.lucene.queryParser.nodes</tt> contains all the basic query nodes. The interface 
+that represents a query node is {@link org.apache.lucene.queryParser.core.nodes.QueryNode}. Every query node must be serializable.
+</p>
+<p>
+{@link org.apache.lucene.queryParser.core.nodes.QueryNode}s are used by the text parser to create a syntax tree.
+These nodes are designed to be used by UI or other text parsers.
+The default Lucene text parser is {@link org.apache.lucene.queryParser.original.parser.OriginalSyntaxParser},
+it implements original syntax.
+</p>
+<p>
+{@link org.apache.lucene.queryParser.core.nodes.QueryNode} interface should be implemented by all query nodes, 
+the class {@link org.apache.lucene.queryParser.core.nodes.QueryNodeImpl} implements {@link org.apache.lucene.queryParser.core.nodes.QueryNode} and is extended
+by all current query node implementations.
+</p>
+<p>
+A query node tree can be printed to the a stream, and it generates a pseudo XML representation
+with all the nodes.
+</p>
+<p>
+A query node tree can also generate a query string that can be parsed back by the original text parser,
+at this point only the original lucene syntax is supported.
+</p>
+<p>
+Grouping nodes:
+<ul>
+<li>AndQueryNode - used for AND operator</li>
+<li>AnyQueryNode - used for ANY operator</li>
+<li>OrQueryNode - used for OR operator</li>
+<li>BooleanQueryNode - used when no operator is specified</li>
+<li>ModifierQueryNode - used for modifier operator</li>
+<li>GroupQueryNode - used for parenthesis</li>
+<li>BoostQueryNode - used for boost operator</li>
+<li>SlopQueryNode - phrase slop</li>
+<li>FuzzyQueryNode - fuzzy node</li>
+<li>ParametricRangeQueryNode - used for parametric field:[low_value TO high_value]</li>
+<li>ProximityQueryNode - used for proximity search</li>
+<li>TokenizedPhraseQueryNode - used by tokenizers/lemmatizers/analizers for phrases/autophrases</li>
+</ul>
+</p>
+<p>
+Leaf Nodes:
+<ul>
+<li>FieldQueryNode - field/value node</li>
+<li>PathQueryNode - {@link org.apache.lucene.queryParser.core.nodes.QueryNode} object used with path-like queries</li>
+<li>OpaqueQueryNode - Used as for part of the query that can be parsed by other parsers. schema/value</li> 
+<li>ParametricQueryNode - used for parametric field [>=|<=|=|<|>] value</li>
+<li>PrefixWildcardQueryNode - non-phrase wildcard query</li>
+<li>QuotedFieldQUeryNode - regular phrase node</li>
+<li>WildcardQueryNode - non-phrase wildcard query</li>
+</ul>
+</p>
+<p>
+Utility Nodes:
+<ul>
+<li>DeletedQueryNode - used by processors on optimizations</li>
+<li>MatchAllDocsQueryNode - used by processors on optimizations</li>
+<li>MatchNoDocsQueryNode - used by processors on optimizations</li>
+<li>NoTokenFoundQueryNode - used by tokenizers/lemmatizers/analizers</li>
+</ul>
+</p>
+</body>
+</html>

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/package.html
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/package.html?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/package.html (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/package.html Mon Aug  3 03:38:44 2009
@@ -0,0 +1,59 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+
+Contains the core classes of the flexible query parser framework  
+
+<h2>Flexible Query Parser</h2>
+
+<p>
+This package contains the necessary classes to implement a query parser.  
+</p>
+
+<p>
+A query parser is divided in at least 2 phases, text parsing and query building, and one optional phase called query processing.
+</p>
+
+<h3>First Phase: Text Parsing</h3>
+<p>
+The text parsing phase is performed by a text parser, which implements {@link org.apache.lucene.queryParser.core.parser.SyntaxParser} interface.
+A text parser is responsible to get a query string and convert it to a {@link org.apache.lucene.queryParser.core.nodes.QueryNode} tree, 
+which is an object structure that represents the elements defined in the query string.
+</p>
+
+<h3>Second (optional) Phase: Query Processing</h3>
+<p>
+The query processing phase is performed by a query processor, which implements {@link org.apache.lucene.queryParser.core.processors.QueryNodeProcessor}.
+A query processor is responsible to perform any processing on a {@link org.apache.lucene.queryParser.core.nodes.QueryNode} tree. This phase
+is optional and is used only if an extra processing, validation, query expansion, etc needs to be perfomed in a {@link org.apache.lucene.queryParser.core.nodes.QueryNode} tree.
+The {@link org.apache.lucene.queryParser.core.nodes.QueryNode} tree can be either be generated by a text parser or programmatically created.
+</p>
+
+<h3>Third Phase: Query Building</h3>
+<p>
+The query building phase is performed by a query builder, which implements {@link org.apache.lucene.queryParser.core.builders.QueryBuilder}.
+A query builder is responsible to convert a {@link org.apache.lucene.queryParser.core.nodes.QueryNode} tree into an arbitrary object, which
+is usually used to be executed against a search index.   
+</p>
+
+</body>
+</html>

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/package.html
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/EscapeQuerySyntax.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/EscapeQuerySyntax.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/EscapeQuerySyntax.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/EscapeQuerySyntax.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,41 @@
+package org.apache.lucene.queryParser.core.parser;
+
+/**
+ * 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.
+ */
+
+import java.util.Locale;
+
+/**
+ * A parser needs to implement {@link EscapeQuerySyntax} to allow the QueryNode
+ * to escape the queries, when the toQueryString method is called.
+ */
+public interface EscapeQuerySyntax {
+  public enum Type {
+    STRING, NORMAL;
+  }
+
+  /**
+   * @param text
+   *          - text to be escaped
+   * @param locale
+   *          - locale for the current query
+   * @param type
+   *          - select the type of escape operation to use
+   * @return escaped text
+   */
+  CharSequence escape(CharSequence text, Locale locale, Type type);
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/EscapeQuerySyntax.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/SyntaxParser.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/SyntaxParser.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/SyntaxParser.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/SyntaxParser.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,36 @@
+package org.apache.lucene.queryParser.core.parser;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.queryParser.core.QueryNodeParseException;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+
+/**
+ * A parser needs to implement {@link SyntaxParser} interface
+ */
+public interface SyntaxParser {
+  /**
+   * @param query
+   *          - query data to be parsed
+   * @param field
+   *          - default field name
+   * @return QueryNode tree
+   */
+  public QueryNode parse(CharSequence query, CharSequence field)
+      throws QueryNodeParseException;
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/SyntaxParser.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/package.html
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/package.html?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/package.html (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/package.html Mon Aug  3 03:38:44 2009
@@ -0,0 +1,44 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+
+Contains the necessary interfaces to implement text parsers
+
+<h2>Parser</h2>
+<p>
+The package <tt>org.apache.lucene.queryParser.parser</tt> contains interfaces
+that should be implemented by the parsers.
+
+Parsers produce QueryNode Trees from a string object.
+These package still needs some work to add support to for multiple parsers.
+
+Features that should be supported for the future, related with the parser:
+- QueryNode tree should be able convertible to any parser syntax.
+- The query syntax should support calling other parsers.
+- QueryNode tree created by multiple parsers.
+
+</p>
+<p>
+
+</p>
+</body>
+</html>

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/parser/package.html
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/NoChildOptimizationQueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/NoChildOptimizationQueryNodeProcessor.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/NoChildOptimizationQueryNodeProcessor.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/NoChildOptimizationQueryNodeProcessor.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,90 @@
+package org.apache.lucene.queryParser.core.processors;
+
+/**
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.nodes.BooleanQueryNode;
+import org.apache.lucene.queryParser.core.nodes.BoostQueryNode;
+import org.apache.lucene.queryParser.core.nodes.DeletedQueryNode;
+import org.apache.lucene.queryParser.core.nodes.MatchNoDocsQueryNode;
+import org.apache.lucene.queryParser.core.nodes.ModifierQueryNode;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+import org.apache.lucene.queryParser.core.nodes.TokenizedPhraseQueryNode;
+
+/**
+ * <p>
+ * A {@link NoChildOptimizationQueryNodeProcessor} removes every
+ * BooleanQueryNode, BoostQueryNode, TokenizedPhraseQueryNode or
+ * ModifierQueryNode that do not have a valid children.
+ * </p>
+ * <p>
+ * Example: When the children of these nodes are removed for any reason then the
+ * nodes may become invalid.
+ * </p>
+ */
+public class NoChildOptimizationQueryNodeProcessor extends
+    QueryNodeProcessorImpl {
+
+  public NoChildOptimizationQueryNodeProcessor() {
+    // empty constructor
+  }
+
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+
+    if (node instanceof BooleanQueryNode || node instanceof BoostQueryNode
+        || node instanceof TokenizedPhraseQueryNode
+        || node instanceof ModifierQueryNode) {
+
+      List<QueryNode> children = node.getChildren();
+
+      if (children != null && children.size() > 0) {
+
+        for (QueryNode child : children) {
+
+          if (!(child instanceof DeletedQueryNode)) {
+            return node;
+          }
+
+        }
+
+      }
+
+      return new MatchNoDocsQueryNode();
+
+    }
+
+    return node;
+
+  }
+
+  protected QueryNode preProcessNode(QueryNode node) throws QueryNodeException {
+
+    return node;
+
+  }
+
+  protected List<QueryNode> setChildrenOrder(List<QueryNode> children)
+      throws QueryNodeException {
+
+    return children;
+
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/NoChildOptimizationQueryNodeProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessor.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessor.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessor.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,79 @@
+package org.apache.lucene.queryParser.core.processors;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.config.QueryConfigHandler;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+
+/**
+ * <p>
+ * A {@link QueryNodeProcessor} is an interface for classes that process a
+ * {@link QueryNode} tree.
+ * <p>
+ * </p>
+ * The implementor of this class should perform some operation on a query node
+ * tree and return the same or another query node tree.
+ * <p>
+ * </p>
+ * It also may carry a {@link QueryConfigHandler} object that contains
+ * configuration about the query represented by the query tree or the
+ * collection/index where it's intended to be executed.
+ * <p>
+ * </p>
+ * In case there is any {@link QueryConfigHandler} associated to the query tree
+ * to be processed, it should be set using
+ * {@link QueryNodeProcessor#setQueryConfigHandler(QueryConfigHandler)} before
+ * {@link QueryNodeProcessor#process(QueryNode)} is invoked.
+ * 
+ * @see QueryNode
+ * @see QueryNodeProcessor
+ * @see QueryConfigHandler
+ */
+public interface QueryNodeProcessor {
+
+  /**
+   * Processes a query node tree. It may return the same or another query tree.
+   * I should never return <code>null</code>.
+   * 
+   * @param queryTree
+   *          tree root node
+   * 
+   * @return the processed query tree
+   * 
+   * @throws QueryNodeException
+   */
+  public QueryNode process(QueryNode queryTree) throws QueryNodeException;
+
+  /**
+   * Sets the {@link QueryConfigHandler} associated to the query tree.
+   * 
+   * @param queryConfigHandler
+   */
+  public void setQueryConfigHandler(QueryConfigHandler queryConfigHandler);
+
+  /**
+   * Returns the {@link QueryConfigHandler} associated to the query tree if any,
+   * otherwise it returns <code>null</code>
+   * 
+   * @return the {@link QueryConfigHandler} associated to the query tree if any,
+   *         otherwise it returns <code>null</code>
+   */
+  public QueryConfigHandler getQueryConfigHandler();
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorImpl.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorImpl.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorImpl.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorImpl.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,253 @@
+package org.apache.lucene.queryParser.core.processors;
+
+/**
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.config.QueryConfigHandler;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+
+/**
+ * <p>
+ * This is a default implementation for the {@link QueryNodeProcessor}
+ * interface, it's an abstract class, so it should be extended by classes that
+ * want to process a {@link QueryNode} tree.
+ * </p>
+ * <p>
+ * This class process {@link QueryNode}s from left to right in the tree. While
+ * it's walking down the tree, for every node,
+ * {@link #preProcessNode(QueryNode)} is invoked. After a node's children are
+ * processed, {@link #postProcessNode(QueryNode)} is invoked for that node.
+ * {@link #setChildrenOrder(List)} is invoked before
+ * {@link #postProcessNode(QueryNode)} only if the node has at least one child,
+ * in {@link #setChildrenOrder(List)} the implementor might redefine the
+ * children order or remove any children from the children list.
+ * </p>
+ * <p>
+ * Here is an example about how it process the nodes:
+ * </p>
+ * 
+ * <pre>
+ *      a
+ *     / \
+ *    b   e
+ *   / \
+ *  c   d
+ * </pre>
+ * 
+ * Here is the order the methods would be invoked for the tree described above:
+ * 
+ * <pre>
+ *      preProcessNode( a );
+ *      preProcessNode( b );
+ *      preProcessNode( c );
+ *      postProcessNode( c );
+ *      preProcessNode( d );
+ *      postProcessNode( d );
+ *      setChildrenOrder( bChildrenList );
+ *      postProcessNode( b );
+ *      preProcessNode( e );
+ *      postProcessNode( e );
+ *      setChildrenOrder( aChildrenList );
+ *      postProcessNode( a )
+ * </pre>
+ * 
+ * @see org.apache.lucene.queryParser.core.processors.QueryNodeProcessor
+ */
+public abstract class QueryNodeProcessorImpl implements QueryNodeProcessor {
+
+  private ArrayList<ChildrenList> childrenListPool = new ArrayList<ChildrenList>();
+
+  private QueryConfigHandler queryConfig;
+
+  public QueryNodeProcessorImpl() {
+    // empty constructor
+  }
+
+  public QueryNodeProcessorImpl(QueryConfigHandler queryConfigHandler) {
+    this.queryConfig = queryConfigHandler;
+  }
+
+  public QueryNode process(QueryNode queryTree) throws QueryNodeException {
+    return processIteration(queryTree);
+  }
+
+  private QueryNode processIteration(QueryNode queryTree)
+      throws QueryNodeException {
+    queryTree = preProcessNode(queryTree);
+
+    processChildren(queryTree);
+
+    queryTree = postProcessNode(queryTree);
+
+    return queryTree;
+
+  }
+
+  /**
+   * This method is called every time a child is processed.
+   * 
+   * @param queryTree
+   *          the query node child to be processed
+   * @throws QueryNodeException
+   *           if something goes wrong during the query node processing
+   */
+  protected void processChildren(QueryNode queryTree) throws QueryNodeException {
+
+    List<QueryNode> children = queryTree.getChildren();
+    ChildrenList newChildren;
+
+    if (children != null && children.size() > 0) {
+
+      newChildren = allocateChildrenList();
+
+      try {
+
+        for (QueryNode child : children) {
+          child = processIteration(child);
+
+          if (child == null) {
+            throw new NullPointerException();
+
+          }
+
+          newChildren.add(child);
+
+        }
+
+        List<QueryNode> orderedChildrenList = setChildrenOrder(newChildren);
+
+        queryTree.set(orderedChildrenList);
+
+      } finally {
+        newChildren.beingUsed = false;
+      }
+
+    }
+
+  }
+
+  private ChildrenList allocateChildrenList() {
+    ChildrenList list = null;
+
+    for (ChildrenList auxList : this.childrenListPool) {
+
+      if (!auxList.beingUsed) {
+        list = auxList;
+        list.clear();
+
+        break;
+
+      }
+
+    }
+
+    if (list == null) {
+      list = new ChildrenList();
+      this.childrenListPool.add(list);
+
+    }
+
+    list.beingUsed = true;
+
+    return list;
+
+  }
+
+  /**
+   * For reference about this method check:
+   * {@link QueryNodeProcessor#setQueryConfigHandler(QueryConfigHandler)}.
+   * 
+   * @param queryConfigHandler
+   *          the query configuration handler to be set.
+   * 
+   * @see QueryNodeProcessor#getQueryConfigHandler()
+   * @see QueryConfigHandler
+   */
+  public void setQueryConfigHandler(QueryConfigHandler queryConfigHandler) {
+    this.queryConfig = queryConfigHandler;
+  }
+
+  /**
+   * For reference about this method check:
+   * {@link QueryNodeProcessor#getQueryConfigHandler()}.
+   * 
+   * @return QueryConfigHandler the query configuration handler to be set.
+   * 
+   * @see QueryNodeProcessor#setQueryConfigHandler(QueryConfigHandler)
+   * @see QueryConfigHandler
+   */
+  public QueryConfigHandler getQueryConfigHandler() {
+    return this.queryConfig;
+  }
+
+  /**
+   * This method is invoked for every node when walking down the tree.
+   * 
+   * @param node
+   *          the query node to be pre-processed
+   * 
+   * @return a query node
+   * 
+   * @throws QueryNodeException
+   *           if something goes wrong during the query node processing
+   */
+  abstract protected QueryNode preProcessNode(QueryNode node)
+      throws QueryNodeException;
+
+  /**
+   * This method is invoked for every node when walking up the tree.
+   * 
+   * @param node
+   *          node the query node to be post-processed
+   * 
+   * @return a query node
+   * 
+   * @throws QueryNodeException
+   *           if something goes wrong during the query node processing
+   */
+  abstract protected QueryNode postProcessNode(QueryNode node)
+      throws QueryNodeException;
+
+  /**
+   * This method is invoked for every node that has at least on child. It's
+   * invoked right before {@link #postProcessNode(QueryNode)} is invoked.
+   * 
+   * @param children
+   *          the list containing all current node's children
+   * 
+   * @return a new list containing all children that should be set to the
+   *         current node
+   * 
+   * @throws QueryNodeException
+   *           if something goes wrong during the query node processing
+   */
+  abstract protected List<QueryNode> setChildrenOrder(List<QueryNode> children)
+      throws QueryNodeException;
+
+  private static class ChildrenList extends ArrayList<QueryNode> {
+
+    private static final long serialVersionUID = -2613518456949297135L;
+
+    boolean beingUsed;
+
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorPipeline.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorPipeline.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorPipeline.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorPipeline.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,128 @@
+package org.apache.lucene.queryParser.core.processors;
+
+/**
+ * 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.
+ */
+
+import java.util.LinkedList;
+
+import org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.config.QueryConfigHandler;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+
+/**
+ * A {@link QueryNodeProcessorPipeline} class should be used to build a query
+ * node processor pipeline.
+ * 
+ * When a query node tree is processed using this class, it passes the query
+ * node tree to each processor on the pipeline and the result from each
+ * processor is passed to the next one, always following the order the
+ * processors were on the pipeline.
+ * 
+ * When a {@link QueryConfigHandler} object is set on a
+ * {@link QueryNodeProcessorPipeline}, it takes care of also setting this
+ * {@link QueryConfigHandler} on all processor on pipeline.
+ * 
+ */
+public class QueryNodeProcessorPipeline implements QueryNodeProcessor {
+
+  private LinkedList<QueryNodeProcessor> processors = new LinkedList<QueryNodeProcessor>();
+
+  private QueryConfigHandler queryConfig;
+
+  /**
+   * Constructs an empty query node processor pipeline.
+   */
+  public QueryNodeProcessorPipeline() {
+    // empty constructor
+  }
+
+  /**
+   * Constructs with a {@link QueryConfigHandler} object.
+   */
+  public QueryNodeProcessorPipeline(QueryConfigHandler queryConfigHandler) {
+    this.queryConfig = queryConfigHandler;
+  }
+
+  /**
+   * For reference about this method check:
+   * {@link QueryNodeProcessor#getQueryConfigHandler()}.
+   * 
+   * @return QueryConfigHandler the query configuration handler to be set.
+   * 
+   * @see QueryNodeProcessor#setQueryConfigHandler(QueryConfigHandler)
+   * @see QueryConfigHandler
+   */
+  public QueryConfigHandler getQueryConfigHandler() {
+    return this.queryConfig;
+  }
+
+  /**
+   * For reference about this method check:
+   * {@link QueryNodeProcessor#process(QueryNode)}.
+   * 
+   * @param queryTree
+   *          the query node tree to be processed
+   * 
+   * @throws QueryNodeException
+   *           if something goes wrong during the query node processing
+   * 
+   * @see QueryNode
+   */
+  public QueryNode process(QueryNode queryTree) throws QueryNodeException {
+
+    for (QueryNodeProcessor processor : this.processors) {
+      queryTree = processor.process(queryTree);
+    }
+
+    return queryTree;
+
+  }
+
+  /**
+   * Adds a processor to the pipeline, it's always added to the end of the
+   * pipeline.
+   * 
+   * @param processor
+   *          the processor to be added
+   */
+  public void addProcessor(QueryNodeProcessor processor) {
+    this.processors.add(processor);
+
+    processor.setQueryConfigHandler(this.queryConfig);
+
+  }
+
+  /**
+   * For reference about this method check:
+   * {@link QueryNodeProcessor#setQueryConfigHandler(QueryConfigHandler)}.
+   * 
+   * @param queryConfigHandler
+   *          the query configuration handler to be set.
+   * 
+   * @see QueryNodeProcessor#getQueryConfigHandler()
+   * @see QueryConfigHandler
+   */
+  public void setQueryConfigHandler(QueryConfigHandler queryConfigHandler) {
+    this.queryConfig = queryConfigHandler;
+
+    for (QueryNodeProcessor processor : this.processors) {
+      processor.setQueryConfigHandler(this.queryConfig);
+    }
+
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/QueryNodeProcessorPipeline.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/RemoveDeletedQueryNodesProcessor.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/RemoveDeletedQueryNodesProcessor.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/RemoveDeletedQueryNodesProcessor.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/RemoveDeletedQueryNodesProcessor.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,109 @@
+package org.apache.lucene.queryParser.core.processors;
+
+/**
+ * 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.
+ */
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.nodes.DeletedQueryNode;
+import org.apache.lucene.queryParser.core.nodes.MatchNoDocsQueryNode;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+
+/**
+ * A {@link QueryNodeProcessorPipeline} class removes every instance of
+ * {@link DeletedQueryNode} from a query node tree. If the resulting root node
+ * is a {@link DeletedQueryNode}, {@link MatchNoDocsQueryNode} is returned.
+ * 
+ */
+public class RemoveDeletedQueryNodesProcessor extends QueryNodeProcessorImpl {
+
+  public RemoveDeletedQueryNodesProcessor() {
+    // empty constructor
+  }
+
+  public QueryNode process(QueryNode queryTree) throws QueryNodeException {
+    queryTree = super.process(queryTree);
+
+    if (queryTree instanceof DeletedQueryNode
+        && !(queryTree instanceof MatchNoDocsQueryNode)) {
+
+      return new MatchNoDocsQueryNode();
+
+    }
+
+    return queryTree;
+
+  }
+
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+
+    if (!node.isLeaf()) {
+      List<QueryNode> children = node.getChildren();
+      boolean removeBoolean = false;
+
+      if (children == null || children.size() == 0) {
+        removeBoolean = true;
+
+      } else {
+        removeBoolean = true;
+
+        for (Iterator<QueryNode> it = children.iterator(); it.hasNext();) {
+
+          if (!(it.next() instanceof DeletedQueryNode)) {
+            removeBoolean = false;
+            break;
+
+          }
+
+        }
+
+      }
+
+      if (removeBoolean) {
+        return new DeletedQueryNode();
+      }
+
+    }
+
+    return node;
+
+  }
+
+  protected List<QueryNode> setChildrenOrder(List<QueryNode> children)
+      throws QueryNodeException {
+
+    for (int i = 0; i < children.size(); i++) {
+
+      if (children.get(i) instanceof DeletedQueryNode) {
+        children.remove(i--);
+      }
+
+    }
+
+    return children;
+
+  }
+
+  protected QueryNode preProcessNode(QueryNode node) throws QueryNodeException {
+
+    return node;
+
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/RemoveDeletedQueryNodesProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/package.html
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/package.html?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/package.html (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/package.html Mon Aug  3 03:38:44 2009
@@ -0,0 +1,57 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+
+Interfaces and implementations used by query node processors
+
+<h2>Query Node Processors</h2>
+<p>
+The package <tt>org.apache.lucene.queryParser.processors</tt> contains interfaces
+that should be implemented by every query node processor.
+</p>
+<p>
+The interface that every query node processor should implement is {@link org.apache.lucene.queryParser.core.processors.QueryNodeProcessor}.
+</p>
+<p>
+A query node processor should be used to process a {@link org.apache.lucene.queryParser.core.nodes.QueryNode} tree. 
+{@link org.apache.lucene.queryParser.core.nodes.QueryNode} trees can be programmatically created or generated by a
+text parser. See {@link org.apache.lucene.queryParser.core.parser} for more details about text parsers.
+</p>
+
+<p>
+A query node processor should be used to process a {@link org.apache.lucene.queryParser.core.nodes.QueryNode} tree. 
+{@link org.apache.lucene.queryParser.core.nodes.QueryNode} trees can be programmatically created or generated by a
+text parser. See {@link org.apache.lucene.queryParser.core.parser} for more details about text parsers.
+</p>
+
+<p>
+A pipeline of processors can be assembled using {@link org.apache.lucene.queryParser.core.processors.QueryNodeProcessorPipeline}.
+</p>
+
+<p>
+Implementors may want to extend {@link org.apache.lucene.queryParser.core.processors.QueryNodeProcessorImpl}, which simplifies 
+the implementation, because it walks automatically the {@link org.apache.lucene.queryParser.core.nodes.QueryNode}. See
+{@link org.apache.lucene.queryParser.core.processors.QueryNodeProcessorImpl} for more details.   
+</p>
+
+</body>
+</html>

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/processors/package.html
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/util/QueryNodeOperation.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/util/QueryNodeOperation.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/util/QueryNodeOperation.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/util/QueryNodeOperation.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,94 @@
+package org.apache.lucene.queryParser.core.util;
+
+/**
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.queryParser.core.QueryNodeError;
+import org.apache.lucene.queryParser.core.nodes.AndQueryNode;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+
+/**
+ * Allow joining 2 QueryNode Trees, into one.
+ */
+public final class QueryNodeOperation {
+  private QueryNodeOperation() {
+    // Exists only to defeat instantiation.
+  }
+
+  private enum ANDOperation {
+    BOTH, Q1, Q2, NONE
+  }
+
+  /**
+   * perform a logical and of 2 QueryNode trees. if q1 and q2 are ANDQueryNode
+   * nodes it uses head Node from q1 and adds the children of q2 to q1 if q1 is
+   * a AND node and q2 is not, add q2 as a child of the head node of q1 if q2 is
+   * a AND node and q1 is not, add q1 as a child of the head node of q2 if q1
+   * and q2 are not ANDQueryNode nodes, create a AND node and make q1 and q2
+   * children of that node if q1 or q2 is null it returns the not null node if
+   * q1 = q2 = null it returns null
+   */
+  public final static QueryNode logicalAnd(QueryNode q1, QueryNode q2) {
+    if (q1 == null)
+      return q2;
+    if (q2 == null)
+      return q1;
+
+    ANDOperation op = null;
+    if (q1 instanceof AndQueryNode && q2 instanceof AndQueryNode)
+      op = ANDOperation.BOTH;
+    else if (q1 instanceof AndQueryNode)
+      op = ANDOperation.Q1;
+    else if (q1 instanceof AndQueryNode)
+      op = ANDOperation.Q2;
+    else
+      op = ANDOperation.NONE;
+
+    try {
+      QueryNode result = null;
+      switch (op) {
+      case NONE:
+        List<QueryNode> children = new ArrayList<QueryNode>();
+        children.add(q1.cloneTree());
+        children.add(q2.cloneTree());
+        result = new AndQueryNode(children);
+        return result;
+      case Q1:
+        result = q1.cloneTree();
+        result.add(q2.cloneTree());
+        return result;
+      case Q2:
+        result = q2.cloneTree();
+        result.add(q1.cloneTree());
+        return result;
+      case BOTH:
+        result = q1.cloneTree();
+        result.add(q2.cloneTree().getChildren());
+        return result;
+      }
+    } catch (CloneNotSupportedException e) {
+      throw new QueryNodeError(e);
+    }
+
+    return null;
+
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/util/QueryNodeOperation.java
------------------------------------------------------------------------------
    svn:eol-style = native