You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by Robert Muir <rc...@gmail.com> on 2013/03/21 18:18:17 UTC

Re: svn commit: r1459400 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/search/ core/src/test/org/apache/lucene/search/

Its not clear to me that this works when IndexSearcher has an executorService.

maybe it does.. but can the test use newSearcher or something?

On Thu, Mar 21, 2013 at 1:04 PM,  <jp...@apache.org> wrote:
> Author: jpountz
> Date: Thu Mar 21 17:04:10 2013
> New Revision: 1459400
>
> URL: http://svn.apache.org/r1459400
> Log:
> LUCENE-4862: Added CollectionTerminatedException to allow permature termination of the collection of a single IndexReader leaf.
>
> Added:
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java   (with props)
>     lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java   (with props)
> Modified:
>     lucene/dev/trunk/lucene/CHANGES.txt
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java
>     lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
>
> Modified: lucene/dev/trunk/lucene/CHANGES.txt
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1459400&r1=1459399&r2=1459400&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/CHANGES.txt (original)
> +++ lucene/dev/trunk/lucene/CHANGES.txt Thu Mar 21 17:04:10 2013
> @@ -116,6 +116,10 @@ New Features
>  * LUCENE-4859: IndexReader now exposes Terms statistics: getDocCount,
>    getSumDocFreq, getSumTotalTermFreq. (Shai Erera)
>
> +* LUCENE-4862: It is now possible to terminate collection of a single
> +  IndexReader leaf by throwing a CollectionTerminatedException in
> +  Collector.collect. (Adrien Grand, Shai Erera)
> +
>  API Changes
>
>  * LUCENE-4844: removed TaxonomyReader.getParent(), you should use
>
> Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java?rev=1459400&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java (added)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CollectionTerminatedException.java Thu Mar 21 17:04:10 2013
> @@ -0,0 +1,34 @@
> +package org.apache.lucene.search;
> +
> +/*
> + * 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.
> + */
> +
> +/** Throw this exception in {@link Collector#collect(int)} to prematurely
> + *  terminate collection of the current leaf.
> + *  <p>Note: IndexSearcher swallows this exception and never re-throws it.
> + *  As a consequence, you should not catch it when calling
> + *  {@link IndexSearcher#search} as it is unnecessary and might hide misuse
> + *  of this exception. */
> +@SuppressWarnings("serial")
> +public final class CollectionTerminatedException extends RuntimeException {
> +
> +  /** Sole constructor. */
> +  public CollectionTerminatedException() {
> +    super();
> +  }
> +
> +}
>
> Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java?rev=1459400&r1=1459399&r2=1459400&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java (original)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java Thu Mar 21 17:04:10 2013
> @@ -134,7 +134,10 @@ public abstract class Collector {
>    /**
>     * Called once for every document matching a query, with the unbased document
>     * number.
> -   *
> +   * <p>Note: The collection of the current segment can be terminated by throwing
> +   * a {@link CollectionTerminatedException}. In this case, the last docs of the
> +   * current {@link AtomicReaderContext} will be skipped and {@link IndexSearcher}
> +   * will swallow the exception and continue collection with the next leaf.
>     * <p>
>     * Note: This is called in an inner search loop. For good search performance,
>     * implementations of this method should not call {@link IndexSearcher#doc(int)} or
>
> Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1459400&r1=1459399&r2=1459400&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
> +++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Thu Mar 21 17:04:10 2013
> @@ -32,7 +32,6 @@ import java.util.concurrent.ExecutorServ
>  import java.util.concurrent.locks.Lock;
>  import java.util.concurrent.locks.ReentrantLock;
>
> -import org.apache.lucene.document.Document;
>  import org.apache.lucene.index.AtomicReaderContext;
>  import org.apache.lucene.index.DirectoryReader; // javadocs
>  import org.apache.lucene.index.IndexReader;
> @@ -595,7 +594,12 @@ public class IndexSearcher {
>        collector.setNextReader(ctx);
>        Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, ctx.reader().getLiveDocs());
>        if (scorer != null) {
> -        scorer.score(collector);
> +        try {
> +          scorer.score(collector);
> +        } catch (CollectionTerminatedException e) {
> +          // collection was terminated prematurely
> +          // continue with the following leaf
> +        }
>        }
>      }
>    }
>
> Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java?rev=1459400&view=auto
> ==============================================================================
> --- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java (added)
> +++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java Thu Mar 21 17:04:10 2013
> @@ -0,0 +1,93 @@
> +package org.apache.lucene.search;
> +
> +/*
> + * Licensed to the Apache Software Foundation (ASF) under one or more
> + * contributor license agreements.  See the NOTICE file distributed with
> + * this work for additional information regarding copyright ownership.
> + * The ASF licenses this file to You under the Apache License, Version 2.0
> + * (the "License"); you may not use this file except in compliance with
> + * the License.  You may obtain a copy of the License at
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +import java.io.IOException;
> +
> +import org.apache.lucene.document.Document;
> +import org.apache.lucene.index.AtomicReaderContext;
> +import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.index.RandomIndexWriter;
> +import org.apache.lucene.store.Directory;
> +import org.apache.lucene.util.LuceneTestCase;
> +
> +public class TestEarlyTermination extends LuceneTestCase {
> +
> +  Directory dir;
> +  RandomIndexWriter writer;
> +
> +  public void setUp() throws Exception {
> +    super.setUp();
> +    dir = newDirectory();
> +    writer = new RandomIndexWriter(random(), dir);
> +    final int numDocs = atLeast(100);
> +    for (int i = 0; i < numDocs; i++) {
> +      writer.addDocument(new Document());
> +      if (rarely()) {
> +        writer.commit();
> +      }
> +    }
> +  }
> +
> +  public void tearDown() throws Exception {
> +    super.tearDown();
> +    writer.close();
> +    dir.close();
> +  }
> +
> +  public void testEarlyTermination() throws IOException {
> +    final int iters = atLeast(5);
> +    final IndexReader reader = writer.getReader();
> +    final IndexSearcher searcher = new IndexSearcher(reader);
> +
> +    for (int i = 0; i < iters; ++i) {
> +      final Collector collector = new Collector() {
> +
> +        final boolean outOfOrder = random().nextBoolean();
> +        boolean collectionTerminated = true;
> +
> +        @Override
> +        public void setScorer(Scorer scorer) throws IOException {}
> +
> +        @Override
> +        public void collect(int doc) throws IOException {
> +          assertFalse(collectionTerminated);
> +          if (rarely()) {
> +            collectionTerminated = true;
> +            throw new CollectionTerminatedException();
> +          }
> +        }
> +
> +        @Override
> +        public void setNextReader(AtomicReaderContext context) throws IOException {
> +          collectionTerminated = false;
> +        }
> +
> +        @Override
> +        public boolean acceptsDocsOutOfOrder() {
> +          return outOfOrder;
> +        }
> +
> +      };
> +
> +      searcher.search(new MatchAllDocsQuery(), collector);
> +    }
> +    reader.close();
> +  }
> +
> +}
>
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


Re: svn commit: r1459400 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/search/ core/src/test/org/apache/lucene/search/

Posted by Adrien Grand <jp...@gmail.com>.
On Thu, Mar 21, 2013 at 6:18 PM, Robert Muir <rc...@gmail.com> wrote:
> Its not clear to me that this works when IndexSearcher has an executorService.

It works because the executed callables use the IndexSearcher.search
wich has been modified to deal with CollectionTerminatedException
(they just pass a singleton leaf context instead of all the reader's
leaves).

>  can the test use newSearcher or something?

Good point, I didn't know about this method. I'll improve the test.

--
Adrien

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org