Return-Path: Delivered-To: apmail-lucene-java-commits-archive@www.apache.org Received: (qmail 56906 invoked from network); 3 Aug 2009 03:39:25 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 3 Aug 2009 03:39:25 -0000 Received: (qmail 54736 invoked by uid 500); 3 Aug 2009 03:39:30 -0000 Delivered-To: apmail-lucene-java-commits-archive@lucene.apache.org Received: (qmail 54654 invoked by uid 500); 3 Aug 2009 03:39:29 -0000 Mailing-List: contact java-commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: java-dev@lucene.apache.org Delivered-To: mailing list java-commits@lucene.apache.org Received: (qmail 54645 invoked by uid 99); 3 Aug 2009 03:39:29 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 03 Aug 2009 03:39:29 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 03 Aug 2009 03:39:19 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id BFFD323888FD; Mon, 3 Aug 2009 03:38:58 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r800191 [5/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... Date: Mon, 03 Aug 2009 03:38:50 -0000 To: java-commits@lucene.apache.org From: buschmi@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20090803033858.BFFD323888FD@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/MultiPhraseQueryNodeBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/MultiPhraseQueryNodeBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/MultiPhraseQueryNodeBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/MultiPhraseQueryNodeBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,84 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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 java.util.List; +import java.util.TreeMap; + +import org.apache.lucene.index.Term; +import org.apache.lucene.queryParser.core.QueryNodeException; +import org.apache.lucene.queryParser.core.builders.QueryTreeBuilder; +import org.apache.lucene.queryParser.core.nodes.FieldQueryNode; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.queryParser.original.nodes.MultiPhraseQueryNode; +import org.apache.lucene.search.MultiPhraseQuery; +import org.apache.lucene.search.TermQuery; + +/** + * Builds a {@link MultiPhraseQuery} object from a {@link MultiPhraseQueryNode} + * object. + */ +public class MultiPhraseQueryNodeBuilder implements OriginalQueryBuilder { + + public MultiPhraseQueryNodeBuilder() { + // empty constructor + } + + public MultiPhraseQuery build(QueryNode queryNode) throws QueryNodeException { + MultiPhraseQueryNode phraseNode = (MultiPhraseQueryNode) queryNode; + + MultiPhraseQuery phraseQuery = new MultiPhraseQuery(); + + List children = phraseNode.getChildren(); + + if (children != null) { + TreeMap> positionTermMap = new TreeMap>(); + + for (QueryNode child : children) { + FieldQueryNode termNode = (FieldQueryNode) child; + TermQuery termQuery = (TermQuery) termNode + .getTag(QueryTreeBuilder.QUERY_TREE_BUILDER_TAGID); + List termList = positionTermMap.get(termNode + .getPositionIncrement()); + + if (termList == null) { + termList = new LinkedList(); + positionTermMap.put(termNode.getPositionIncrement(), termList); + + } + + termList.add(termQuery.getTerm()); + + } + + for (int positionIncrement : positionTermMap.keySet()) { + List termList = positionTermMap.get(positionIncrement); + + phraseQuery.add(termList.toArray(new Term[termList.size()]), + positionIncrement); + + } + + } + + return phraseQuery; + + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/MultiPhraseQueryNodeBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalBooleanQueryNodeBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalBooleanQueryNodeBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalBooleanQueryNodeBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalBooleanQueryNodeBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,109 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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.QueryNodeException; +import org.apache.lucene.queryParser.core.builders.QueryTreeBuilder; +import org.apache.lucene.queryParser.core.messages.QueryParserMessages; +import org.apache.lucene.queryParser.core.nodes.ModifierQueryNode; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.queryParser.core.nodes.ModifierQueryNode.Modifier; +import org.apache.lucene.queryParser.original.nodes.OriginalBooleanQueryNode; +import org.apache.lucene.queryParser.original.parser.EscapeQuerySyntaxImpl; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Similarity; +import org.apache.lucene.search.BooleanQuery.TooManyClauses; + +/** + * This builder does the same as the {@link BooleanQueryNodeBuilder}, but this + * considers if the built {@link BooleanQuery} should have its coord disabled or + * not.
+ * + * @see BooleanQueryNodeBuilder + * @see BooleanQuery + * @see Similarity#coord(int, int) + */ +public class OriginalBooleanQueryNodeBuilder implements OriginalQueryBuilder { + + public OriginalBooleanQueryNodeBuilder() { + // empty constructor + } + + public BooleanQuery build(QueryNode queryNode) throws QueryNodeException { + OriginalBooleanQueryNode booleanNode = (OriginalBooleanQueryNode) queryNode; + + BooleanQuery bQuery = new BooleanQuery(booleanNode.isDisableCoord()); + List children = booleanNode.getChildren(); + + if (children != null) { + + for (QueryNode child : children) { + Object obj = child.getTag(QueryTreeBuilder.QUERY_TREE_BUILDER_TAGID); + + if (obj != null) { + Query query = (Query) obj; + + try { + bQuery.add(query, getModifierValue(child)); + } catch (TooManyClauses ex) { + + throw new QueryNodeException(new MessageImpl( + QueryParserMessages.TOO_MANY_BOOLEAN_CLAUSES, BooleanQuery + .getMaxClauseCount(), queryNode + .toQueryString(new EscapeQuerySyntaxImpl())), ex); + + } + + } + + } + + } + + return bQuery; + + } + + private static BooleanClause.Occur getModifierValue(QueryNode node) + throws QueryNodeException { + + if (node instanceof ModifierQueryNode) { + ModifierQueryNode mNode = ((ModifierQueryNode) node); + Modifier modifier = mNode.getModifier(); + + if (Modifier.MOD_NONE.equals(modifier)) { + return BooleanClause.Occur.SHOULD; + + } else if (Modifier.MOD_NOT.equals(modifier)) { + return BooleanClause.Occur.MUST_NOT; + + } else { + return BooleanClause.Occur.MUST; + } + } + + return BooleanClause.Occur.SHOULD; + + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalBooleanQueryNodeBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,37 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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.builders.QueryBuilder; +import org.apache.lucene.queryParser.core.builders.QueryTreeBuilder; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.search.Query; + +/** + * This interface should be implemented by every class that wants to build + * {@link Query} objects from {@link QueryNode} objects.
+ * + * @see QueryBuilder + * @see QueryTreeBuilder + */ +public interface OriginalQueryBuilder extends QueryBuilder { + + public Query build(QueryNode queryNode) throws QueryNodeException; + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryTreeBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryTreeBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryTreeBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryTreeBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,78 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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.builders.QueryTreeBuilder; +import org.apache.lucene.queryParser.core.nodes.BooleanQueryNode; +import org.apache.lucene.queryParser.core.nodes.BoostQueryNode; +import org.apache.lucene.queryParser.core.nodes.FieldQueryNode; +import org.apache.lucene.queryParser.core.nodes.FuzzyQueryNode; +import org.apache.lucene.queryParser.core.nodes.GroupQueryNode; +import org.apache.lucene.queryParser.core.nodes.MatchAllDocsQueryNode; +import org.apache.lucene.queryParser.core.nodes.MatchNoDocsQueryNode; +import org.apache.lucene.queryParser.core.nodes.ModifierQueryNode; +import org.apache.lucene.queryParser.core.nodes.PrefixWildcardQueryNode; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.queryParser.core.nodes.SlopQueryNode; +import org.apache.lucene.queryParser.core.nodes.TokenizedPhraseQueryNode; +import org.apache.lucene.queryParser.core.nodes.WildcardQueryNode; +import org.apache.lucene.queryParser.original.nodes.OriginalBooleanQueryNode; +import org.apache.lucene.queryParser.original.nodes.MultiPhraseQueryNode; +import org.apache.lucene.queryParser.original.nodes.RangeQueryNode; +import org.apache.lucene.queryParser.original.processors.OriginalQueryNodeProcessorPipeline; +import org.apache.lucene.search.Query; + +/** + * This query tree builder only defines the necessary map to build a + * {@link Query} tree object. It should be used to generate a {@link Query} tree + * object from a query node tree processed by a + * {@link OriginalQueryNodeProcessorPipeline}.
+ * + * @see QueryTreeBuilder + * @see OriginalQueryNodeProcessorPipeline + */ +public class OriginalQueryTreeBuilder extends QueryTreeBuilder implements + OriginalQueryBuilder { + + public OriginalQueryTreeBuilder() { + setBuilder(GroupQueryNode.class, new GroupQueryNodeBuilder()); + setBuilder(FieldQueryNode.class, new FieldQueryNodeBuilder()); + setBuilder(BooleanQueryNode.class, new BooleanQueryNodeBuilder()); + setBuilder(FuzzyQueryNode.class, new FuzzyQueryNodeBuilder()); + setBuilder(BoostQueryNode.class, new BoostQueryNodeBuilder()); + setBuilder(ModifierQueryNode.class, new ModifierQueryNodeBuilder()); + setBuilder(WildcardQueryNode.class, new WildcardQueryNodeBuilder()); + setBuilder(TokenizedPhraseQueryNode.class, new PhraseQueryNodeBuilder()); + setBuilder(MatchNoDocsQueryNode.class, new MatchNoDocsQueryNodeBuilder()); + setBuilder(PrefixWildcardQueryNode.class, + new PrefixWildcardQueryNodeBuilder()); + setBuilder(RangeQueryNode.class, new RangeQueryNodeBuilder()); + setBuilder(SlopQueryNode.class, new SlopQueryNodeBuilder()); + setBuilder(OriginalBooleanQueryNode.class, + new OriginalBooleanQueryNodeBuilder()); + setBuilder(MultiPhraseQueryNode.class, new MultiPhraseQueryNodeBuilder()); + setBuilder(MatchAllDocsQueryNode.class, new MatchAllDocsQueryNodeBuilder()); + + } + + public Query build(QueryNode queryNode) throws QueryNodeException { + return (Query) super.build(queryNode); + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/OriginalQueryTreeBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PhraseQueryNodeBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PhraseQueryNodeBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PhraseQueryNodeBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PhraseQueryNodeBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,64 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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.builders.QueryTreeBuilder; +import org.apache.lucene.queryParser.core.nodes.FieldQueryNode; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.queryParser.core.nodes.TokenizedPhraseQueryNode; +import org.apache.lucene.search.PhraseQuery; +import org.apache.lucene.search.TermQuery; + +/** + * Builds a {@link PhraseQuery} object from a {@link TokenizedPhraseQueryNode} + * object. + */ +public class PhraseQueryNodeBuilder implements OriginalQueryBuilder { + + public PhraseQueryNodeBuilder() { + // empty constructor + } + + public PhraseQuery build(QueryNode queryNode) throws QueryNodeException { + TokenizedPhraseQueryNode phraseNode = (TokenizedPhraseQueryNode) queryNode; + + PhraseQuery phraseQuery = new PhraseQuery(); + + List children = phraseNode.getChildren(); + + if (children != null) { + + for (QueryNode child : children) { + TermQuery termQuery = (TermQuery) child + .getTag(QueryTreeBuilder.QUERY_TREE_BUILDER_TAGID); + FieldQueryNode termNode = (FieldQueryNode) child; + + phraseQuery.add(termQuery.getTerm(), termNode.getPositionIncrement()); + + } + + } + + return phraseQuery; + + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PhraseQueryNodeBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PrefixWildcardQueryNodeBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PrefixWildcardQueryNodeBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PrefixWildcardQueryNodeBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PrefixWildcardQueryNodeBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,44 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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.index.Term; +import org.apache.lucene.queryParser.core.QueryNodeException; +import org.apache.lucene.queryParser.core.nodes.PrefixWildcardQueryNode; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.search.PrefixQuery; + +/** + * Builds a {@link PrefixQuery} object from a {@link PrefixWildcardQueryNode} + * object. + */ +public class PrefixWildcardQueryNodeBuilder implements OriginalQueryBuilder { + + public PrefixWildcardQueryNodeBuilder() { + // empty constructor + } + + public PrefixQuery build(QueryNode queryNode) throws QueryNodeException { + PrefixWildcardQueryNode wildcardNode = (PrefixWildcardQueryNode) queryNode; + + return new PrefixQuery(new Term(wildcardNode.getFieldAsString(), + wildcardNode.getTextAsString())); + + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/PrefixWildcardQueryNodeBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/RangeQueryNodeBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/RangeQueryNodeBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/RangeQueryNodeBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/RangeQueryNodeBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,63 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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.nodes.ParametricQueryNode; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.queryParser.core.nodes.ParametricQueryNode.CompareOperator; +import org.apache.lucene.queryParser.original.nodes.RangeQueryNode; +import org.apache.lucene.search.TermRangeQuery; + +/** + * Builds a {@link TermRangeQuery} object from a {@link RangeQueryNode} object. + */ +public class RangeQueryNodeBuilder implements OriginalQueryBuilder { + + public RangeQueryNodeBuilder() { + // empty constructor + } + + public TermRangeQuery build(QueryNode queryNode) throws QueryNodeException { + RangeQueryNode rangeNode = (RangeQueryNode) queryNode; + ParametricQueryNode upper = rangeNode.getUpperBound(); + ParametricQueryNode lower = rangeNode.getLowerBound(); + + boolean lowerInclusive = false; + boolean upperInclusive = false; + + if (upper.getOperator() == CompareOperator.LE) { + upperInclusive = true; + } + + if (lower.getOperator() == CompareOperator.GE) { + lowerInclusive = true; + } + + String field = rangeNode.getField().toString(); + + TermRangeQuery rangeQuery = new TermRangeQuery(field, lower + .getTextAsString(), upper.getTextAsString(), lowerInclusive, + upperInclusive, rangeNode.getCollator()); + rangeQuery.setRewriteMethod(rangeNode.getMultiTermRewriteMethod()); + + return rangeQuery; + + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/RangeQueryNodeBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/SlopQueryNodeBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/SlopQueryNodeBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/SlopQueryNodeBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/SlopQueryNodeBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,57 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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.builders.QueryTreeBuilder; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.queryParser.core.nodes.SlopQueryNode; +import org.apache.lucene.search.MultiPhraseQuery; +import org.apache.lucene.search.PhraseQuery; +import org.apache.lucene.search.Query; + +/** + * This builder basically reads the {@link Query} object set on the + * {@link SlopQueryNode} child using + * {@link QueryTreeBuilder#QUERY_TREE_BUILDER_TAGID} and applies the slop value + * defined in the {@link SlopQueryNode}. + */ +public class SlopQueryNodeBuilder implements OriginalQueryBuilder { + + public SlopQueryNodeBuilder() { + // empty constructor + } + + public Query build(QueryNode queryNode) throws QueryNodeException { + SlopQueryNode phraseSlopNode = (SlopQueryNode) queryNode; + + Query query = (Query) phraseSlopNode.getChild().getTag( + QueryTreeBuilder.QUERY_TREE_BUILDER_TAGID); + + if (query instanceof PhraseQuery) { + ((PhraseQuery) query).setSlop(phraseSlopNode.getValue()); + + } else { + ((MultiPhraseQuery) query).setSlop(phraseSlopNode.getValue()); + } + + return query; + + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/SlopQueryNodeBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/WildcardQueryNodeBuilder.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/WildcardQueryNodeBuilder.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/WildcardQueryNodeBuilder.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/WildcardQueryNodeBuilder.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,44 @@ +package org.apache.lucene.queryParser.original.builders; + +/** + * 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.index.Term; +import org.apache.lucene.queryParser.core.QueryNodeException; +import org.apache.lucene.queryParser.core.nodes.QueryNode; +import org.apache.lucene.queryParser.core.nodes.WildcardQueryNode; +import org.apache.lucene.search.WildcardQuery; + +/** + * Builds a {@link WildcardQuery} object from a {@link WildcardQueryNode} + * object. + */ +public class WildcardQueryNodeBuilder implements OriginalQueryBuilder { + + public WildcardQueryNodeBuilder() { + // empty constructor + } + + public WildcardQuery build(QueryNode queryNode) throws QueryNodeException { + WildcardQueryNode wildcardNode = (WildcardQueryNode) queryNode; + + return new WildcardQuery(new Term(wildcardNode.getFieldAsString(), + wildcardNode.getTextAsString())); + + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/WildcardQueryNodeBuilder.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/package.html URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/package.html?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/package.html (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/package.html Mon Aug 3 03:38:44 2009 @@ -0,0 +1,35 @@ + + + + + + + + +

Original Lucene Query Node Builders

+

+The package org.apache.lucene.queryParser.original.builders contains all the builders needed +to build a Lucene Query object from a query node tree. These builders expect the query node tree was +already processed by the {@link org.apache.lucene.queryParser.original.processors.OriginalQueryNodeProcessorPipeline}. +

+

+{@link org.apache.lucene.queryParser.original.builders.OriginalQueryTreeBuilder} is a builder that already contains a defined map that maps each QueryNode object +with its respective builder. +

+ + Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/builders/package.html ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttribute.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttribute.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttribute.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttribute.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,33 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.AllowLeadingWildcardProcessor; +import org.apache.lucene.util.Attribute; + +/** + * This attribute is used by {@link AllowLeadingWildcardProcessor} processor and + * must be defined in the {@link QueryConfigHandler}. It basically tells the + * processor if it should allow leading wildcard.
+ * + */ +public interface AllowLeadingWildcardAttribute extends Attribute { + public void setAllowLeadingWildcard(boolean allowLeadingWildcard); + public boolean isAllowLeadingWildcard(); +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttribute.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttributeImpl.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttributeImpl.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttributeImpl.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttributeImpl.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,80 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.AllowLeadingWildcardProcessor; +import org.apache.lucene.util.AttributeImpl; + +/** + * This attribute is used by {@link AllowLeadingWildcardProcessor} processor and + * must be defined in the {@link QueryConfigHandler}. It basically tells the + * processor if it should allow leading wildcard.
+ * + * @see org.apache.lucene.queryParser.original.config.AllowLeadingWildcardAttribute + */ +public class AllowLeadingWildcardAttributeImpl extends AttributeImpl + implements AllowLeadingWildcardAttribute { + + private static final long serialVersionUID = -2804763012723049527L; + + private boolean allowLeadingWildcard = true; + + public AllowLeadingWildcardAttributeImpl() { + allowLeadingWildcard = true; // default in 2.4 + } + + public void setAllowLeadingWildcard(boolean allowLeadingWildcard) { + this.allowLeadingWildcard = allowLeadingWildcard; + } + + public boolean isAllowLeadingWildcard() { + return this.allowLeadingWildcard; + } + + public void clear() { + throw new UnsupportedOperationException(); + } + + public void copyTo(AttributeImpl target) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object other) { + + if (other instanceof AllowLeadingWildcardAttributeImpl + && ((AllowLeadingWildcardAttributeImpl) other).allowLeadingWildcard == this.allowLeadingWildcard) { + + return true; + + } + + return false; + + } + + public int hashCode() { + return this.allowLeadingWildcard ? -1 : Integer.MAX_VALUE; + } + + public String toString() { + return ""; + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AllowLeadingWildcardAttributeImpl.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttribute.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttribute.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttribute.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttribute.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,35 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.AnalyzerQueryNodeProcessor; +import org.apache.lucene.util.Attribute; + +/** + * This attribute is used by {@link AnalyzerQueryNodeProcessor} processor and + * must be defined in the {@link QueryConfigHandler}. It provides to this + * processor the {@link Analyzer}, if there is one, which will be used to + * analyze the query terms.
+ * + */ +public interface AnalyzerAttribute extends Attribute { + public void setAnalyzer(Analyzer analyzer); + public Analyzer getAnalyzer(); +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttribute.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttributeImpl.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttributeImpl.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttributeImpl.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttributeImpl.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,87 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.AnalyzerQueryNodeProcessor; +import org.apache.lucene.util.AttributeImpl; + +/** + * This attribute is used by {@link AnalyzerQueryNodeProcessor} processor and + * must be defined in the {@link QueryConfigHandler}. It provides to this + * processor the {@link Analyzer}, if there is one, which will be used to + * analyze the query terms.
+ * + * @see org.apache.lucene.queryParser.original.config.AnalyzerAttribute + */ +public class AnalyzerAttributeImpl extends AttributeImpl + implements AnalyzerAttribute { + + private static final long serialVersionUID = -6804760312723049526L; + + private Analyzer analyzer; + + public AnalyzerAttributeImpl() { + analyzer = null; //default value 2.4 + } + + public void setAnalyzer(Analyzer analyzer) { + this.analyzer = analyzer; + } + + public Analyzer getAnalyzer() { + return this.analyzer; + } + + public void clear() { + throw new UnsupportedOperationException(); + } + + public void copyTo(AttributeImpl target) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object other) { + + if (other instanceof AnalyzerAttributeImpl) { + AnalyzerAttributeImpl analyzerAttr = (AnalyzerAttributeImpl) other; + + if (analyzerAttr.analyzer == this.analyzer + || (this.analyzer != null && analyzerAttr.analyzer != null && this.analyzer + .equals(analyzerAttr.analyzer))) { + + return true; + + } + + } + + return false; + + } + + public int hashCode() { + return (this.analyzer == null) ? 0 : this.analyzer.hashCode(); + } + + public String toString() { + return ""; + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/AnalyzerAttributeImpl.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttribute.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttribute.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttribute.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttribute.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,35 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.FieldConfig; +import org.apache.lucene.queryParser.original.processors.MultiFieldQueryNodeProcessor; +import org.apache.lucene.util.Attribute; + +/** + * This attribute is used by {@link MultiFieldQueryNodeProcessor} processor and + * it should be defined in a {@link FieldConfig}. This processor uses this + * attribute to define which boost a specific field should have when none is + * defined to it.
+ *
+ * + */ +public interface BoostAttribute extends Attribute { + public void setBoost(float boost); + public float getBoost(); +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttribute.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttributeImpl.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttributeImpl.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttributeImpl.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttributeImpl.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,81 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.FieldConfig; +import org.apache.lucene.queryParser.original.processors.MultiFieldQueryNodeProcessor; +import org.apache.lucene.util.AttributeImpl; + +/** + * This attribute is used by {@link MultiFieldQueryNodeProcessor} processor and + * it should be defined in a {@link FieldConfig}. This processor uses this + * attribute to define which boost a specific field should have when none is + * defined to it.
+ *
+ * + * @see org.apache.lucene.queryParser.original.config.BoostAttribute + */ +public class BoostAttributeImpl extends AttributeImpl + implements BoostAttribute { + + private static final long serialVersionUID = -2104763012523049527L; + + private float boost = 1.0f; + + public BoostAttributeImpl() { + // empty constructor + } + + public void setBoost(float boost) { + this.boost = boost; + } + + public float getBoost() { + return this.boost; + } + + public void clear() { + throw new UnsupportedOperationException(); + } + + public void copyTo(AttributeImpl target) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object other) { + + if (other instanceof BoostAttributeImpl && other != null + && ((BoostAttributeImpl) other).boost == this.boost) { + + return true; + + } + + return false; + + } + + public int hashCode() { + return Float.valueOf(this.boost).hashCode(); + } + + public String toString() { + return ""; + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/BoostAttributeImpl.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttribute.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttribute.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttribute.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttribute.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,43 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.document.DateTools; +import org.apache.lucene.document.DateTools.Resolution; +import org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.nodes.RangeQueryNode; +import org.apache.lucene.queryParser.original.processors.ParametricRangeQueryNodeProcessor; +import org.apache.lucene.util.Attribute; + +/** + * This attribute is used by {@link ParametricRangeQueryNodeProcessor} processor + * and must be defined in the {@link QueryConfigHandler}. This attribute tells + * the processor which {@link Resolution} to use when parsing the date.
+ * + */ +public interface DateResolutionAttribute extends Attribute { + /** + * Sets the default date resolution used by {@link RangeQueryNode}s for + * fields for which no specific date resolutions has been set. Field + * specific resolutions can be set with + * + * @param dateResolution the default date resolution to set + */ + public void setDateResolution(DateTools.Resolution dateResolution); + public DateTools.Resolution getDateResolution(); +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttribute.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttributeImpl.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttributeImpl.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttributeImpl.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttributeImpl.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,87 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.document.DateTools; +import org.apache.lucene.document.DateTools.Resolution; +import org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.ParametricRangeQueryNodeProcessor; +import org.apache.lucene.util.AttributeImpl; + +/** + * This attribute is used by {@link ParametricRangeQueryNodeProcessor} processor + * and must be defined in the {@link QueryConfigHandler}. This attribute tells + * the processor which {@link Resolution} to use when parsing the date.
+ * + * @see org.apache.lucene.queryParser.original.config.DateResolutionAttribute + */ +public class DateResolutionAttributeImpl extends AttributeImpl + implements DateResolutionAttribute { + + private static final long serialVersionUID = -6804360312723049526L; + + private DateTools.Resolution dateResolution = null; + + public DateResolutionAttributeImpl() { + dateResolution = null; //default in 2.4 + } + + public void setDateResolution(DateTools.Resolution dateResolution) { + this.dateResolution = dateResolution; + } + + public DateTools.Resolution getDateResolution() { + return this.dateResolution; + } + + public void clear() { + throw new UnsupportedOperationException(); + } + + public void copyTo(AttributeImpl target) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object other) { + + if (other instanceof DateResolutionAttributeImpl) { + DateResolutionAttributeImpl dateResAttr = (DateResolutionAttributeImpl) other; + + if (dateResAttr.getDateResolution() == getDateResolution() + || dateResAttr.getDateResolution().equals(getDateResolution())) { + + return true; + + } + + } + + return false; + + } + + public int hashCode() { + return (this.dateResolution == null) ? 0 : this.dateResolution.hashCode(); + } + + public String toString() { + return ""; + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DateResolutionAttributeImpl.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttribute.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttribute.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttribute.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttribute.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,39 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.GroupQueryNodeProcessor; +import org.apache.lucene.util.Attribute; + +/** + * This attribute is used by {@link GroupQueryNodeProcessor} processor and must + * be defined in the {@link QueryConfigHandler}. This attribute tells the + * processor which is the default boolean operator when no operator is defined + * between terms.
+ * + */ +public interface DefaultOperatorAttribute extends Attribute { + public static enum Operator { + AND, OR; + } + + public void setOperator(Operator operator); + public Operator getOperator(); +} + Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttribute.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttributeImpl.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttributeImpl.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttributeImpl.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttributeImpl.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,89 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.GroupQueryNodeProcessor; +import org.apache.lucene.util.AttributeImpl; + +/** + * This attribute is used by {@link GroupQueryNodeProcessor} processor and must + * be defined in the {@link QueryConfigHandler}. This attribute tells the + * processor which is the default boolean operator when no operator is defined + * between terms.
+ * + * @see org.apache.lucene.queryParser.original.config.DefaultOperatorAttribute + */ +public class DefaultOperatorAttributeImpl extends AttributeImpl + implements DefaultOperatorAttribute { + + private static final long serialVersionUID = -6804760312723049526L; + + private Operator operator = Operator.OR; + + public DefaultOperatorAttributeImpl() { + // empty constructor + } + + public void setOperator(Operator operator) { + + if (operator == null) { + throw new IllegalArgumentException("default operator cannot be null!"); + } + + this.operator = operator; + + } + + public Operator getOperator() { + return this.operator; + } + + public void clear() { + throw new UnsupportedOperationException(); + } + + public void copyTo(AttributeImpl target) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object other) { + + if (other instanceof DefaultOperatorAttributeImpl) { + DefaultOperatorAttributeImpl defaultOperatorAttr = (DefaultOperatorAttributeImpl) other; + + if (defaultOperatorAttr.getOperator() == this.getOperator()) { + return true; + + } + + } + + return false; + + } + + public int hashCode() { + return getOperator().hashCode() * 31; + } + + public String toString() { + return ""; + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultOperatorAttributeImpl.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttribute.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttribute.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttribute.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttribute.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,34 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.PhraseSlopQueryNodeProcessor; +import org.apache.lucene.util.Attribute; + +/** + * This attribute is used by {@link PhraseSlopQueryNodeProcessor} processor and + * must be defined in the {@link QueryConfigHandler}. This attribute tells the + * processor what is the default phrase slop when no slop is defined in a + * phrase.
+ * + */ +public interface DefaultPhraseSlopAttribute extends Attribute { + public void setDefaultPhraseSlop(int defaultPhraseSlop); + public int getDefaultPhraseSlop(); +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttribute.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttributeImpl.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttributeImpl.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttributeImpl.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttributeImpl.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,82 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.QueryConfigHandler; +import org.apache.lucene.queryParser.original.processors.PhraseSlopQueryNodeProcessor; +import org.apache.lucene.util.AttributeImpl; + +/** + * This attribute is used by {@link PhraseSlopQueryNodeProcessor} processor and + * must be defined in the {@link QueryConfigHandler}. This attribute tells the + * processor what is the default phrase slop when no slop is defined in a + * phrase.
+ * + * @see org.apache.lucene.queryParser.original.config.DefaultOperatorAttribute + */ +public class DefaultPhraseSlopAttributeImpl extends AttributeImpl + implements DefaultPhraseSlopAttribute { + + private static final long serialVersionUID = -2104763012527049527L; + + private int defaultPhraseSlop = 0; + + public DefaultPhraseSlopAttributeImpl() { + defaultPhraseSlop = 0; //default value in 2.4 + } + + public void setDefaultPhraseSlop(int defaultPhraseSlop) { + this.defaultPhraseSlop = defaultPhraseSlop; + } + + public int getDefaultPhraseSlop() { + return this.defaultPhraseSlop; + } + + public void clear() { + throw new UnsupportedOperationException(); + } + + public void copyTo(AttributeImpl target) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object other) { + + if (other instanceof DefaultPhraseSlopAttributeImpl + && other != null + && ((DefaultPhraseSlopAttributeImpl) other).defaultPhraseSlop == this.defaultPhraseSlop) { + + return true; + + } + + return false; + + } + + public int hashCode() { + return Integer.valueOf(this.defaultPhraseSlop).hashCode(); + } + + public String toString() { + return ""; + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/DefaultPhraseSlopAttributeImpl.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttribute.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttribute.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttribute.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttribute.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,35 @@ +package org.apache.lucene.queryParser.original.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.Map; + +import org.apache.lucene.queryParser.core.config.FieldConfig; +import org.apache.lucene.util.Attribute; + +/** + * This attribute enables the user to define a default boost per field. + * it's used by {@link FieldBoostMapFCListener#buildFieldConfig(FieldConfig)} + */ +public interface FieldBoostMapAttribute extends Attribute { + /** + * @param boosts a mapping from field name to its default boost + */ + public void setFieldBoostMap(Map boosts); + public Map getFieldBoostMap(); +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttribute.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttributeImpl.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttributeImpl.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttributeImpl.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttributeImpl.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,89 @@ +package org.apache.lucene.queryParser.original.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.LinkedHashMap; +import java.util.Map; + +import org.apache.lucene.queryParser.core.config.FieldConfig; +import org.apache.lucene.queryParser.original.processors.MultiFieldQueryNodeProcessor; +import org.apache.lucene.util.AttributeImpl; + +/** + * This attribute is used by {@link MultiFieldQueryNodeProcessor} processor and + * it should be defined in a {@link FieldConfig}. This processor uses this + * attribute to define which boost a specific field should have when none is + * defined to it.
+ *
+ * + * @see org.apache.lucene.queryParser.original.config.BoostAttribute + */ +public class FieldBoostMapAttributeImpl extends AttributeImpl + implements FieldBoostMapAttribute { + + private static final long serialVersionUID = -2104763012523049527L; + + private Map boosts = new LinkedHashMap(); + + + public FieldBoostMapAttributeImpl() { + // empty constructor + } + + public void setFieldBoostMap(Map boosts) { + this.boosts = boosts; + } + + public Map getFieldBoostMap() { + return this.boosts; + } + + public void clear() { + throw new UnsupportedOperationException(); + } + + public void copyTo(AttributeImpl target) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object other) { + + if (other instanceof FieldBoostMapAttributeImpl && other != null + && ((FieldBoostMapAttributeImpl) other).boosts.equals(this.boosts) ) { + + return true; + + } + + return false; + + } + + public int hashCode() { + final int prime = 97; + if (this.boosts != null) + return this.boosts.hashCode() * prime; + else + return Float.valueOf(prime).hashCode(); + } + + public String toString() { + return ""; + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapAttributeImpl.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapFCListener.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapFCListener.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapFCListener.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapFCListener.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,59 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.queryParser.core.config.FieldConfig; +import org.apache.lucene.queryParser.core.config.FieldConfigListener; +import org.apache.lucene.queryParser.core.config.QueryConfigHandler; + +/** + * This listener listens for every field configuration request and assign a + * {@link BoostAttribute} to the equivalent {@link FieldConfig} based on a + * defined map: fieldName -> boostValue store in {@link FieldBoostMapAttribute} + * in the {@link FieldBoostMapAttribute}. + * + * @see BoostAttribute + * @see FieldBoostMapAttribute + * @see FieldConfig + * @see FieldConfigListener + */ +public class FieldBoostMapFCListener implements FieldConfigListener { + + private static final long serialVersionUID = -5929802948798314067L; + + private QueryConfigHandler config = null; + + public FieldBoostMapFCListener(QueryConfigHandler config) { + this.config = config; + } + + public void buildFieldConfig(FieldConfig fieldConfig) { + if (this.config.hasAttribute(FieldBoostMapAttribute.class)) { + FieldBoostMapAttribute fieldBoostMapAttr = (FieldBoostMapAttribute) this.config.getAttribute(FieldBoostMapAttribute.class); + BoostAttribute boostAttr = (BoostAttribute) fieldConfig.addAttribute(BoostAttribute.class); + + Float boost = fieldBoostMapAttr.getFieldBoostMap().get(fieldConfig.getFieldName()); + + if (boost != null) { + boostAttr.setBoost(boost.floatValue()); + } + + } + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldBoostMapFCListener.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionFCListener.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionFCListener.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionFCListener.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionFCListener.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,73 @@ +package org.apache.lucene.queryParser.original.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 org.apache.lucene.document.DateTools; +import org.apache.lucene.queryParser.core.config.FieldConfig; +import org.apache.lucene.queryParser.core.config.FieldConfigListener; +import org.apache.lucene.queryParser.core.config.QueryConfigHandler; + +/** + * This listener listens for every field configuration request and assign a + * {@link DateResolutionAttribute} to the equivalent {@link FieldConfig} based on a + * defined map: fieldName -> DateTools.Resolution stored in {@link FieldDateResolutionMapAttribute} + * in the {@link DateResolutionAttribute}. + * + * @see DateResolutionAttribute + * @see FieldDateResolutionMapAttribute + * @see FieldConfig + * @see FieldConfigListener + */ +public class FieldDateResolutionFCListener implements FieldConfigListener { + + private static final long serialVersionUID = -5929802948798314067L; + + private QueryConfigHandler config = null; + + public FieldDateResolutionFCListener(QueryConfigHandler config) { + this.config = config; + } + + public void buildFieldConfig(FieldConfig fieldConfig) { + DateResolutionAttribute fieldDateResAttr = (DateResolutionAttribute) fieldConfig + .addAttribute(DateResolutionAttribute.class); + DateTools.Resolution dateRes = null; + + if (this.config.hasAttribute(FieldDateResolutionMapAttribute.class)) { + FieldDateResolutionMapAttribute dateResMapAttr = (FieldDateResolutionMapAttribute) this.config + .addAttribute(FieldDateResolutionMapAttribute.class); + dateRes = dateResMapAttr.getFieldDateResolutionMap().get( + fieldConfig.getFieldName().toString()); + } + + if (dateRes == null) { + + if (this.config.hasAttribute(DateResolutionAttribute.class)) { + DateResolutionAttribute dateResAttr = (DateResolutionAttribute) this.config + .addAttribute(DateResolutionAttribute.class); + dateRes = dateResAttr.getDateResolution(); + + } + + } + + fieldDateResAttr.setDateResolution(dateRes); + + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionFCListener.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttribute.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttribute.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttribute.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttribute.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,35 @@ +package org.apache.lucene.queryParser.original.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.Map; + +import org.apache.lucene.document.DateTools; +import org.apache.lucene.util.Attribute; + +/** + * This attribute enables the user to define a default DateResolution per field. + * it's used by {@link FieldDateResolutionFCListener#buildFieldConfig(org.apache.lucene.queryParser.core.config.FieldConfig)} + */ +public interface FieldDateResolutionMapAttribute extends Attribute { + /** + * @param dateRes a mapping from field name to its default boost + */ + public void setFieldDateResolutionMap(Map dateRes); + public Map getFieldDateResolutionMap(); +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttribute.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttributeImpl.java URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttributeImpl.java?rev=800191&view=auto ============================================================================== --- lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttributeImpl.java (added) +++ lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttributeImpl.java Mon Aug 3 03:38:44 2009 @@ -0,0 +1,86 @@ +package org.apache.lucene.queryParser.original.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.HashMap; +import java.util.Map; + +import org.apache.lucene.document.DateTools; +import org.apache.lucene.document.DateTools.Resolution; +import org.apache.lucene.util.AttributeImpl; + +/** + * This attribute enables the user to define a default DateResolution per field. + * it's used by {@link FieldDateResolutionFCListener#buildFieldConfig(org.apache.lucene.queryParser.core.config.FieldConfig)} + * + * @see FieldDateResolutionMapAttribute + */ +public class FieldDateResolutionMapAttributeImpl extends AttributeImpl + implements FieldDateResolutionMapAttribute { + + private static final long serialVersionUID = -2104763012523049527L; + + private Map dateRes = new HashMap(); + + + public FieldDateResolutionMapAttributeImpl() { + // empty constructor + } + + public void setFieldDateResolutionMap(Map dateRes) { + this.dateRes = dateRes; + } + + public Map getFieldDateResolutionMap() { + return this.dateRes; + } + + public void clear() { + throw new UnsupportedOperationException(); + } + + public void copyTo(AttributeImpl target) { + throw new UnsupportedOperationException(); + } + + public boolean equals(Object other) { + + if (other instanceof FieldDateResolutionMapAttributeImpl && other != null + && ((FieldDateResolutionMapAttributeImpl) other).dateRes.equals(this.dateRes) ) { + + return true; + + } + + return false; + + } + + public int hashCode() { + final int prime = 97; + if (this.dateRes != null) + return this.dateRes.hashCode() * prime; + else + return Float.valueOf(prime).hashCode(); + } + + public String toString() { + return ""; + } + +} Propchange: lucene/java/trunk/contrib/queryparser/src/java/org/apache/lucene/queryParser/original/config/FieldDateResolutionMapAttributeImpl.java ------------------------------------------------------------------------------ svn:eol-style = native