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 [2/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/config/FieldConfigListener.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/FieldConfigListener.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/FieldConfigListener.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/FieldConfigListener.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,21 @@
+package org.apache.lucene.queryParser.core.config;
+
+/**
+ * This interface should be implemented by classes that wants to listen for
+ * field configuration requests. The implementation receives a
+ * {@link FieldConfig} object and may add/change its attributes.
+ * 
+ * @see FieldConfig
+ * @see QueryConfigHandler
+ */
+public interface FieldConfigListener {
+
+  /**
+   * This method is called ever time a field configuration is requested.
+   * 
+   * @param fieldConfig
+   *          the field configuration requested, should never be null
+   */
+  void buildFieldConfig(FieldConfig fieldConfig);
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/QueryConfigHandler.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/QueryConfigHandler.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/QueryConfigHandler.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/QueryConfigHandler.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,85 @@
+package org.apache.lucene.queryParser.core.config;
+
+/**
+ * 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.processors.QueryNodeProcessor;
+import org.apache.lucene.util.Attribute;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * This class can be used to hold any query configuration and no field
+ * configuration. For field configuration, it creates a empty
+ * {@link FieldConfig} object and delegate it to field config listeners, 
+ * these are responsible for setting up all the field configuration.
+ * 
+ * {@link QueryConfigHandler} should be extended by classes that intends to
+ * provide configuration to {@link QueryNodeProcessor} objects.
+ * 
+ * This class extends {@link AttributeSource}, so {@link Attribute}s can be
+ * attached to it.
+ * 
+ * The class that extends {@link QueryConfigHandler} should also provide
+ * {@link FieldConfig} objects for each collection field.
+ * 
+ * @see Attribute
+ * @see FieldConfig
+ * @see FieldConfigListener
+ * @see QueryConfigHandler
+ */
+public abstract class QueryConfigHandler extends AttributeSource {
+
+  private LinkedList<FieldConfigListener> listeners = new LinkedList<FieldConfigListener>();;
+
+  /**
+   * Returns an implementation of
+   * {@link FieldConfig} for a specific field name. If the implemented
+   * {@link QueryConfigHandler} does not know a specific field name, it may
+   * return <code>null</code>, indicating there is no configuration for that
+   * field.
+   * 
+   * @param fieldName
+   *          the field name
+   * @return a {@link FieldConfig} object containing the field name
+   *         configuration or <code>null</code>, if the implemented
+   *         {@link QueryConfigHandler} has no configuration for that field
+   */
+  public FieldConfig getFieldConfig(CharSequence fieldName) {
+    FieldConfig fieldConfig = new FieldConfig(fieldName);
+
+    for (FieldConfigListener listener : this.listeners) {
+      listener.buildFieldConfig(fieldConfig);
+    }
+
+    return fieldConfig;
+
+  }
+
+  /**
+   * Adds a listener. The added listeners are called in the order they are
+   * added.
+   * 
+   * @param listener
+   *          the listener to be added
+   */
+  public void addFieldConfigListener(FieldConfigListener listener) {
+    this.listeners.add(listener);
+  }
+  
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/package.html
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/package.html?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/package.html (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/config/package.html Mon Aug  3 03:38:44 2009
@@ -0,0 +1,50 @@
+<!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 base classes used to configure the query processing
+
+<h2>Query Configuration Interfaces</h2>
+<p>
+The package <tt>org.apache.lucene.queryParser.config</tt> contains query configuration handler
+abstract class that all config handlers should extend.
+</p>
+<p>
+See {@link org.apache.lucene.queryParser.original.config.OriginalQueryConfigHandler} for a reference
+implementation.
+</p>
+<p>
+{@link org.apache.lucene.queryParser.core.config.FieldConfig} and {@link org.apache.lucene.queryParser.core.config.QueryConfigHandler}
+ should use {@link org.apache.lucene.util.Attribute} to store all attributes
+required by the config implementation. See <tt>org.apache.lucene.queryParser.original.config.*Attribute</tt>
+for reference implementation.
+</p>
+<p>
+The {@link org.apache.lucene.queryParser.core.config.QueryConfigHandler}, {@link org.apache.lucene.queryParser.core.config.FieldConfig},
+ and {@link org.apache.lucene.util.Attribute}s are used in the processors to access config
+information in a flexible and independent way.
+See {@link org.apache.lucene.queryParser.original.processors.ParametricRangeQueryNodeProcessor} for a 
+reference implementation.
+</p>
+</body>
+</html
+

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,54 @@
+package org.apache.lucene.queryParser.core.messages;
+
+/**
+ * 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.NLS;
+
+/**
+ * Flexible Query Parser message bundle class
+ */
+public class QueryParserMessages extends NLS {
+
+  private static final String BUNDLE_NAME = QueryParserMessages.class.getName();
+
+  private QueryParserMessages() {
+    // Do not instantiate
+  }
+
+  static {
+    // register all string ids with NLS class and initialize static string
+    // values
+    NLS.initializeMessages(BUNDLE_NAME, QueryParserMessages.class);
+  }
+
+  // static string must match the strings in the property files.
+  public static String INVALID_SYNTAX;
+  public static String INVALID_SYNTAX_CANNOT_PARSE;
+  public static String INVALID_SYNTAX_FUZZY_LIMITS;
+  public static String INVALID_SYNTAX_ESCAPE_UNICODE_TRUNCATION;
+  public static String INVALID_SYNTAX_ESCAPE_CHARACTER;
+  public static String INVALID_SYNTAX_ESCAPE_NONE_HEX_UNICODE;
+  public static String NODE_ACTION_NOT_SUPPORTED;
+  public static String PARAMETER_VALUE_NOT_SUPPORTED;
+  public static String LUCENE_QUERY_CONVERSION_ERROR;
+  public static String EMPTY_MESSAGE;
+  public static String WILDCARD_NOT_SUPPORTED;
+  public static String TOO_MANY_BOOLEAN_CLAUSES;
+  public static String LEADING_WILDCARD_NOT_ALLOWED;
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.properties
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.properties?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.properties (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.properties Mon Aug  3 03:38:44 2009
@@ -0,0 +1,45 @@
+# This resource bundle contains Flexible Query Parser messages.
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+#  <REPLACEMENT arg="{0}" value="detailed_message"/>
+INVALID_SYNTAX = Syntax Error: {0}
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+#  <REPLACEMENT arg="{0}" value="invalid_query"/>
+#  <REPLACEMENT arg="{1}" value="detailed_message"/>
+INVALID_SYNTAX_CANNOT_PARSE = Syntax Error, cannot parse {0}: {1} 
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+INVALID_SYNTAX_FUZZY_LIMITS = The similarity value for a fuzzy search must be between 0.0 and 1.0.
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+INVALID_SYNTAX_ESCAPE_UNICODE_TRUNCATION = Truncated unicode escape sequence.
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+INVALID_SYNTAX_ESCAPE_CHARACTER = Term can not end with escape character.
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+INVALID_SYNTAX_ESCAPE_NONE_HEX_UNICODE = None-hex character in unicode escape sequence: {0}
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+NODE_ACTION_NOT_SUPPORTED = This node does not support this action.
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+PARAMETER_VALUE_NOT_SUPPORTED = Parameter {1} with value {0} not supported.
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+#  <REPLACEMENT arg="{0}" value="query"/>
+#  <REPLACEMENT arg="{1}" value="error/class"/>
+LUCENE_QUERY_CONVERSION_ERROR = Cannot convert query to lucene syntax: {0} error: {1}
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+EMPTY_MESSAGE = 
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+WILDCARD_NOT_SUPPORTED = Wildcard is not supported for query: {0} 
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+TOO_MANY_BOOLEAN_CLAUSES = Too many boolean clauses, the maximum supported is {0}: {1}
+
+#<CREATEDBY>Apache Lucene Community</CREATEDBY>
+LEADING_WILDCARD_NOT_ALLOWED = Leading wildcard is not allowed: {0}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/package.html
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/package.html?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/package.html (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/package.html Mon Aug  3 03:38:44 2009
@@ -0,0 +1,31 @@
+<!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 messages usually used by query parser implementations
+
+<h2>Query Parser Messages</h2>
+
+Messages for the Flexible Query Parser, they use <tt>org.apache.lucene.messages.NLS</tt> API.
+
+</body>
+</html>

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/AndQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/AndQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/AndQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/AndQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,77 @@
+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 AndQueryNode} represents an AND boolean operation performed on a
+ * list of nodes.
+ */
+public class AndQueryNode extends BooleanQueryNode {
+
+  private static final long serialVersionUID = 118496077529151825L;
+
+  /**
+   * @param clauses
+   *          - the query nodes to be and'ed
+   */
+  public AndQueryNode(List<QueryNode> clauses) {
+    super(clauses);
+    if ((clauses == null) || (clauses.size() == 0)) {
+      throw new IllegalArgumentException(
+          "AND query must have at least one clause");
+    }
+  }
+
+  public String toString() {
+    if (getChildren() == null || getChildren().size() == 0)
+      return "<boolean operation='and'/>";
+    StringBuilder sb = new StringBuilder();
+    sb.append("<boolean operation='and'>");
+    for (QueryNode child : getChildren()) {
+      sb.append("\n");
+      sb.append(child.toString());
+
+    }
+    sb.append("\n</boolean>");
+    return sb.toString();
+  }
+
+  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 = " AND ";
+    }
+
+    // in case is root or the parent is a group node avoid parenthesis
+    if ((getParent() != null && getParent() instanceof GroupQueryNode)
+        || isRoot())
+      return sb.toString();
+    else
+      return "( " + sb.toString() + " )";
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/AnyQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/AnyQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/AnyQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/AnyQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,143 @@
+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 AnyQueryNode} represents an ANY operator performed on a list of
+ * nodes.
+ */
+public class AnyQueryNode extends AndQueryNode {
+  private static final long serialVersionUID = 1000791433562954187L;
+
+  private CharSequence field = null;
+  private int minimumMatchingmElements = 0;
+
+  /**
+   * @param clauses
+   *          - the query nodes to be or'ed
+   */
+  public AnyQueryNode(List<QueryNode> clauses, CharSequence field,
+      int minimumMatchingElements) {
+    super(clauses);
+    this.field = field;
+    this.minimumMatchingmElements = minimumMatchingElements;
+
+    if (clauses != null) {
+
+      for (QueryNode clause : clauses) {
+
+        if (clause instanceof FieldQueryNode) {
+
+          if (clause instanceof QueryNodeImpl) {
+            ((QueryNodeImpl) clause).toQueryStringIgnoreFields = true;
+          }
+
+          if (clause instanceof FieldableNode) {
+            ((FieldableNode) clause).setField(field);
+          }
+
+        }
+      }
+
+    }
+
+  }
+
+  public int getMinimumMatchingElements() {
+    return this.minimumMatchingmElements;
+  }
+
+  /**
+   * returns null if the field was not specified
+   * 
+   * @return the field
+   */
+  public CharSequence getField() {
+    return this.field;
+  }
+
+  /**
+   * returns - null if the field was not specified
+   * 
+   * @return the field as a String
+   */
+  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;
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    AnyQueryNode clone = (AnyQueryNode) super.cloneTree();
+
+    clone.field = this.field;
+    clone.minimumMatchingmElements = this.minimumMatchingmElements;
+
+    return clone;
+  }
+
+  public String toString() {
+    if (getChildren() == null || getChildren().size() == 0)
+      return "<any field='" + this.field + "'  matchelements="
+          + this.minimumMatchingmElements + "/>";
+    StringBuilder sb = new StringBuilder();
+    sb.append("<any field='" + this.field + "'  matchelements="
+        + this.minimumMatchingmElements + ">");
+    for (QueryNode clause : getChildren()) {
+      sb.append("\n");
+      sb.append(clause.toString());
+    }
+    sb.append("\n</any>");
+    return sb.toString();
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    String anySTR = "ANY " + this.minimumMatchingmElements;
+
+    StringBuilder sb = new StringBuilder();
+    if (getChildren() == null || getChildren().size() == 0) {
+      // no childs case
+    } else {
+      String filler = "";
+      for (QueryNode clause : getChildren()) {
+        sb.append(filler).append(clause.toQueryString(escapeSyntaxParser));
+        filler = " ";
+      }
+    }
+
+    if (isDefaultField(this.field)) {
+      return "( " + sb.toString() + " ) " + anySTR;
+    } else {
+      return this.field + ":(( " + sb.toString() + " ) " + anySTR + ")";
+    }
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/BooleanQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/BooleanQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/BooleanQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/BooleanQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,83 @@
+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 BooleanQueryNode} represents a list of elements which do not have an
+ * explicit boolean operator defined between them. It can be used to express a
+ * boolean query that intends to use the default boolean operator.
+ */
+public class BooleanQueryNode extends QueryNodeImpl {
+
+  private static final long serialVersionUID = -2206623652088638072L;
+
+  /**
+   * @param clauses
+   *          - the query nodes to be and'ed
+   */
+  public BooleanQueryNode(List<QueryNode> clauses) {
+    setLeaf(false);
+    allocate();
+    set(clauses);
+  }
+
+  public String toString() {
+    if (getChildren() == null || getChildren().size() == 0)
+      return "<boolean operation='default'/>";
+    StringBuilder sb = new StringBuilder();
+    sb.append("<boolean operation='default'>");
+    for (QueryNode child : getChildren()) {
+      sb.append("\n");
+      sb.append(child.toString());
+    }
+    sb.append("\n</boolean>");
+    return sb.toString();
+  }
+
+  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 = " ";
+    }
+
+    // in case is root or the parent is a group node avoid parenthesis
+    if ((getParent() != null && getParent() instanceof GroupQueryNode)
+        || isRoot())
+      return sb.toString();
+    else
+      return "( " + sb.toString() + " )";
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    BooleanQueryNode clone = (BooleanQueryNode) super.cloneTree();
+
+    // nothing to do here
+
+    return clone;
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/BoostQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/BoostQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/BoostQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/BoostQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,122 @@
+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.QueryNodeException;
+import org.apache.lucene.queryParser.core.messages.QueryParserMessages;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link BoostQueryNode} boosts the QueryNode tree which is under this node.
+ * So, it must only and always have one child.
+ * 
+ * The boost value may vary from 0.0 to 1.0.
+ * 
+ */
+public class BoostQueryNode extends QueryNodeImpl {
+
+  private static final long serialVersionUID = -3929082630855807593L;
+
+  private float value = 0;
+
+  /**
+   * Constructs a boost node
+   * 
+   * @param query
+   *          the query to be boosted
+   * @param value
+   *          the boost value, it may vary from 0.0 to 1.0
+   * 
+   * @throws QueryNodeException
+   */
+  public BoostQueryNode(QueryNode query, float value) throws QueryNodeException {
+    if (query == null) {
+      throw new QueryNodeError(new MessageImpl(
+          QueryParserMessages.NODE_ACTION_NOT_SUPPORTED, "query", "null"));
+    }
+
+    this.value = value;
+    setLeaf(false);
+    allocate();
+    add(query);
+  }
+
+  /**
+   * Returns the single child which this node boosts.
+   * 
+   * @return the single child which this node boosts
+   */
+  public QueryNode getChild() {
+    List<QueryNode> children = getChildren();
+
+    if (children == null || children.size() == 0) {
+      return null;
+    }
+
+    return children.get(0);
+
+  }
+
+  /**
+   * Returns the boost value. It may vary from 0.0 to 1.0.
+   * 
+   * @return the boost value
+   */
+  public float getValue() {
+    return this.value;
+  }
+
+  /**
+   * Returns the boost value parsed to a string.
+   * 
+   * @return the parsed value
+   */
+  private CharSequence getValueString() {
+    Float f = new Float(this.value);
+    if (f == f.longValue())
+      return "" + f.longValue();
+    else
+      return "" + f;
+
+  }
+
+  public String toString() {
+    return "<boost value='" + getValueString() + "'>" + "\n"
+        + getChild().toString() + "\n</boost>";
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    if (getChild() == null)
+      return "";
+    return getChild().toQueryString(escapeSyntaxParser) + "^"
+        + getValueString();
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    BoostQueryNode clone = (BoostQueryNode) super.cloneTree();
+
+    clone.value = this.value;
+
+    return clone;
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/DeletedQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/DeletedQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/DeletedQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/DeletedQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,51 @@
+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;
+import org.apache.lucene.queryParser.core.processors.RemoveDeletedQueryNodesProcessor;
+
+/**
+ * A {@link DeletedQueryNode} represents a node that was deleted from the query
+ * node tree. It can be removed from the tree using the
+ * {@link RemoveDeletedQueryNodesProcessor} processor.
+ */
+public class DeletedQueryNode extends QueryNodeImpl {
+
+  private static final long serialVersionUID = -9151675506000425293L;
+
+  public DeletedQueryNode() {
+    // empty constructor
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escaper) {
+    return "[DELETEDCHILD]";
+  }
+
+  public String toString() {
+    return "<deleted/>";
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    DeletedQueryNode clone = (DeletedQueryNode) super.cloneTree();
+
+    return clone;
+
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,184 @@
+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.Locale;
+
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax.Type;
+
+/**
+ * A {@link FieldQueryNode} represents a element that contains field/text tuple
+ */
+public class FieldQueryNode extends QueryNodeImpl implements TextableQueryNode,
+    FieldableNode {
+
+  private static final long serialVersionUID = 3634521145130758265L;
+
+  /**
+   * The term's field
+   */
+  protected CharSequence field;
+
+  /**
+   * The term's text.
+   */
+  protected CharSequence text;
+
+  /**
+   * The term's begin position.
+   */
+  protected int begin;
+
+  /**
+   * The term's end position.
+   */
+  protected int end;
+
+  /**
+   * The term's position increment.
+   */
+  protected int positionIncrement;
+
+  /**
+   * @param field
+   *          - field name
+   * @param text
+   *          - value
+   * @param begin
+   *          - position in the query string
+   * @param end
+   *          - position in the query string
+   */
+  public FieldQueryNode(CharSequence field, CharSequence text, int begin,
+      int end) {
+    this.field = field;
+    this.text = text;
+    this.begin = begin;
+    this.end = end;
+    this.setLeaf(true);
+
+  }
+
+  CharSequence getTermEscaped(EscapeQuerySyntax escaper) {
+    return escaper.escape(this.text, Locale.getDefault(), Type.NORMAL);
+  }
+
+  CharSequence getTermEscapeQuoted(EscapeQuerySyntax escaper) {
+    return escaper.escape(this.text, Locale.getDefault(), Type.STRING);
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escaper) {
+    if (isDefaultField(this.field)) {
+      return getTermEscaped(escaper);
+    } else {
+      return this.field + ":" + getTermEscaped(escaper);
+    }
+  }
+
+  public String toString() {
+    return "<field start='" + this.begin + "' end='" + this.end + "' field='"
+        + this.field + "' text='" + this.text + "'/>";
+  }
+
+  /**
+   * @return the term
+   */
+  public String getTextAsString() {
+    if (this.text == null)
+      return null;
+    else
+      return this.text.toString();
+  }
+
+  /**
+   * 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();
+  }
+
+  public int getBegin() {
+    return this.begin;
+  }
+
+  public void setBegin(int begin) {
+    this.begin = begin;
+  }
+
+  public int getEnd() {
+    return this.end;
+  }
+
+  public void setEnd(int end) {
+    this.end = end;
+  }
+
+  public CharSequence getField() {
+    return this.field;
+  }
+
+  public void setField(CharSequence field) {
+    this.field = field;
+  }
+
+  public int getPositionIncrement() {
+    return this.positionIncrement;
+  }
+
+  public void setPositionIncrement(int pi) {
+    this.positionIncrement = pi;
+  }
+
+  /**
+   * Returns the term.
+   * 
+   * @return The "original" form of the term.
+   */
+  public CharSequence getText() {
+    return this.text;
+  }
+
+  /**
+   * @param text
+   *          the text to set
+   */
+  public void setText(CharSequence text) {
+    this.text = text;
+  }
+
+  @Override
+  public FieldQueryNode cloneTree() throws CloneNotSupportedException {
+    FieldQueryNode fqn = (FieldQueryNode) super.cloneTree();
+    fqn.begin = this.begin;
+    fqn.end = this.end;
+    fqn.field = this.field;
+    fqn.text = this.text;
+    fqn.positionIncrement = this.positionIncrement;
+    fqn.toQueryStringIgnoreFields = this.toQueryStringIgnoreFields;
+
+    return fqn;
+
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldableNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldableNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldableNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldableNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,45 @@
+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 query node implements {@link FieldableNode} interface to indicate that its
+ * children and itself are associated to a specific field.
+ * 
+ * If it has any children which also implements this interface, it must ensure
+ * the children are associated to the same field.
+ * 
+ */
+public interface FieldableNode extends QueryNode {
+
+  /**
+   * Returns the field associated to the node and every node under it.
+   * 
+   * @return the field name
+   */
+  CharSequence getField();
+
+  /**
+   * Associates the node to a field.
+   * 
+   * @param fieldName
+   *          the field name
+   */
+  void setField(CharSequence fieldName);
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FuzzyQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FuzzyQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FuzzyQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FuzzyQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,98 @@
+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 FuzzyQueryNode} represents a element that contains
+ * field/text/similarity tuple
+ */
+public class FuzzyQueryNode extends FieldQueryNode {
+
+  private static final long serialVersionUID = -1794537213032589441L;
+
+  private float similarity;
+
+  private int prefixLength;
+
+  /**
+   * @param field
+   *          Name of the field query will use.
+   * @param termStr
+   *          Term token to use for building term for the query
+   */
+  /**
+   * @param field
+   *          - Field name
+   * @param term
+   *          - Value
+   * @param minSimilarity
+   *          - similarity value
+   * @param begin
+   *          - position in the query string
+   * @param end
+   *          - position in the query string
+   */
+  public FuzzyQueryNode(CharSequence field, CharSequence term,
+      float minSimilarity, int begin, int end) {
+    super(field, term, begin, end);
+    this.similarity = minSimilarity;
+    setLeaf(true);
+  }
+
+  public void setPrefixLength(int prefixLength) {
+    this.prefixLength = prefixLength;
+  }
+
+  public int getPrefixLength() {
+    return this.prefixLength;
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escaper) {
+    if (isDefaultField(this.field)) {
+      return getTermEscaped(escaper) + "~" + this.similarity;
+    } else {
+      return this.field + ":" + getTermEscaped(escaper) + "~" + this.similarity;
+    }
+  }
+
+  public String toString() {
+    return "<fuzzy field='" + this.field + "' similarity='" + this.similarity
+        + "' term='" + this.text + "'/>";
+  }
+
+  public void setSimilarity(float similarity) {
+    this.similarity = similarity;
+  }
+
+  public FuzzyQueryNode cloneTree() throws CloneNotSupportedException {
+    FuzzyQueryNode clone = (FuzzyQueryNode) super.cloneTree();
+
+    clone.similarity = this.similarity;
+
+    return clone;
+  }
+
+  /**
+   * @return the similarity
+   */
+  public float getSimilarity() {
+    return this.similarity;
+  }
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/GroupQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/GroupQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/GroupQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/GroupQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,83 @@
+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.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 GroupQueryNode} represents a location where the original user typed
+ * real parenthesis on the query string. This class is useful for queries like:
+ * a) a AND b OR c b) ( a AND b) OR c
+ * 
+ * Parenthesis might be used to define the boolean operation precedence.
+ */
+public class GroupQueryNode extends QueryNodeImpl {
+
+  private static final long serialVersionUID = -9204673493869114999L;
+
+  /**
+   * This QueryNode is used to identify parenthesis on the original query string
+   */
+  public GroupQueryNode(QueryNode query) {
+    if (query == null) {
+      throw new QueryNodeError(new MessageImpl(
+          QueryParserMessages.PARAMETER_VALUE_NOT_SUPPORTED, "query", "null"));
+    }
+
+    allocate();
+    setLeaf(false);
+    add(query);
+  }
+
+  public QueryNode getChild() {
+    return getChildren().get(0);
+  }
+
+  public String toString() {
+    return "<group>" + "\n" + getChild().toString() + "\n</group>";
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    if (getChild() == null)
+      return "";
+
+    return "( " + getChild().toQueryString(escapeSyntaxParser) + " )";
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    GroupQueryNode clone = (GroupQueryNode) super.cloneTree();
+
+    return clone;
+  }
+
+  /**
+   * @param child
+   */
+  public void setChild(QueryNode child) {
+    List<QueryNode> list = new ArrayList<QueryNode>();
+    list.add(child);
+    this.set(list);
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/MatchAllDocsQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/MatchAllDocsQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/MatchAllDocsQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/MatchAllDocsQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,49 @@
+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 MatchAllDocsQueryNode} indicates that a query node tree or subtree
+ * will match all documents if executed in the index.
+ */
+public class MatchAllDocsQueryNode extends QueryNodeImpl {
+
+  private static final long serialVersionUID = -7050381275423477809L;
+
+  public MatchAllDocsQueryNode() {
+    // empty constructor
+  }
+
+  public String toString() {
+    return "<matchAllDocs field='*' term='*'>";
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    return "*:*";
+  }
+
+  public MatchAllDocsQueryNode cloneTree() throws CloneNotSupportedException {
+    MatchAllDocsQueryNode clone = (MatchAllDocsQueryNode) super.cloneTree();
+
+    // nothing to clone
+
+    return clone;
+  }
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/MatchNoDocsQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/MatchNoDocsQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/MatchNoDocsQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/MatchNoDocsQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,37 @@
+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 MatchNoDocsQueryNode} indicates that a query node tree or subtree
+ * will not match any documents if executed in the index.
+ * 
+ */
+public class MatchNoDocsQueryNode extends DeletedQueryNode {
+
+  private static final long serialVersionUID = 8081805751679581497L;
+
+  public MatchNoDocsQueryNode() {
+    // empty constructor
+  }
+
+  public String toString() {
+    return "<matchNoDocsQueryNode/>";
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ModifierQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ModifierQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ModifierQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ModifierQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,156 @@
+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.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 ModifierQueryNode} indicates the modifier value (+,-,?,NONE) for
+ * each term on the query string for example "+t1 -t2 t3" will have a tree of
+ * <BooleanQueryNode> <ModifierQueryNode modifier="MOD_REQ"> <t1/>
+ * </ModifierQueryNode> <ModifierQueryNode modifier="MOD_NOT"> <t2/>
+ * </ModifierQueryNode> <t3/> </BooleanQueryNode>
+ * 
+ */
+public class ModifierQueryNode extends QueryNodeImpl {
+
+  private static final long serialVersionUID = -391209837953928169L;
+
+  public enum Modifier {
+    MOD_NONE, MOD_NOT, MOD_REQ;
+
+    public String toString() {
+      switch (this) {
+      case MOD_NONE:
+        return "MOD_NONE";
+      case MOD_NOT:
+        return "MOD_NOT";
+      case MOD_REQ:
+        return "MOD_REQ";
+      }
+      // this code is never executed
+      return "MOD_DEFAULT";
+    }
+
+    public String toDigitString() {
+      switch (this) {
+      case MOD_NONE:
+        return "";
+      case MOD_NOT:
+        return "-";
+      case MOD_REQ:
+        return "+";
+      }
+      // this code is never executed
+      return "";
+    }
+
+    public String toLargeString() {
+      switch (this) {
+      case MOD_NONE:
+        return "";
+      case MOD_NOT:
+        return "NOT ";
+      case MOD_REQ:
+        return "+";
+      }
+      // this code is never executed
+      return "";
+    }
+  }
+
+  private Modifier modifier = Modifier.MOD_NONE;
+
+  /**
+   * Used to store the modifier value on the original query string
+   * 
+   * @param query
+   *          - QueryNode subtree
+   * @param mod
+   *          - Modifier Value
+   */
+  public ModifierQueryNode(QueryNode query, Modifier mod) {
+    if (query == null) {
+      throw new QueryNodeError(new MessageImpl(
+          QueryParserMessages.PARAMETER_VALUE_NOT_SUPPORTED, "query", "null"));
+    }
+
+    allocate();
+    setLeaf(false);
+    add(query);
+    this.modifier = mod;
+  }
+
+  public QueryNode getChild() {
+    return getChildren().get(0);
+  }
+
+  public Modifier getModifier() {
+    return this.modifier;
+  }
+
+  public String toString() {
+    return "<modifier operation='" + this.modifier.toString() + "'>" + "\n"
+        + getChild().toString() + "\n</modifier>";
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    if (getChild() == null)
+      return "";
+
+    String leftParenthensis = "";
+    String rightParenthensis = "";
+
+    if (getChild() != null && getChild() instanceof ModifierQueryNode) {
+      leftParenthensis = "(";
+      rightParenthensis = ")";
+    }
+
+    if (getChild() instanceof BooleanQueryNode) {
+      return this.modifier.toLargeString() + leftParenthensis
+          + getChild().toQueryString(escapeSyntaxParser) + rightParenthensis;
+    } else {
+      return this.modifier.toDigitString() + leftParenthensis
+          + getChild().toQueryString(escapeSyntaxParser) + rightParenthensis;
+    }
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    ModifierQueryNode clone = (ModifierQueryNode) super.cloneTree();
+
+    clone.modifier = this.modifier;
+
+    return clone;
+  }
+
+  /**
+   * @param child
+   */
+  public void setChild(QueryNode child) {
+    List<QueryNode> list = new ArrayList<QueryNode>();
+    list.add(child);
+    this.set(list);
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/NoTokenFoundQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/NoTokenFoundQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/NoTokenFoundQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/NoTokenFoundQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,49 @@
+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 NoTokenFoundQueryNode} is used if a term is convert into no tokens
+ * by the tokenizer/lemmatizer/analyzer (null).
+ */
+public class NoTokenFoundQueryNode extends DeletedQueryNode {
+
+  private static final long serialVersionUID = 7332975497586993833L;
+
+  public NoTokenFoundQueryNode() {
+    super();
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escaper) {
+    return "[NTF]";
+  }
+
+  public String toString() {
+    return "<notokenfound/>";
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    NoTokenFoundQueryNode clone = (NoTokenFoundQueryNode) super.cloneTree();
+
+    // nothing to do here
+
+    return clone;
+  }
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/OpaqueQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/OpaqueQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/OpaqueQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/OpaqueQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,80 @@
+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 OpaqueQueryNode} is used for specify values that are not supposed to
+ * be parsed by the parser. For example: and XPATH query in the middle of a
+ * query string a b @xpath:'/bookstore/book[1]/title' c d
+ */
+public class OpaqueQueryNode extends QueryNodeImpl {
+
+  private static final long serialVersionUID = 0L;
+
+  private CharSequence schema = null;
+
+  private CharSequence value = null;
+
+  /**
+   * @param schema
+   *          - schema identifier
+   * @param value
+   *          - value that was not parsed
+   */
+  public OpaqueQueryNode(CharSequence schema, CharSequence value) {
+    this.setLeaf(true);
+
+    this.schema = schema;
+    this.value = value;
+
+  }
+
+  public String toString() {
+    return "<opaque schema='" + this.schema + "' value='" + this.value + "'/>";
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    return "@" + this.schema + ":'" + this.value + "'";
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    OpaqueQueryNode clone = (OpaqueQueryNode) super.cloneTree();
+
+    clone.schema = this.schema;
+    clone.value = this.value;
+
+    return clone;
+  }
+
+  /**
+   * @return the schema
+   */
+  public CharSequence getSchema() {
+    return this.schema;
+  }
+
+  /**
+   * @return the value
+   */
+  public CharSequence getValue() {
+    return this.value;
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/OrQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/OrQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/OrQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/OrQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,78 @@
+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.Iterator;
+import java.util.List;
+
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link OrQueryNode} represents an OR boolean operation performed on a list
+ * of nodes.
+ * 
+ */
+public class OrQueryNode extends BooleanQueryNode {
+
+  private static final long serialVersionUID = -3692323307688017852L;
+
+  /**
+   * @param clauses
+   *          - the query nodes to be or'ed
+   */
+  public OrQueryNode(List<QueryNode> clauses) {
+    super(clauses);
+    if ((clauses == null) || (clauses.size() == 0)) {
+      throw new IllegalArgumentException(
+          "OR query must have at least one clause");
+    }
+  }
+
+  public String toString() {
+    if (getChildren() == null || getChildren().size() == 0)
+      return "<boolean operation='or'/>";
+    StringBuilder sb = new StringBuilder();
+    sb.append("<boolean operation='or'>");
+    for (QueryNode child : getChildren()) {
+      sb.append("\n");
+      sb.append(child.toString());
+
+    }
+    sb.append("\n</boolean>");
+    return sb.toString();
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    if (getChildren() == null || getChildren().size() == 0)
+      return "";
+
+    StringBuilder sb = new StringBuilder();
+    String filler = "";
+    for (Iterator<QueryNode> it = getChildren().iterator(); it.hasNext();) {
+      sb.append(filler).append(it.next().toQueryString(escapeSyntaxParser));
+      filler = " OR ";
+    }
+
+    // in case is root or the parent is a group node avoid parenthesis
+    if ((getParent() != null && getParent() instanceof GroupQueryNode)
+        || isRoot())
+      return sb.toString();
+    else
+      return "( " + sb.toString() + " )";
+  }
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,100 @@
+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 ParametricQueryNode} represents LE, LT, GE, GT, EQ, NE query.
+ * Example: date >= "2009-10-10" OR price = 200
+ */
+public class ParametricQueryNode extends FieldQueryNode {
+
+  private static final long serialVersionUID = -5770038129741218116L;
+
+  private CompareOperator operator;
+
+  public enum CompareOperator {
+    LE, LT, GE, GT, EQ, NE;
+
+    public String toString() {
+      if (LE.equals(this)) {
+        return "<=";
+      } else if (LT.equals(this)) {
+        return "<";
+      } else if (GE.equals(this)) {
+        return ">=";
+      } else if (GT.equals(this)) {
+        return ">";
+      } else if (EQ.equals(this)) {
+        return "=";
+      } else if (NE.equals(this)) {
+        return "!=";
+      } else {
+        throw new IllegalArgumentException("Unknown operator");
+      }
+    }
+  }
+
+  /**
+   * @param field
+   *          - field name
+   * @param comp
+   *          - CompareOperator
+   * @param value
+   *          - text value
+   * @param begin
+   *          - position in the query string
+   * @param end
+   *          - position in the query string
+   */
+  public ParametricQueryNode(CharSequence field, CompareOperator comp,
+      CharSequence value, int begin, int end) {
+    super(field, value, begin, end);
+    this.operator = comp;
+    setLeaf(true);
+  }
+
+  public CharSequence getOperand() {
+    return getText();
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    return this.field + "" + this.operator.toString() + "\"" + this.text + "\"";
+  }
+
+  public String toString() {
+    return "<parametric field='" + this.field + "' operator='"
+        + this.operator.toString() + "' text='" + this.text + "'/>";
+  }
+
+  public ParametricQueryNode cloneTree() throws CloneNotSupportedException {
+    ParametricQueryNode clone = (ParametricQueryNode) super.cloneTree();
+
+    clone.operator = this.operator;
+
+    return clone;
+  }
+
+  /**
+   * @return the operator
+   */
+  public CompareOperator getOperator() {
+    return this.operator;
+  }
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,120 @@
+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.nodes.ParametricQueryNode.CompareOperator;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+/**
+ * A {@link ParametricRangeQueryNode} represents LE, LT, GE, GT, EQ, NE query.
+ * Example: date >= "2009-10-10" OR price = 200
+ */
+public class ParametricRangeQueryNode extends QueryNodeImpl implements
+    FieldableNode {
+
+  private static final long serialVersionUID = 7120958816535573935L;
+
+  public ParametricRangeQueryNode(ParametricQueryNode lowerBound,
+      ParametricQueryNode upperBound) {
+
+    if (upperBound.getOperator() != CompareOperator.LE
+        && upperBound.getOperator() != CompareOperator.LT) {
+      throw new IllegalArgumentException("upper bound should have "
+          + CompareOperator.LE + " or " + CompareOperator.LT);
+    }
+
+    if (lowerBound.getOperator() != CompareOperator.GE
+        && lowerBound.getOperator() != CompareOperator.GT) {
+      throw new IllegalArgumentException("lower bound should have "
+          + CompareOperator.GE + " or " + CompareOperator.GT);
+    }
+
+    if (upperBound.getField() != lowerBound.getField()
+        || (upperBound.getField() != null && !upperBound.getField().equals(
+            lowerBound.getField()))) {
+
+      throw new IllegalArgumentException(
+          "lower and upper bounds should have the same field name!");
+
+    }
+
+    allocate();
+    setLeaf(false);
+
+    add(lowerBound);
+    add(upperBound);
+
+  }
+
+  public ParametricQueryNode getUpperBound() {
+    return (ParametricQueryNode) getChildren().get(1);
+  }
+
+  public ParametricQueryNode getLowerBound() {
+    return (ParametricQueryNode) getChildren().get(0);
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    return getLowerBound().toQueryString(escapeSyntaxParser) + " AND "
+        + getUpperBound().toQueryString(escapeSyntaxParser);
+  }
+
+  public CharSequence getField() {
+    return getLowerBound().getField();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("<parametricRange>\n\t");
+    sb.append(getUpperBound()).append("\n\t");
+    sb.append(getLowerBound()).append("\n");
+    sb.append("</parametricRange>\n");
+
+    return sb.toString();
+
+  }
+
+  public ParametricRangeQueryNode cloneTree() throws CloneNotSupportedException {
+    ParametricRangeQueryNode clone = (ParametricRangeQueryNode) super
+        .cloneTree();
+
+    // nothing to do here
+
+    return clone;
+  }
+
+  public void setField(CharSequence fieldName) {
+    List<QueryNode> children = getChildren();
+
+    if (children != null) {
+
+      for (QueryNode child : getChildren()) {
+
+        if (child instanceof FieldableNode) {
+          ((FieldableNode) child).setField(fieldName);
+        }
+
+      }
+
+    }
+
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PathQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PathQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PathQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PathQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,214 @@
+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.List;
+import java.util.Locale;
+
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax.Type;
+
+/**
+ * A {@link PathQueryNode} is used for to store queries like
+ * /company/USA/California /product/shoes/brown QueryText are objects that
+ * contain the text, begin position and end position in the query.
+ * 
+ * Example how the text parser creates these objects:
+ * 
+ * List values = ArrayList(); values.add(new PathQueryNode.QueryText("company",
+ * 1, 7)); values.add(new PathQueryNode.QueryText("USA", 9, 12)); values.add(new
+ * PathQueryNode.QueryText("California", 14, 23)); QueryNode q = new
+ * PathQueryNode(values);
+ * 
+ */
+public class PathQueryNode extends QueryNodeImpl {
+
+  private static final long serialVersionUID = -8325921322405804789L;
+
+  public static class QueryText implements Cloneable {
+    CharSequence value = null;
+    /**
+     * != null The term's begin position.
+     */
+    int begin;
+
+    /**
+     * The term's end position.
+     */
+    int end;
+
+    /**
+     * @param value
+     *          - text value
+     * @param begin
+     *          - position in the query string
+     * @param end
+     *          - position in the query string
+     */
+    public QueryText(CharSequence value, int begin, int end) {
+      super();
+      this.value = value;
+      this.begin = begin;
+      this.end = end;
+    }
+
+    public QueryText clone() throws CloneNotSupportedException {
+      QueryText clone = (QueryText) super.clone();
+      clone.value = this.value;
+      clone.begin = this.begin;
+      clone.end = this.end;
+      return clone;
+    }
+
+    /**
+     * @return the value
+     */
+    public CharSequence getValue() {
+      return value;
+    }
+
+    /**
+     * @return the begin
+     */
+    public int getBegin() {
+      return begin;
+    }
+
+    /**
+     * @return the end
+     */
+    public int getEnd() {
+      return end;
+    }
+
+    public String toString() {
+      return value + ", " + begin + ", " + end;
+    }
+  }
+
+  private List<QueryText> values = null;
+
+  /**
+   * @param pathElements
+   *          - List of QueryText objects
+   */
+  public PathQueryNode(List<QueryText> pathElements) {
+    this.values = pathElements;
+    if (pathElements.size() <= 1) {
+      // this should not happen
+      throw new RuntimeException(
+          "PathQuerynode requires more 2 or more path elements.");
+    }
+  }
+
+  /**
+   * Returns the a List with all QueryText elements
+   * 
+   * @return QueryText List size
+   */
+  public List<QueryText> getPathElements() {
+    return values;
+  }
+
+  /**
+   * Returns the a List with all QueryText elements
+   */
+  public void setPathElements(List<QueryText> elements) {
+    this.values = elements;
+  }
+
+  /**
+   * Returns the a specific QueryText element
+   * 
+   * @return QueryText List size
+   */
+  public QueryText getPathElement(int index) {
+    return values.get(index);
+  }
+
+  /**
+   * Returns the CharSequence value of a specific QueryText element
+   * 
+   * @return the CharSequence for a specific QueryText element
+   */
+  public CharSequence getFirstPathElement() {
+    return values.get(0).value;
+  }
+
+  /**
+   * Returns a List QueryText element from position startIndex
+   * 
+   * @return a List QueryText element from position startIndex
+   */
+  public List<QueryText> getPathElements(int startIndex) {
+    List<PathQueryNode.QueryText> rValues = new ArrayList<PathQueryNode.QueryText>();
+    for (int i = startIndex; i < this.values.size(); i++) {
+      try {
+        rValues.add(this.values.get(i).clone());
+      } catch (CloneNotSupportedException e) {
+        // this will not happen
+      }
+    }
+    return rValues;
+  }
+
+  private CharSequence getPathString() {
+    StringBuffer path = new StringBuffer();
+
+    for (QueryText pathelement : values) {
+      path.append("/").append(pathelement.value);
+    }
+    return path.toString();
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escaper) {
+    StringBuffer path = new StringBuffer();
+    path.append("/").append(getFirstPathElement());
+
+    for (QueryText pathelement : getPathElements(1)) {
+      CharSequence value = escaper.escape(pathelement.value, Locale
+          .getDefault(), Type.STRING);
+      path.append("/\"").append(value).append("\"");
+    }
+    return path.toString();
+  }
+
+  public String toString() {
+    QueryText text = this.values.get(0);
+
+    return "<path start='" + text.begin + "' end='" + text.end + "' path='"
+        + getPathString() + "'/>";
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    PathQueryNode clone = (PathQueryNode) super.cloneTree();
+
+    // copy children
+    if (this.values != null) {
+      List<QueryText> localValues = new ArrayList<QueryText>();
+      for (QueryText value : this.values) {
+        localValues.add(value.clone());
+      }
+      clone.values = localValues;
+    }
+
+    return clone;
+  }
+
+}

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

Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PhraseSlopQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PhraseSlopQueryNode.java?rev=800191&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PhraseSlopQueryNode.java (added)
+++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/PhraseSlopQueryNode.java Mon Aug  3 03:38:44 2009
@@ -0,0 +1,109 @@
+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.QueryNodeException;
+import org.apache.lucene.queryParser.core.QueryNodeParseException;
+import org.apache.lucene.queryParser.core.messages.QueryParserMessages;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+
+public class PhraseSlopQueryNode extends QueryNodeImpl implements FieldableNode {
+
+  private static final long serialVersionUID = 0L;
+
+  private int value = 0;
+
+  /**
+   * @throws QueryNodeException
+   * @throws QueryNodeParseException
+   * @exception QueryNodeParseException
+   *              throw in overridden method to disallow
+   */
+  public PhraseSlopQueryNode(QueryNode query, int value)
+      throws QueryNodeException {
+    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 "<phraseslop value='" + getValueString() + "'>" + "\n"
+        + getChild().toString() + "\n</phraseslop>";
+  }
+
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    if (getChild() == null)
+      return "";
+    return getChild().toQueryString(escapeSyntaxParser) + "~"
+        + getValueString();
+  }
+
+  public QueryNode cloneTree() throws CloneNotSupportedException {
+    PhraseSlopQueryNode clone = (PhraseSlopQueryNode) 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/PhraseSlopQueryNode.java
------------------------------------------------------------------------------
    svn:eol-style = native