You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by GitBox <gi...@apache.org> on 2019/09/11 10:42:52 UTC

[GitHub] [lucene-solr] mikemccand commented on a change in pull request #854: Shared PQ Based Early Termination for Concurrent Search

mikemccand commented on a change in pull request #854: Shared PQ Based Early Termination for Concurrent Search
URL: https://github.com/apache/lucene-solr/pull/854#discussion_r322909770
 
 

 ##########
 File path: lucene/core/src/java/org/apache/lucene/search/EarlyTerminatingFieldCollectorManager.java
 ##########
 @@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Collection;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.apache.lucene.search.TopFieldCollector.EMPTY_SCOREDOCS;
+
+/**
+ * CollectorManager which allows early termination across multiple slices
+ * when the index sort key and the query sort key are the same
+ */
+public class EarlyTerminatingFieldCollectorManager implements CollectorManager<TopFieldCollector, TopFieldDocs> {
+  private final Sort sort;
+  private final int numHits;
+  private final int totalHitsThreshold;
+  private final AtomicInteger globalTotalHits;
+  private final ReentrantLock lock;
+  private int numCollectors;
+
+  private final ConcurrentLinkedQueue<TopFieldCollector.EarlyTerminatingFieldCollector> mergeableCollectors;
+  private FieldValueHitQueue globalHitQueue;
+  private FieldValueHitQueue.Entry bottom;
+  // We do not make this Atomic since it will be sought under a lock
+  private int queueSlotCounter;
+  private final AtomicBoolean mergeStarted;
+  public final AtomicBoolean mergeCompleted;
+
+  public EarlyTerminatingFieldCollectorManager(Sort sort, int numHits, int totalHitsThreshold) {
+    this.sort = sort;
+    this.numHits = numHits;
+    this.totalHitsThreshold = totalHitsThreshold;
+    this.globalTotalHits = new AtomicInteger();
+    this.lock = new ReentrantLock();
+    this.mergeStarted = new AtomicBoolean();
+    this.mergeCompleted = new AtomicBoolean();
+    this.mergeableCollectors = new ConcurrentLinkedQueue();
+    this.globalHitQueue = null;
+  }
+
+  @Override
+  public TopFieldCollector.EarlyTerminatingFieldCollector newCollector() {
+    ++numCollectors;
+
+    return new TopFieldCollector.EarlyTerminatingFieldCollector(sort, FieldValueHitQueue.create(sort.fields, numHits), numHits,
+        totalHitsThreshold, this, globalTotalHits);
+  }
+
+  @Override
+  public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) {
+
+    if (globalHitQueue == null) {
+      final TopFieldDocs[] topDocs = new TopFieldDocs[collectors.size()];
+      int i = 0;
+      for (TopFieldCollector collector : collectors) {
+        topDocs[i++] = collector.topDocs();
+      }
+      return TopDocs.merge(sort, 0, numHits, topDocs);
+    }
+
+    ScoreDoc[] results = populateResults(globalHitQueue.size());
+
+    return newTopDocs(results);
+  }
+
+  public int compareAndUpdateBottom(int docBase, int doc, Object value) {
+
+    try {
+      lock.lock();
+
+      // If not enough hits are accumulated, add this hit to the global hit queue
+      if (globalHitQueue.size() < numHits) {
+        FieldValueHitQueue.Entry newEntry = new FieldValueHitQueue.Entry(queueSlotCounter++, (doc + docBase), value);
+        bottom = (FieldValueHitQueue.Entry) globalHitQueue.add(newEntry);
+        return 1;
+      }
+
+      FieldComparator[] comparators = globalHitQueue.getComparators();
+      int[] reverseMul = globalHitQueue.getReverseMul();
+      Object bottomValues = bottom.values;
+      Object[] valuesArray;
+      Object[] bottomValuesArray;
+
+      if (comparators.length > 1) {
+        assert value instanceof Object[];
+        valuesArray = (Object[]) value;
+
+        assert bottomValues instanceof Object[];
+        bottomValuesArray = (Object[]) bottomValues;
+      } else {
+        valuesArray = new Object[1];
+        valuesArray[0] = value;
+
+        bottomValuesArray = new Object[1];
 
 Review comment:
   Hmm, spooky we must allocate two `Object[]` per collected hit in this case ...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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