You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/02/09 00:53:25 UTC
svn commit: r1658277 [7/38] - in /lucene/dev/branches/lucene6005: ./
dev-tools/ dev-tools/idea/solr/contrib/dataimporthandler/
dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/lucene/replicator/
dev-tools/maven/solr/ dev-tools/maven/solr/contrib/v...
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java Sun Feb 8 23:53:14 2015
@@ -27,816 +27,378 @@ import org.apache.lucene.util.PriorityQu
* A {@link Collector} that sorts by {@link SortField} using
* {@link FieldComparator}s.
* <p/>
- * See the {@link #create(org.apache.lucene.search.Sort, int, boolean, boolean, boolean, boolean)} method
+ * See the {@link #create(org.apache.lucene.search.Sort, int, boolean, boolean, boolean)} method
* for instantiating a TopFieldCollector.
- *
+ *
* @lucene.experimental
*/
public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
-
+
// TODO: one optimization we could do is to pre-fill
// the queue with sentinel value that guaranteed to
// always compare lower than a real hit; this would
// save having to check queueFull on each insert
- /*
- * Implements a TopFieldCollector over one SortField criteria, without
- * tracking document scores and maxScore.
- */
- private static class OneComparatorNonScoringCollector extends
- TopFieldCollector {
+ private static abstract class OneComparatorLeafCollector implements LeafCollector {
- FieldComparator<?> comparator;
+ final LeafFieldComparator comparator;
final int reverseMul;
- final FieldValueHitQueue<Entry> queue;
-
- public OneComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- this.queue = queue;
- comparator = queue.getComparators()[0];
- reverseMul = queue.getReverseMul()[0];
- }
-
- final void updateBottom(int doc) {
- // bottom.score is already set to Float.NaN in add().
- bottom.doc = docBase + doc;
- bottom = pq.updateTop();
- }
+ Scorer scorer;
- @Override
- public void collect(int doc) throws IOException {
- ++totalHits;
- if (queueFull) {
- if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
- // since docs are visited in doc Id order, if compare is 0, it means
- // this document is larger than anything else in the queue, and
- // therefore not competitive.
- return;
- }
-
- // This hit is competitive - replace bottom element in queue & adjustTop
- comparator.copy(bottom.slot, doc);
- updateBottom(doc);
- comparator.setBottom(bottom.slot);
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- comparator.copy(slot, doc);
- add(slot, doc, Float.NaN);
- if (queueFull) {
- comparator.setBottom(bottom.slot);
- }
- }
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- this.docBase = context.docBase;
- queue.setComparator(0, comparator.setNextReader(context));
- comparator = queue.firstComparator;
+ OneComparatorLeafCollector(LeafFieldComparator comparator, int reverseMul) {
+ this.comparator = comparator;
+ this.reverseMul = reverseMul;
}
-
+
@Override
public void setScorer(Scorer scorer) throws IOException {
+ this.scorer = scorer;
comparator.setScorer(scorer);
}
-
}
- /*
- * Implements a TopFieldCollector over one SortField criteria, without
- * tracking document scores and maxScore, and assumes out of orderness in doc
- * Ids collection.
- */
- private static class OutOfOrderOneComparatorNonScoringCollector extends
- OneComparatorNonScoringCollector {
-
- public OutOfOrderOneComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- final int cmp = reverseMul * comparator.compareBottom(doc);
- if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
- return;
- }
-
- // This hit is competitive - replace bottom element in queue & adjustTop
- comparator.copy(bottom.slot, doc);
- updateBottom(doc);
- comparator.setBottom(bottom.slot);
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- comparator.copy(slot, doc);
- add(slot, doc, Float.NaN);
- if (queueFull) {
- comparator.setBottom(bottom.slot);
- }
- }
- }
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
- }
-
- }
-
- /*
- * Implements a TopFieldCollector over one SortField criteria, while tracking
- * document scores but no maxScore.
- */
- private static class OneComparatorScoringNoMaxScoreCollector extends
- OneComparatorNonScoringCollector {
+ private static abstract class MultiComparatorLeafCollector implements LeafCollector {
+ final LeafFieldComparator[] comparators;
+ final int[] reverseMul;
+ final LeafFieldComparator firstComparator;
+ final int firstReverseMul;
Scorer scorer;
- public OneComparatorScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- }
-
- final void updateBottom(int doc, float score) {
- bottom.doc = docBase + doc;
- bottom.score = score;
- bottom = pq.updateTop();
+ MultiComparatorLeafCollector(LeafFieldComparator[] comparators, int[] reverseMul) {
+ this.comparators = comparators;
+ this.reverseMul = reverseMul;
+ firstComparator = comparators[0];
+ firstReverseMul = reverseMul[0];
}
- @Override
- public void collect(int doc) throws IOException {
- ++totalHits;
- if (queueFull) {
- if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
- // since docs are visited in doc Id order, if compare is 0, it means
- // this document is largest than anything else in the queue, and
- // therefore not competitive.
- return;
- }
-
- // Compute the score only if the hit is competitive.
- final float score = scorer.score();
-
- // This hit is competitive - replace bottom element in queue & adjustTop
- comparator.copy(bottom.slot, doc);
- updateBottom(doc, score);
- comparator.setBottom(bottom.slot);
- } else {
- // Compute the score only if the hit is competitive.
- final float score = scorer.score();
-
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- comparator.copy(slot, doc);
- add(slot, doc, score);
- if (queueFull) {
- comparator.setBottom(bottom.slot);
+ protected final int compareBottom(int doc) throws IOException {
+ int cmp = firstReverseMul * firstComparator.compareBottom(doc);
+ if (cmp != 0) {
+ return cmp;
+ }
+ for (int i = 1; i < comparators.length; ++i) {
+ cmp = reverseMul[i] * comparators[i].compareBottom(doc);
+ if (cmp != 0) {
+ return cmp;
}
}
+ return 0;
}
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- comparator.setScorer(scorer);
- }
-
- }
- /*
- * Implements a TopFieldCollector over one SortField criteria, while tracking
- * document scores but no maxScore, and assumes out of orderness in doc Ids
- * collection.
- */
- private static class OutOfOrderOneComparatorScoringNoMaxScoreCollector extends
- OneComparatorScoringNoMaxScoreCollector {
-
- public OutOfOrderOneComparatorScoringNoMaxScoreCollector(
- FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- final int cmp = reverseMul * comparator.compareBottom(doc);
- if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
- return;
- }
-
- // Compute the score only if the hit is competitive.
- final float score = scorer.score();
-
- // This hit is competitive - replace bottom element in queue & adjustTop
- comparator.copy(bottom.slot, doc);
- updateBottom(doc, score);
- comparator.setBottom(bottom.slot);
- } else {
- // Compute the score only if the hit is competitive.
- final float score = scorer.score();
-
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
+ protected final void copy(int slot, int doc) throws IOException {
+ for (LeafFieldComparator comparator : comparators) {
comparator.copy(slot, doc);
- add(slot, doc, score);
- if (queueFull) {
- comparator.setBottom(bottom.slot);
- }
}
}
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
- }
-
- }
- /*
- * Implements a TopFieldCollector over one SortField criteria, with tracking
- * document scores and maxScore.
- */
- private static class OneComparatorScoringMaxScoreCollector extends
- OneComparatorNonScoringCollector {
-
- Scorer scorer;
-
- public OneComparatorScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
- maxScore = Float.NEGATIVE_INFINITY;
- }
-
- final void updateBottom(int doc, float score) {
- bottom.doc = docBase + doc;
- bottom.score = score;
- bottom = pq.updateTop();
+ protected final void setBottom(int slot) {
+ for (LeafFieldComparator comparator : comparators) {
+ comparator.setBottom(slot);
+ }
}
- @Override
- public void collect(int doc) throws IOException {
- final float score = scorer.score();
- if (score > maxScore) {
- maxScore = score;
+ protected final int compareTop(int doc) throws IOException {
+ int cmp = firstReverseMul * firstComparator.compareTop(doc);
+ if (cmp != 0) {
+ return cmp;
}
- ++totalHits;
- if (queueFull) {
- if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
- // since docs are visited in doc Id order, if compare is 0, it means
- // this document is largest than anything else in the queue, and
- // therefore not competitive.
- return;
- }
-
- // This hit is competitive - replace bottom element in queue & adjustTop
- comparator.copy(bottom.slot, doc);
- updateBottom(doc, score);
- comparator.setBottom(bottom.slot);
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- comparator.copy(slot, doc);
- add(slot, doc, score);
- if (queueFull) {
- comparator.setBottom(bottom.slot);
+ for (int i = 1; i < comparators.length; ++i) {
+ cmp = reverseMul[i] * comparators[i].compareTop(doc);
+ if (cmp != 0) {
+ return cmp;
}
}
-
+ return 0;
}
-
+
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
- super.setScorer(scorer);
- }
- }
-
- /*
- * Implements a TopFieldCollector over one SortField criteria, with tracking
- * document scores and maxScore, and assumes out of orderness in doc Ids
- * collection.
- */
- private static class OutOfOrderOneComparatorScoringMaxScoreCollector extends
- OneComparatorScoringMaxScoreCollector {
-
- public OutOfOrderOneComparatorScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- final float score = scorer.score();
- if (score > maxScore) {
- maxScore = score;
+ for (LeafFieldComparator comparator : comparators) {
+ comparator.setScorer(scorer);
}
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- final int cmp = reverseMul * comparator.compareBottom(doc);
- if (cmp < 0 || (cmp == 0 && doc + docBase > bottom.doc)) {
- return;
- }
-
- // This hit is competitive - replace bottom element in queue & adjustTop
- comparator.copy(bottom.slot, doc);
- updateBottom(doc, score);
- comparator.setBottom(bottom.slot);
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- comparator.copy(slot, doc);
- add(slot, doc, score);
- if (queueFull) {
- comparator.setBottom(bottom.slot);
- }
- }
- }
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
}
-
}
/*
- * Implements a TopFieldCollector over multiple SortField criteria, without
+ * Implements a TopFieldCollector over one SortField criteria, without
* tracking document scores and maxScore.
*/
- private static class MultiComparatorNonScoringCollector extends TopFieldCollector {
-
- final FieldComparator<?>[] comparators;
- final int[] reverseMul;
+ private static class NonScoringCollector extends TopFieldCollector {
+
final FieldValueHitQueue<Entry> queue;
- public MultiComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
+
+ public NonScoringCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
super(queue, numHits, fillFields);
this.queue = queue;
- comparators = queue.getComparators();
- reverseMul = queue.getReverseMul();
- }
-
- final void updateBottom(int doc) {
- // bottom.score is already set to Float.NaN in add().
- bottom.doc = docBase + doc;
- bottom = pq.updateTop();
}
@Override
- public void collect(int doc) throws IOException {
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- for (int i = 0;; i++) {
- final int c = reverseMul[i] * comparators[i].compareBottom(doc);
- if (c < 0) {
- // Definitely not competitive.
- return;
- } else if (c > 0) {
- // Definitely competitive.
- break;
- } else if (i == comparators.length - 1) {
- // Here c=0. If we're at the last comparator, this doc is not
- // competitive, since docs are visited in doc Id order, which means
- // this doc cannot compete with any other document in the queue.
- return;
- }
- }
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+ docBase = context.docBase;
- // This hit is competitive - replace bottom element in queue & adjustTop
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(bottom.slot, doc);
- }
+ final LeafFieldComparator[] comparators = queue.getComparators(context);
+ final int[] reverseMul = queue.getReverseMul();
- updateBottom(doc);
+ if (comparators.length == 1) {
+ return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(slot, doc);
- }
- add(slot, doc, Float.NaN);
- if (queueFull) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
+ @Override
+ public void collect(int doc) throws IOException {
+ ++totalHits;
+ if (queueFull) {
+ if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
+ // since docs are visited in doc Id order, if compare is 0, it means
+ // this document is larger than anything else in the queue, and
+ // therefore not competitive.
+ return;
+ }
+
+ // This hit is competitive - replace bottom element in queue & adjustTop
+ comparator.copy(bottom.slot, doc);
+ updateBottom(doc);
+ comparator.setBottom(bottom.slot);
+ } else {
+ // Startup transient: queue hasn't gathered numHits yet
+ final int slot = totalHits - 1;
+ // Copy hit into queue
+ comparator.copy(slot, doc);
+ add(slot, doc, Float.NaN);
+ if (queueFull) {
+ comparator.setBottom(bottom.slot);
+ }
+ }
}
- }
- }
- }
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- docBase = context.docBase;
- for (int i = 0; i < comparators.length; i++) {
- queue.setComparator(i, comparators[i].setNextReader(context));
- }
- }
+ };
+ } else {
+ return new MultiComparatorLeafCollector(comparators, reverseMul) {
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- // set the scorer on all comparators
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setScorer(scorer);
- }
- }
- }
-
- /*
- * Implements a TopFieldCollector over multiple SortField criteria, without
- * tracking document scores and maxScore, and assumes out of orderness in doc
- * Ids collection.
- */
- private static class OutOfOrderMultiComparatorNonScoringCollector extends
- MultiComparatorNonScoringCollector {
-
- public OutOfOrderMultiComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- for (int i = 0;; i++) {
- final int c = reverseMul[i] * comparators[i].compareBottom(doc);
- if (c < 0) {
- // Definitely not competitive.
- return;
- } else if (c > 0) {
- // Definitely competitive.
- break;
- } else if (i == comparators.length - 1) {
- // This is the equals case.
- if (doc + docBase > bottom.doc) {
- // Definitely not competitive
- return;
+ @Override
+ public void collect(int doc) throws IOException {
+ ++totalHits;
+ if (queueFull) {
+ if ((compareBottom(doc)) <= 0) {
+ // since docs are visited in doc Id order, if compare is 0, it means
+ // this document is larger than anything else in the queue, and
+ // therefore not competitive.
+ return;
+ }
+
+ // This hit is competitive - replace bottom element in queue & adjustTop
+ copy(bottom.slot, doc);
+ updateBottom(doc);
+ setBottom(bottom.slot);
+ } else {
+ // Startup transient: queue hasn't gathered numHits yet
+ final int slot = totalHits - 1;
+ // Copy hit into queue
+ copy(slot, doc);
+ add(slot, doc, Float.NaN);
+ if (queueFull) {
+ setBottom(bottom.slot);
+ }
}
- break;
}
- }
- // This hit is competitive - replace bottom element in queue & adjustTop
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(bottom.slot, doc);
- }
-
- updateBottom(doc);
-
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(slot, doc);
- }
- add(slot, doc, Float.NaN);
- if (queueFull) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- }
+ };
}
}
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
- }
}
/*
- * Implements a TopFieldCollector over multiple SortField criteria, with
- * tracking document scores and maxScore.
+ * Implements a TopFieldCollector over one SortField criteria, while tracking
+ * document scores but no maxScore.
*/
- private static class MultiComparatorScoringMaxScoreCollector extends MultiComparatorNonScoringCollector {
-
- Scorer scorer;
-
- public MultiComparatorScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
+ private static class ScoringNoMaxScoreCollector extends TopFieldCollector {
+
+ final FieldValueHitQueue<Entry> queue;
+
+ public ScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
super(queue, numHits, fillFields);
- // Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
- maxScore = Float.NEGATIVE_INFINITY;
- }
-
- final void updateBottom(int doc, float score) {
- bottom.doc = docBase + doc;
- bottom.score = score;
- bottom = pq.updateTop();
+ this.queue = queue;
}
@Override
- public void collect(int doc) throws IOException {
- final float score = scorer.score();
- if (score > maxScore) {
- maxScore = score;
- }
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- for (int i = 0;; i++) {
- final int c = reverseMul[i] * comparators[i].compareBottom(doc);
- if (c < 0) {
- // Definitely not competitive.
- return;
- } else if (c > 0) {
- // Definitely competitive.
- break;
- } else if (i == comparators.length - 1) {
- // Here c=0. If we're at the last comparator, this doc is not
- // competitive, since docs are visited in doc Id order, which means
- // this doc cannot compete with any other document in the queue.
- return;
- }
- }
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+ docBase = context.docBase;
- // This hit is competitive - replace bottom element in queue & adjustTop
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(bottom.slot, doc);
- }
+ final LeafFieldComparator[] comparators = queue.getComparators(context);
+ final int[] reverseMul = queue.getReverseMul();
- updateBottom(doc, score);
+ if (comparators.length == 1) {
+ return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(slot, doc);
- }
- add(slot, doc, score);
- if (queueFull) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
+ @Override
+ public void collect(int doc) throws IOException {
+ ++totalHits;
+ if (queueFull) {
+ if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
+ // since docs are visited in doc Id order, if compare is 0, it means
+ // this document is largest than anything else in the queue, and
+ // therefore not competitive.
+ return;
+ }
+
+ // Compute the score only if the hit is competitive.
+ final float score = scorer.score();
+
+ // This hit is competitive - replace bottom element in queue & adjustTop
+ comparator.copy(bottom.slot, doc);
+ updateBottom(doc, score);
+ comparator.setBottom(bottom.slot);
+ } else {
+ // Compute the score only if the hit is competitive.
+ final float score = scorer.score();
+
+ // Startup transient: queue hasn't gathered numHits yet
+ final int slot = totalHits - 1;
+ // Copy hit into queue
+ comparator.copy(slot, doc);
+ add(slot, doc, score);
+ if (queueFull) {
+ comparator.setBottom(bottom.slot);
+ }
+ }
}
- }
- }
- }
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- super.setScorer(scorer);
- }
- }
+ };
+ } else {
+ return new MultiComparatorLeafCollector(comparators, reverseMul) {
- /*
- * Implements a TopFieldCollector over multiple SortField criteria, with
- * tracking document scores and maxScore, and assumes out of orderness in doc
- * Ids collection.
- */
- private final static class OutOfOrderMultiComparatorScoringMaxScoreCollector
- extends MultiComparatorScoringMaxScoreCollector {
-
- public OutOfOrderMultiComparatorScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- final float score = scorer.score();
- if (score > maxScore) {
- maxScore = score;
- }
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- for (int i = 0;; i++) {
- final int c = reverseMul[i] * comparators[i].compareBottom(doc);
- if (c < 0) {
- // Definitely not competitive.
- return;
- } else if (c > 0) {
- // Definitely competitive.
- break;
- } else if (i == comparators.length - 1) {
- // This is the equals case.
- if (doc + docBase > bottom.doc) {
- // Definitely not competitive
- return;
+ @Override
+ public void collect(int doc) throws IOException {
+ ++totalHits;
+ if (queueFull) {
+ if ((compareBottom(doc)) <= 0) {
+ // since docs are visited in doc Id order, if compare is 0, it means
+ // this document is largest than anything else in the queue, and
+ // therefore not competitive.
+ return;
+ }
+
+ // Compute the score only if the hit is competitive.
+ final float score = scorer.score();
+
+ // This hit is competitive - replace bottom element in queue & adjustTop
+ copy(bottom.slot, doc);
+ updateBottom(doc, score);
+ setBottom(bottom.slot);
+ } else {
+ // Compute the score only if the hit is competitive.
+ final float score = scorer.score();
+
+ // Startup transient: queue hasn't gathered numHits yet
+ final int slot = totalHits - 1;
+ // Copy hit into queue
+ copy(slot, doc);
+ add(slot, doc, score);
+ if (queueFull) {
+ setBottom(bottom.slot);
+ }
}
- break;
}
- }
-
- // This hit is competitive - replace bottom element in queue & adjustTop
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(bottom.slot, doc);
- }
-
- updateBottom(doc, score);
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(slot, doc);
- }
- add(slot, doc, score);
- if (queueFull) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- }
+ };
}
}
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
- }
}
/*
- * Implements a TopFieldCollector over multiple SortField criteria, with
- * tracking document scores and maxScore.
+ * Implements a TopFieldCollector over one SortField criteria, with tracking
+ * document scores and maxScore.
*/
- private static class MultiComparatorScoringNoMaxScoreCollector extends MultiComparatorNonScoringCollector {
-
- Scorer scorer;
-
- public MultiComparatorScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue,
- int numHits, boolean fillFields) {
+ private static class ScoringMaxScoreCollector extends TopFieldCollector {
+
+ final FieldValueHitQueue<Entry> queue;
+
+ public ScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
super(queue, numHits, fillFields);
- }
-
- final void updateBottom(int doc, float score) {
- bottom.doc = docBase + doc;
- bottom.score = score;
- bottom = pq.updateTop();
+ this.queue = queue;
+ maxScore = Float.MIN_NORMAL; // otherwise we would keep NaN
}
@Override
- public void collect(int doc) throws IOException {
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- for (int i = 0;; i++) {
- final int c = reverseMul[i] * comparators[i].compareBottom(doc);
- if (c < 0) {
- // Definitely not competitive.
- return;
- } else if (c > 0) {
- // Definitely competitive.
- break;
- } else if (i == comparators.length - 1) {
- // Here c=0. If we're at the last comparator, this doc is not
- // competitive, since docs are visited in doc Id order, which means
- // this doc cannot compete with any other document in the queue.
- return;
- }
- }
-
- // This hit is competitive - replace bottom element in queue & adjustTop
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(bottom.slot, doc);
- }
-
- // Compute score only if it is competitive.
- final float score = scorer.score();
- updateBottom(doc, score);
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+ docBase = context.docBase;
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- } else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(slot, doc);
- }
+ final LeafFieldComparator[] comparators = queue.getComparators(context);
+ final int[] reverseMul = queue.getReverseMul();
- // Compute score only if it is competitive.
- final float score = scorer.score();
- add(slot, doc, score);
- if (queueFull) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- }
- }
- }
+ if (comparators.length == 1) {
+ return new OneComparatorLeafCollector(comparators[0], reverseMul[0]) {
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- super.setScorer(scorer);
- }
- }
-
- /*
- * Implements a TopFieldCollector over multiple SortField criteria, with
- * tracking document scores and maxScore, and assumes out of orderness in doc
- * Ids collection.
- */
- private final static class OutOfOrderMultiComparatorScoringNoMaxScoreCollector
- extends MultiComparatorScoringNoMaxScoreCollector {
-
- public OutOfOrderMultiComparatorScoringNoMaxScoreCollector(
- FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
- super(queue, numHits, fillFields);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- ++totalHits;
- if (queueFull) {
- // Fastmatch: return if this hit is not competitive
- for (int i = 0;; i++) {
- final int c = reverseMul[i] * comparators[i].compareBottom(doc);
- if (c < 0) {
- // Definitely not competitive.
- return;
- } else if (c > 0) {
- // Definitely competitive.
- break;
- } else if (i == comparators.length - 1) {
- // This is the equals case.
- if (doc + docBase > bottom.doc) {
- // Definitely not competitive
- return;
+ @Override
+ public void collect(int doc) throws IOException {
+ final float score = scorer.score();
+ if (score > maxScore) {
+ maxScore = score;
+ }
+ ++totalHits;
+ if (queueFull) {
+ if (reverseMul * comparator.compareBottom(doc) <= 0) {
+ // since docs are visited in doc Id order, if compare is 0, it means
+ // this document is largest than anything else in the queue, and
+ // therefore not competitive.
+ return;
+ }
+
+ // This hit is competitive - replace bottom element in queue & adjustTop
+ comparator.copy(bottom.slot, doc);
+ updateBottom(doc, score);
+ comparator.setBottom(bottom.slot);
+ } else {
+ // Startup transient: queue hasn't gathered numHits yet
+ final int slot = totalHits - 1;
+ // Copy hit into queue
+ comparator.copy(slot, doc);
+ add(slot, doc, score);
+ if (queueFull) {
+ comparator.setBottom(bottom.slot);
+ }
}
- break;
}
- }
- // This hit is competitive - replace bottom element in queue & adjustTop
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(bottom.slot, doc);
- }
-
- // Compute score only if it is competitive.
- final float score = scorer.score();
- updateBottom(doc, score);
-
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
+ };
} else {
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = totalHits - 1;
- // Copy hit into queue
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(slot, doc);
- }
+ return new MultiComparatorLeafCollector(comparators, reverseMul) {
- // Compute score only if it is competitive.
- final float score = scorer.score();
- add(slot, doc, score);
- if (queueFull) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
+ @Override
+ public void collect(int doc) throws IOException {
+ final float score = scorer.score();
+ if (score > maxScore) {
+ maxScore = score;
+ }
+ ++totalHits;
+ if (queueFull) {
+ if (compareBottom(doc) <= 0) {
+ // since docs are visited in doc Id order, if compare is 0, it means
+ // this document is largest than anything else in the queue, and
+ // therefore not competitive.
+ return;
+ }
+
+ // This hit is competitive - replace bottom element in queue & adjustTop
+ copy(bottom.slot, doc);
+ updateBottom(doc, score);
+ setBottom(bottom.slot);
+ } else {
+ // Startup transient: queue hasn't gathered numHits yet
+ final int slot = totalHits - 1;
+ // Copy hit into queue
+ copy(slot, doc);
+ add(slot, doc, score);
+ if (queueFull) {
+ setBottom(bottom.slot);
+ }
+ }
}
- }
- }
- }
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- super.setScorer(scorer);
- }
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
+ };
+ }
}
}
@@ -846,30 +408,24 @@ public abstract class TopFieldCollector
*/
private final static class PagingFieldCollector extends TopFieldCollector {
- Scorer scorer;
int collectedHits;
- final FieldComparator<?>[] comparators;
- final int[] reverseMul;
final FieldValueHitQueue<Entry> queue;
final boolean trackDocScores;
final boolean trackMaxScore;
final FieldDoc after;
- int afterDoc;
-
- public PagingFieldCollector(
- FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields,
+
+ public PagingFieldCollector(FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields,
boolean trackDocScores, boolean trackMaxScore) {
super(queue, numHits, fillFields);
this.queue = queue;
this.trackDocScores = trackDocScores;
this.trackMaxScore = trackMaxScore;
this.after = after;
- comparators = queue.getComparators();
- reverseMul = queue.getReverseMul();
// Must set maxScore to NEG_INF, or otherwise Math.max always returns NaN.
maxScore = Float.NEGATIVE_INFINITY;
+ FieldComparator<?>[] comparators = queue.comparators;
// Tell all comparators their top value:
for(int i=0;i<comparators.length;i++) {
@SuppressWarnings("unchecked")
@@ -877,141 +433,81 @@ public abstract class TopFieldCollector
comparator.setTopValue(after.fields[i]);
}
}
-
- void updateBottom(int doc, float score) {
- bottom.doc = docBase + doc;
- bottom.score = score;
- bottom = pq.updateTop();
- }
- @SuppressWarnings({"unchecked", "rawtypes"})
@Override
- public void collect(int doc) throws IOException {
- //System.out.println(" collect doc=" + doc);
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+ docBase = context.docBase;
+ final int afterDoc = after.doc - docBase;
+ return new MultiComparatorLeafCollector(queue.getComparators(context), queue.getReverseMul()) {
- totalHits++;
+ @Override
+ public void collect(int doc) throws IOException {
+ //System.out.println(" collect doc=" + doc);
- float score = Float.NaN;
- if (trackMaxScore) {
- score = scorer.score();
- if (score > maxScore) {
- maxScore = score;
- }
- }
+ totalHits++;
- if (queueFull) {
- // Fastmatch: return if this hit is no better than
- // the worst hit currently in the queue:
- for (int i = 0;; i++) {
- final int c = reverseMul[i] * comparators[i].compareBottom(doc);
- if (c < 0) {
- // Definitely not competitive.
- return;
- } else if (c > 0) {
- // Definitely competitive.
- break;
- } else if (i == comparators.length - 1) {
- // This is the equals case.
- if (doc + docBase > bottom.doc) {
- // Definitely not competitive
- return;
+ float score = Float.NaN;
+ if (trackMaxScore) {
+ score = scorer.score();
+ if (score > maxScore) {
+ maxScore = score;
}
- break;
}
- }
- }
-
- // Check if this hit was already collected on a
- // previous page:
- boolean sameValues = true;
- for(int compIDX=0;compIDX<comparators.length;compIDX++) {
- final FieldComparator comp = comparators[compIDX];
-
- final int cmp = reverseMul[compIDX] * comp.compareTop(doc);
- if (cmp > 0) {
- // Already collected on a previous page
- //System.out.println(" skip: before");
- return;
- } else if (cmp < 0) {
- // Not yet collected
- sameValues = false;
- //System.out.println(" keep: after; reverseMul=" + reverseMul[compIDX]);
- break;
- }
- }
-
- // Tie-break by docID:
- if (sameValues && doc <= afterDoc) {
- // Already collected on a previous page
- //System.out.println(" skip: tie-break");
- return;
- }
- if (queueFull) {
- // This hit is competitive - replace bottom element in queue & adjustTop
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(bottom.slot, doc);
- }
+ if (queueFull) {
+ // Fastmatch: return if this hit is no better than
+ // the worst hit currently in the queue:
+ final int cmp = compareBottom(doc);
+ if (cmp <= 0) {
+ // not competitive since documents are visited in doc id order
+ return;
+ }
+ }
- // Compute score only if it is competitive.
- if (trackDocScores && !trackMaxScore) {
- score = scorer.score();
- }
- updateBottom(doc, score);
+ final int topCmp = compareTop(doc);
+ if (topCmp > 0 || (topCmp == 0 && doc <= afterDoc)) {
+ // Already collected on a previous page
+ return;
+ }
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
- }
- } else {
- collectedHits++;
+ if (queueFull) {
+ // This hit is competitive - replace bottom element in queue & adjustTop
+ copy(bottom.slot, doc);
+
+ // Compute score only if it is competitive.
+ if (trackDocScores && !trackMaxScore) {
+ score = scorer.score();
+ }
+ updateBottom(doc, score);
- // Startup transient: queue hasn't gathered numHits yet
- final int slot = collectedHits - 1;
- //System.out.println(" slot=" + slot);
- // Copy hit into queue
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].copy(slot, doc);
- }
+ setBottom(bottom.slot);
+ } else {
+ collectedHits++;
- // Compute score only if it is competitive.
- if (trackDocScores && !trackMaxScore) {
- score = scorer.score();
- }
- bottom = pq.add(new Entry(slot, docBase + doc, score));
- queueFull = collectedHits == numHits;
- if (queueFull) {
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setBottom(bottom.slot);
+ // Startup transient: queue hasn't gathered numHits yet
+ final int slot = collectedHits - 1;
+ //System.out.println(" slot=" + slot);
+ // Copy hit into queue
+ copy(slot, doc);
+
+ // Compute score only if it is competitive.
+ if (trackDocScores && !trackMaxScore) {
+ score = scorer.score();
+ }
+ bottom = pq.add(new Entry(slot, docBase + doc, score));
+ queueFull = collectedHits == numHits;
+ if (queueFull) {
+ setBottom(bottom.slot);
+ }
}
}
- }
- }
-
- @Override
- public void setScorer(Scorer scorer) {
- this.scorer = scorer;
- for (int i = 0; i < comparators.length; i++) {
- comparators[i].setScorer(scorer);
- }
- }
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
+ };
}
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- docBase = context.docBase;
- afterDoc = after.doc - docBase;
- for (int i = 0; i < comparators.length; i++) {
- queue.setComparator(i, comparators[i].setNextReader(context));
- }
- }
}
private static final ScoreDoc[] EMPTY_SCOREDOCS = new ScoreDoc[0];
-
+
private final boolean fillFields;
/*
@@ -1024,7 +520,7 @@ public abstract class TopFieldCollector
FieldValueHitQueue.Entry bottom = null;
boolean queueFull;
int docBase;
-
+
// Declaring the constructor private prevents extending this class by anyone
// else. Note that the class cannot be final since it's extended by the
// internal versions. If someone will define a constructor with any other
@@ -1043,7 +539,7 @@ public abstract class TopFieldCollector
* <p><b>NOTE</b>: The instances returned by this method
* pre-allocate a full array of length
* <code>numHits</code>.
- *
+ *
* @param sort
* the sort criteria (SortFields).
* @param numHits
@@ -1065,18 +561,14 @@ public abstract class TopFieldCollector
* true affects performance as it incurs the score computation on
* each result. Also, setting this true automatically sets
* <code>trackDocScores</code> to true as well.
- * @param docsScoredInOrder
- * specifies whether documents are scored in doc Id order or not by
- * the given {@link Scorer} in {@link #setScorer(Scorer)}.
* @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria.
* @throws IOException if there is a low-level I/O error
*/
public static TopFieldCollector create(Sort sort, int numHits,
- boolean fillFields, boolean trackDocScores, boolean trackMaxScore,
- boolean docsScoredInOrder)
+ boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
throws IOException {
- return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore, docsScoredInOrder);
+ return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore);
}
/**
@@ -1086,7 +578,7 @@ public abstract class TopFieldCollector
* <p><b>NOTE</b>: The instances returned by this method
* pre-allocate a full array of length
* <code>numHits</code>.
- *
+ *
* @param sort
* the sort criteria (SortFields).
* @param numHits
@@ -1110,22 +602,18 @@ public abstract class TopFieldCollector
* true affects performance as it incurs the score computation on
* each result. Also, setting this true automatically sets
* <code>trackDocScores</code> to true as well.
- * @param docsScoredInOrder
- * specifies whether documents are scored in doc Id order or not by
- * the given {@link Scorer} in {@link #setScorer(Scorer)}.
* @return a {@link TopFieldCollector} instance which will sort the results by
* the sort criteria.
* @throws IOException if there is a low-level I/O error
*/
public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
- boolean fillFields, boolean trackDocScores, boolean trackMaxScore,
- boolean docsScoredInOrder)
+ boolean fillFields, boolean trackDocScores, boolean trackMaxScore)
throws IOException {
if (sort.fields.length == 0) {
throw new IllegalArgumentException("Sort must contain at least one field");
}
-
+
if (numHits <= 0) {
throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
}
@@ -1133,43 +621,12 @@ public abstract class TopFieldCollector
FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
if (after == null) {
- if (queue.getComparators().length == 1) {
- if (docsScoredInOrder) {
- if (trackMaxScore) {
- return new OneComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
- } else if (trackDocScores) {
- return new OneComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
- } else {
- return new OneComparatorNonScoringCollector(queue, numHits, fillFields);
- }
- } else {
- if (trackMaxScore) {
- return new OutOfOrderOneComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
- } else if (trackDocScores) {
- return new OutOfOrderOneComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
- } else {
- return new OutOfOrderOneComparatorNonScoringCollector(queue, numHits, fillFields);
- }
- }
- }
-
- // multiple comparators.
- if (docsScoredInOrder) {
- if (trackMaxScore) {
- return new MultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
- } else if (trackDocScores) {
- return new MultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
- } else {
- return new MultiComparatorNonScoringCollector(queue, numHits, fillFields);
- }
+ if (trackMaxScore) {
+ return new ScoringMaxScoreCollector(queue, numHits, fillFields);
+ } else if (trackDocScores) {
+ return new ScoringNoMaxScoreCollector(queue, numHits, fillFields);
} else {
- if (trackMaxScore) {
- return new OutOfOrderMultiComparatorScoringMaxScoreCollector(queue, numHits, fillFields);
- } else if (trackDocScores) {
- return new OutOfOrderMultiComparatorScoringNoMaxScoreCollector(queue, numHits, fillFields);
- } else {
- return new OutOfOrderMultiComparatorNonScoringCollector(queue, numHits, fillFields);
- }
+ return new NonScoringCollector(queue, numHits, fillFields);
}
} else {
if (after.fields == null) {
@@ -1183,12 +640,24 @@ public abstract class TopFieldCollector
return new PagingFieldCollector(queue, after, numHits, fillFields, trackDocScores, trackMaxScore);
}
}
-
+
final void add(int slot, int doc, float score) {
bottom = pq.add(new Entry(slot, docBase + doc, score));
queueFull = totalHits == numHits;
}
+ final void updateBottom(int doc) {
+ // bottom.score is already set to Float.NaN in add().
+ bottom.doc = docBase + doc;
+ bottom = pq.updateTop();
+ }
+
+ final void updateBottom(int doc, float score) {
+ bottom.doc = docBase + doc;
+ bottom.score = score;
+ bottom = pq.updateTop();
+ }
+
/*
* Only the following callback methods need to be overridden since
* topDocs(int, int) calls them to return the results.
@@ -1209,7 +678,7 @@ public abstract class TopFieldCollector
}
}
}
-
+
@Override
protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
if (results == null) {
@@ -1218,12 +687,8 @@ public abstract class TopFieldCollector
maxScore = Float.NaN;
}
- // If this is a maxScoring tracking collector and there were no results,
+ // If this is a maxScoring tracking collector and there were no results,
return new TopFieldDocs(totalHits, results, ((FieldValueHitQueue<Entry>) pq).getFields(), maxScore);
}
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return false;
- }
+
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java Sun Feb 8 23:53:14 2015
@@ -36,242 +36,151 @@ import org.apache.lucene.index.LeafReade
*/
public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
- // Assumes docs are scored in order.
- private static class InOrderTopScoreDocCollector extends TopScoreDocCollector {
- private InOrderTopScoreDocCollector(int numHits) {
- super(numHits);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- float score = scorer.score();
+ abstract static class ScorerLeafCollector implements LeafCollector {
+
+ Scorer scorer;
- // This collector cannot handle these scores:
- assert score != Float.NEGATIVE_INFINITY;
- assert !Float.isNaN(score);
-
- totalHits++;
- if (score <= pqTop.score) {
- // Since docs are returned in-order (i.e., increasing doc Id), a document
- // with equal score to pqTop.score cannot compete since HitQueue favors
- // documents with lower doc Ids. Therefore reject those docs too.
- return;
- }
- pqTop.doc = doc + docBase;
- pqTop.score = score;
- pqTop = pq.updateTop();
- }
-
@Override
- public boolean acceptsDocsOutOfOrder() {
- return false;
+ public void setScorer(Scorer scorer) throws IOException {
+ this.scorer = scorer;
}
+
}
-
- // Assumes docs are scored in order.
- private static class InOrderPagingScoreDocCollector extends TopScoreDocCollector {
- private final ScoreDoc after;
- // this is always after.doc - docBase, to save an add when score == after.score
- private int afterDoc;
- private int collectedHits;
- private InOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
+ private static class SimpleTopScoreDocCollector extends TopScoreDocCollector {
+
+ SimpleTopScoreDocCollector(int numHits) {
super(numHits);
- this.after = after;
}
-
+
@Override
- public void collect(int doc) throws IOException {
- float score = scorer.score();
+ public LeafCollector getLeafCollector(LeafReaderContext context)
+ throws IOException {
+ final int docBase = context.docBase;
+ return new ScorerLeafCollector() {
- // This collector cannot handle these scores:
- assert score != Float.NEGATIVE_INFINITY;
- assert !Float.isNaN(score);
-
- totalHits++;
-
- if (score > after.score || (score == after.score && doc <= afterDoc)) {
- // hit was collected on a previous page
- return;
- }
-
- if (score <= pqTop.score) {
- // Since docs are returned in-order (i.e., increasing doc Id), a document
- // with equal score to pqTop.score cannot compete since HitQueue favors
- // documents with lower doc Ids. Therefore reject those docs too.
- return;
- }
- collectedHits++;
- pqTop.doc = doc + docBase;
- pqTop.score = score;
- pqTop = pq.updateTop();
- }
+ @Override
+ public void collect(int doc) throws IOException {
+ float score = scorer.score();
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return false;
- }
+ // This collector cannot handle these scores:
+ assert score != Float.NEGATIVE_INFINITY;
+ assert !Float.isNaN(score);
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- super.doSetNextReader(context);
- afterDoc = after.doc - context.docBase;
- }
+ totalHits++;
+ if (score <= pqTop.score) {
+ // Since docs are returned in-order (i.e., increasing doc Id), a document
+ // with equal score to pqTop.score cannot compete since HitQueue favors
+ // documents with lower doc Ids. Therefore reject those docs too.
+ return;
+ }
+ pqTop.doc = doc + docBase;
+ pqTop.score = score;
+ pqTop = pq.updateTop();
+ }
- @Override
- protected int topDocsSize() {
- return collectedHits < pq.size() ? collectedHits : pq.size();
+ };
}
-
- @Override
- protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
- return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results);
- }
- }
- // Assumes docs are scored out of order.
- private static class OutOfOrderTopScoreDocCollector extends TopScoreDocCollector {
- private OutOfOrderTopScoreDocCollector(int numHits) {
- super(numHits);
- }
-
- @Override
- public void collect(int doc) throws IOException {
- float score = scorer.score();
+ }
- // This collector cannot handle NaN
- assert !Float.isNaN(score);
+ private static class PagingTopScoreDocCollector extends TopScoreDocCollector {
- totalHits++;
- if (score < pqTop.score) {
- // Doesn't compete w/ bottom entry in queue
- return;
- }
- doc += docBase;
- if (score == pqTop.score && doc > pqTop.doc) {
- // Break tie in score by doc ID:
- return;
- }
- pqTop.doc = doc;
- pqTop.score = score;
- pqTop = pq.updateTop();
- }
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
- }
- }
-
- // Assumes docs are scored out of order.
- private static class OutOfOrderPagingScoreDocCollector extends TopScoreDocCollector {
private final ScoreDoc after;
- // this is always after.doc - docBase, to save an add when score == after.score
- private int afterDoc;
private int collectedHits;
- private OutOfOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
+ PagingTopScoreDocCollector(int numHits, ScoreDoc after) {
super(numHits);
this.after = after;
+ this.collectedHits = 0;
}
-
- @Override
- public void collect(int doc) throws IOException {
- float score = scorer.score();
-
- // This collector cannot handle NaN
- assert !Float.isNaN(score);
- totalHits++;
- if (score > after.score || (score == after.score && doc <= afterDoc)) {
- // hit was collected on a previous page
- return;
- }
- if (score < pqTop.score) {
- // Doesn't compete w/ bottom entry in queue
- return;
- }
- doc += docBase;
- if (score == pqTop.score && doc > pqTop.doc) {
- // Break tie in score by doc ID:
- return;
- }
- collectedHits++;
- pqTop.doc = doc;
- pqTop.score = score;
- pqTop = pq.updateTop();
- }
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
- }
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- super.doSetNextReader(context);
- afterDoc = after.doc - context.docBase;
- }
-
@Override
protected int topDocsSize() {
return collectedHits < pq.size() ? collectedHits : pq.size();
}
-
+
@Override
protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results);
}
+
+ @Override
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+ final int docBase = context.docBase;
+ final int afterDoc = after.doc - context.docBase;
+ return new ScorerLeafCollector() {
+ @Override
+ public void collect(int doc) throws IOException {
+ float score = scorer.score();
+
+ // This collector cannot handle these scores:
+ assert score != Float.NEGATIVE_INFINITY;
+ assert !Float.isNaN(score);
+
+ totalHits++;
+
+ if (score > after.score || (score == after.score && doc <= afterDoc)) {
+ // hit was collected on a previous page
+ return;
+ }
+
+ if (score <= pqTop.score) {
+ // Since docs are returned in-order (i.e., increasing doc Id), a document
+ // with equal score to pqTop.score cannot compete since HitQueue favors
+ // documents with lower doc Ids. Therefore reject those docs too.
+ return;
+ }
+ collectedHits++;
+ pqTop.doc = doc + docBase;
+ pqTop.score = score;
+ pqTop = pq.updateTop();
+ }
+ };
+ }
}
/**
* Creates a new {@link TopScoreDocCollector} given the number of hits to
* collect and whether documents are scored in order by the input
- * {@link Scorer} to {@link #setScorer(Scorer)}.
+ * {@link Scorer} to {@link LeafCollector#setScorer(Scorer)}.
*
* <p><b>NOTE</b>: The instances returned by this method
* pre-allocate a full array of length
* <code>numHits</code>, and fill the array with sentinel
* objects.
*/
- public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
- return create(numHits, null, docsScoredInOrder);
+ public static TopScoreDocCollector create(int numHits) {
+ return create(numHits, null);
}
-
+
/**
* Creates a new {@link TopScoreDocCollector} given the number of hits to
* collect, the bottom of the previous page, and whether documents are scored in order by the input
- * {@link Scorer} to {@link #setScorer(Scorer)}.
+ * {@link Scorer} to {@link LeafCollector#setScorer(Scorer)}.
*
* <p><b>NOTE</b>: The instances returned by this method
* pre-allocate a full array of length
* <code>numHits</code>, and fill the array with sentinel
* objects.
*/
- public static TopScoreDocCollector create(int numHits, ScoreDoc after, boolean docsScoredInOrder) {
-
+ public static TopScoreDocCollector create(int numHits, ScoreDoc after) {
+
if (numHits <= 0) {
throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
}
-
- if (docsScoredInOrder) {
- return after == null
- ? new InOrderTopScoreDocCollector(numHits)
- : new InOrderPagingScoreDocCollector(after, numHits);
+
+ if (after == null) {
+ return new SimpleTopScoreDocCollector(numHits);
} else {
- return after == null
- ? new OutOfOrderTopScoreDocCollector(numHits)
- : new OutOfOrderPagingScoreDocCollector(after, numHits);
+ return new PagingTopScoreDocCollector(numHits, after);
}
-
}
-
+
ScoreDoc pqTop;
- int docBase = 0;
- Scorer scorer;
-
+
// prevents instantiation
- private TopScoreDocCollector(int numHits) {
+ TopScoreDocCollector(int numHits) {
super(new HitQueue(numHits, true));
// HitQueue implements getSentinelObject to return a ScoreDoc, so we know
// that at this point top() is already initialized.
@@ -283,7 +192,7 @@ public abstract class TopScoreDocCollect
if (results == null) {
return EMPTY_TOPDOCS;
}
-
+
// We need to compute maxScore in order to set it in TopDocs. If start == 0,
// it means the largest element is already in results, use its score as
// maxScore. Otherwise pop everything else, until the largest element is
@@ -295,17 +204,7 @@ public abstract class TopScoreDocCollect
for (int i = pq.size(); i > 1; i--) { pq.pop(); }
maxScore = pq.pop().score;
}
-
+
return new TopDocs(totalHits, results, maxScore);
}
-
- @Override
- protected void doSetNextReader(LeafReaderContext context) throws IOException {
- docBase = context.docBase;
- }
-
- @Override
- public void setScorer(Scorer scorer) throws IOException {
- this.scorer = scorer;
- }
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java Sun Feb 8 23:53:14 2015
@@ -34,9 +34,4 @@ public class TotalHitCountCollector exte
public void collect(int doc) {
totalHits++;
}
-
- @Override
- public boolean acceptsDocsOutOfOrder() {
- return true;
- }
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Weight.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/Weight.java Sun Feb 8 23:53:14 2015
@@ -79,11 +79,6 @@ public abstract class Weight {
* Returns a {@link Scorer} which scores documents in/out-of order according
* to <code>scoreDocsInOrder</code>.
* <p>
- * <b>NOTE:</b> even if <code>scoreDocsInOrder</code> is false, it is
- * recommended to check whether the returned <code>Scorer</code> indeed scores
- * documents out of order (i.e., call {@link #scoresDocsOutOfOrder()}), as
- * some <code>Scorer</code> implementations will always return documents
- * in-order.<br>
* <b>NOTE:</b> null can be returned if no documents will be scored by this
* query.
*
@@ -108,15 +103,6 @@ public abstract class Weight {
*
* @param context
* the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
- * @param scoreDocsInOrder
- * specifies whether in-order scoring of documents is required. Note
- * that if set to false (i.e., out-of-order scoring is required),
- * this method can return whatever scoring mode it supports, as every
- * in-order scorer is also an out-of-order one. However, an
- * out-of-order scorer may not support {@link Scorer#nextDoc()}
- * and/or {@link Scorer#advance(int)}, therefore it is recommended to
- * request an in-order scorer if use of these
- * methods is required.
* @param acceptDocs
* Bits that represent the allowable docs to match (typically deleted docs
* but possibly filtering other documents)
@@ -125,7 +111,7 @@ public abstract class Weight {
* passes them to a collector.
* @throws IOException if there is a low-level I/O error
*/
- public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
+ public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
Scorer scorer = scorer(context, acceptDocs);
if (scorer == null) {
@@ -150,7 +136,7 @@ public abstract class Weight {
}
@Override
- public boolean score(LeafCollector collector, int max) throws IOException {
+ public int score(LeafCollector collector, int min, int max) throws IOException {
// TODO: this may be sort of weird, when we are
// embedded in a BooleanScorer, because we are
// called for every chunk of 2048 documents. But,
@@ -158,13 +144,13 @@ public abstract class Weight {
// Collector doing something "interesting" in
// setScorer will be forced to use BS2 anyways:
collector.setScorer(scorer);
- if (max == DocIdSetIterator.NO_MORE_DOCS) {
+ if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
scoreAll(collector, scorer);
- return false;
+ return DocIdSetIterator.NO_MORE_DOCS;
} else {
int doc = scorer.docID();
- if (doc < 0) {
- doc = scorer.nextDoc();
+ if (doc < min) {
+ doc = scorer.advance(min);
}
return scoreRange(collector, scorer, doc, max);
}
@@ -174,12 +160,12 @@ public abstract class Weight {
* separate this from {@link #scoreAll} to help out
* hotspot.
* See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
- static boolean scoreRange(LeafCollector collector, Scorer scorer, int currentDoc, int end) throws IOException {
+ static int scoreRange(LeafCollector collector, Scorer scorer, int currentDoc, int end) throws IOException {
while (currentDoc < end) {
collector.collect(currentDoc);
currentDoc = scorer.nextDoc();
}
- return currentDoc != DocIdSetIterator.NO_MORE_DOCS;
+ return currentDoc;
}
/** Specialized method to bulk-score all hits; we
@@ -187,25 +173,9 @@ public abstract class Weight {
* hotspot.
* See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
static void scoreAll(LeafCollector collector, Scorer scorer) throws IOException {
- int doc;
- while ((doc = scorer.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+ for (int doc = scorer.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = scorer.nextDoc()) {
collector.collect(doc);
}
}
}
-
- /**
- * Returns true iff this implementation scores docs only out of order. This
- * method is used in conjunction with {@link Collector}'s
- * {@link LeafCollector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
- * {@link #bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, Bits)} to
- * create a matching {@link Scorer} instance for a given {@link Collector}, or
- * vice versa.
- * <p>
- * <b>NOTE:</b> the default implementation returns <code>false</code>, i.e.
- * the <code>Scorer</code> scores documents in-order.
- */
- public boolean scoresDocsOutOfOrder() {
- return false;
- }
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/package.html?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/search/package.html Sun Feb 8 23:53:14 2015
@@ -443,8 +443,8 @@ on the built-in available scoring models
given the Query.
</li>
<li>
- {@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, org.apache.lucene.util.Bits)
- scorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} —
+ {@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, org.apache.lucene.util.Bits)
+ scorer(LeafReaderContext context, Bits acceptDocs)} —
Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a>
below for help defining a BulkScorer. This is an optional method, and most queries do not implement it.
</li>
@@ -508,7 +508,7 @@ on the built-in available scoring models
abstract method:
<ol>
<li>
- {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int) score(LeafCollector,int)} —
+ {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int,int) score(LeafCollector,int,int)} —
Score all documents up to but not including the specified max document.
</li>
</ol>
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java Sun Feb 8 23:53:14 2015
@@ -170,9 +170,7 @@ abstract class ByteBufferIndexInput exte
this.curBufIndex = bi;
this.curBuf = b;
}
- } catch (ArrayIndexOutOfBoundsException aioobe) {
- throw new EOFException("seek past EOF: " + this);
- } catch (IllegalArgumentException iae) {
+ } catch (ArrayIndexOutOfBoundsException | IllegalArgumentException e) {
throw new EOFException("seek past EOF: " + this);
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
@@ -198,9 +196,7 @@ abstract class ByteBufferIndexInput exte
b.position((int) (pos & chunkSizeMask));
this.curBufIndex = bi;
this.curBuf = b;
- } catch (ArrayIndexOutOfBoundsException aioobe) {
- throw new EOFException("seek past EOF: " + this);
- } catch (IllegalArgumentException iae) {
+ } catch (ArrayIndexOutOfBoundsException | IllegalArgumentException aioobe) {
throw new EOFException("seek past EOF: " + this);
} catch (NullPointerException npe) {
throw new AlreadyClosedException("Already closed: " + this);
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java Sun Feb 8 23:53:14 2015
@@ -23,7 +23,7 @@ import java.util.Collection;
/** Directory implementation that delegates calls to another directory.
* This class can be used to add limitations on top of an existing
* {@link Directory} implementation such as
- * {@link RateLimitedDirectoryWrapper rate limiting} or to add additional
+ * {@link NRTCachingDirectory} or to add additional
* sanity checks for tests. However, if you plan to write your own
* {@link Directory} implementation, you should consider extending directly
* {@link Directory} or {@link BaseDirectory} rather than try to reuse
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java Sun Feb 8 23:53:14 2015
@@ -139,7 +139,7 @@ public class LockStressTest {
// try to create a new instance
try {
return Class.forName(lockFactoryClassName).asSubclass(FSLockFactory.class).newInstance();
- } catch (IllegalAccessException | InstantiationException | ClassCastException | ClassNotFoundException e) {
+ } catch (ReflectiveOperationException | ClassCastException e) {
// fall-through
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java Sun Feb 8 23:53:14 2015
@@ -24,7 +24,8 @@ import java.io.IOException;
*
* @lucene.internal
*/
-final class RateLimitedIndexOutput extends IndexOutput {
+
+public final class RateLimitedIndexOutput extends IndexOutput {
private final IndexOutput delegate;
private final RateLimiter rateLimiter;
@@ -36,7 +37,7 @@ final class RateLimitedIndexOutput exten
* which does volatile read. */
private long currentMinPauseCheckBytes;
- RateLimitedIndexOutput(final RateLimiter rateLimiter, final IndexOutput delegate) {
+ public RateLimitedIndexOutput(final RateLimiter rateLimiter, final IndexOutput delegate) {
super("RateLimitedIndexOutput(" + delegate + ")");
this.delegate = delegate;
this.rateLimiter = rateLimiter;
@@ -72,7 +73,7 @@ final class RateLimitedIndexOutput exten
delegate.writeBytes(b, offset, length);
}
- private void checkRate() {
+ private void checkRate() throws IOException {
if (bytesSinceLastPause > currentMinPauseCheckBytes) {
rateLimiter.pause(bytesSinceLastPause);
bytesSinceLastPause = 0;
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java Sun Feb 8 23:53:14 2015
@@ -17,6 +17,8 @@ package org.apache.lucene.store;
* limitations under the License.
*/
+import java.io.IOException;
+
import org.apache.lucene.util.ThreadInterruptedException;
/** Abstract base class to rate limit IO. Typically implementations are
@@ -27,14 +29,14 @@ import org.apache.lucene.util.ThreadInte
public abstract class RateLimiter {
/**
- * Sets an updated mb per second rate limit.
+ * Sets an updated MB per second rate limit.
*/
- public abstract void setMbPerSec(double mbPerSec);
+ public abstract void setMBPerSec(double mbPerSec);
/**
- * The current mb per second rate limit.
+ * The current MB per second rate limit.
*/
- public abstract double getMbPerSec();
+ public abstract double getMBPerSec();
/** Pauses, if necessary, to keep the instantaneous IO
* rate at or below the target.
@@ -43,7 +45,7 @@ public abstract class RateLimiter {
* </p>
* @return the pause time in nano seconds
* */
- public abstract long pause(long bytes);
+ public abstract long pause(long bytes) throws IOException;
/** How many bytes caller should add up itself before invoking {@link #pause}. */
public abstract long getMinPauseCheckBytes();
@@ -65,7 +67,7 @@ public abstract class RateLimiter {
/** mbPerSec is the MB/sec max IO rate */
public SimpleRateLimiter(double mbPerSec) {
- setMbPerSec(mbPerSec);
+ setMBPerSec(mbPerSec);
lastNS = System.nanoTime();
}
@@ -73,7 +75,7 @@ public abstract class RateLimiter {
* Sets an updated mb per second rate limit.
*/
@Override
- public void setMbPerSec(double mbPerSec) {
+ public void setMBPerSec(double mbPerSec) {
this.mbPerSec = mbPerSec;
minPauseCheckBytes = (long) ((MIN_PAUSE_CHECK_MSEC / 1000.0) * mbPerSec * 1024 * 1024);
}
@@ -87,7 +89,7 @@ public abstract class RateLimiter {
* The current mb per second rate limit.
*/
@Override
- public double getMbPerSec() {
+ public double getMBPerSec() {
return this.mbPerSec;
}
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/IOUtils.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/IOUtils.java Sun Feb 8 23:53:14 2015
@@ -407,7 +407,8 @@ public final class IOUtils {
}
if (isDir) {
- assert (Constants.LINUX || Constants.MAC_OS_X) == false :
+ // TODO: LUCENE-6169 - Fix this assert once Java 9 problems are solved!
+ assert (Constants.LINUX || Constants.MAC_OS_X) == false || Constants.JRE_IS_MINIMUM_JAVA9 :
"On Linux and MacOSX fsyncing a directory should not throw IOException, "+
"we just don't want to rely on that in production (undocumented). Got: " + exc;
// Ignore exception if it is a directory
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java Sun Feb 8 23:53:14 2015
@@ -19,7 +19,10 @@ package org.apache.lucene.util;
import java.io.IOException;
import java.io.PrintStream;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
import java.util.Date;
+import java.util.Locale;
import java.util.concurrent.atomic.AtomicInteger;
/**
@@ -32,6 +35,8 @@ public class PrintStreamInfoStream exten
// Used for printing messages
private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
protected final int messageID;
+
+ private static final DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.ROOT);
protected final PrintStream stream;
@@ -46,7 +51,7 @@ public class PrintStreamInfoStream exten
@Override
public void message(String component, String message) {
- stream.println(component + " " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
+ stream.println(component + " " + messageID + " [" + dateFormat.format(new Date()) + "; " + Thread.currentThread().getName() + "]: " + message);
}
@Override
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Sun Feb 8 23:53:14 2015
@@ -253,7 +253,7 @@ public abstract class StringHelper {
x0 = Long.parseLong(prop, 16);
x1 = x0;
} else {
- // "Rough randomess" from 3 different sources:
+ // Randomess from 3 different sources:
x0 = System.nanoTime();
x1 = StringHelper.class.hashCode() << 32;
StringBuilder sb = new StringBuilder();
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Version.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/Version.java Sun Feb 8 23:53:14 2015
@@ -39,6 +39,13 @@ public final class Version {
@Deprecated
public static final Version LUCENE_5_0_0 = new Version(5, 0, 0);
+ /**
+ * Match settings and bugs in Lucene's 5.1.0 release.
+ * @deprecated Use latest
+ */
+ @Deprecated
+ public static final Version LUCENE_5_1_0 = new Version(5, 1, 0);
+
/** Match settings and bugs in Lucene's 6.0 release.
* <p>
* Use this to get the latest & greatest settings, bug
Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java Sun Feb 8 23:53:14 2015
@@ -19,16 +19,21 @@ package org.apache.lucene.util.automaton
//import java.io.IOException;
//import java.io.PrintWriter;
+
import java.util.Arrays;
import java.util.BitSet;
import java.util.HashSet;
import java.util.Set;
+import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.InPlaceMergeSorter;
+import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.Sorter;
+
+
// TODO
// - could use packed int arrays instead
// - could encode dest w/ delta from to?
@@ -47,7 +52,8 @@ import org.apache.lucene.util.Sorter;
*
* @lucene.experimental */
-public class Automaton {
+public class Automaton implements Accountable {
+
/** Where we next write to the int[] states; this increments by 2 for
* each added state because we pack a pointer to the transitions
* array and a count of how many transitions leave the state. */
@@ -879,4 +885,14 @@ public class Automaton {
}
}
}
+
+ @Override
+ public long ramBytesUsed() {
+ // TODO: BitSet RAM usage (isAccept.size()/8) isn't fully accurate...
+ return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.sizeOf(states) + RamUsageEstimator.sizeOf(transitions) +
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + (isAccept.size() / 8) + RamUsageEstimator.NUM_BYTES_OBJECT_REF +
+ 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF +
+ 3 * RamUsageEstimator.NUM_BYTES_INT +
+ RamUsageEstimator.NUM_BYTES_BOOLEAN;
+ }
}