From e1037bac4ac6f6b8e7f30047c7d9fd3abbcdc15a Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 27 Feb 2023 16:29:55 +0100 Subject: [PATCH] Revert "Remove shortcutTotalHitCount optimization (#89047)" This reverts commit 283f8acd9275d197453e7d9309e62f31abf51f51. --- .../search/query/QueryPhase.java | 13 +- .../search/query/TopDocsCollectorContext.java | 283 +++++++++++++----- .../search/query/QueryPhaseTests.java | 4 +- .../query/TopDocsCollectorContextTests.java | 83 +++++ 4 files changed, 304 insertions(+), 79 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/search/query/TopDocsCollectorContextTests.java diff --git a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java index 2e300be8d40bf..5fbaed04e31bb 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -129,15 +129,21 @@ static boolean executeInternal(SearchContext searchContext) throws QueryPhaseExe } final LinkedList collectors = new LinkedList<>(); + // whether the chain contains a collector that filters documents + boolean hasFilterCollector = false; if (searchContext.terminateAfter() != SearchContext.DEFAULT_TERMINATE_AFTER) { // add terminate_after before the filter collectors // it will only be applied on documents accepted by these filter collectors collectors.add(createEarlyTerminationCollectorContext(searchContext.terminateAfter())); + // this collector can filter documents during the collection + hasFilterCollector = true; } if (searchContext.parsedPostFilter() != null) { // add post filters before aggregations // it will only be applied to top hits collectors.add(createFilteredCollectorContext(searcher, searchContext.parsedPostFilter().query())); + // this collector can filter documents during the collection + hasFilterCollector = true; } if (searchContext.queryCollectors().isEmpty() == false) { // plug in additional collectors, like aggregations @@ -146,6 +152,8 @@ static boolean executeInternal(SearchContext searchContext) throws QueryPhaseExe if (searchContext.minimumScore() != null) { // apply the minimum score after multi collector so we filter aggs as well collectors.add(createMinScoreCollectorContext(searchContext.minimumScore())); + // this collector can filter documents during the collection + hasFilterCollector = true; } boolean timeoutSet = scrollContext == null @@ -168,7 +176,7 @@ static boolean executeInternal(SearchContext searchContext) throws QueryPhaseExe } try { - boolean shouldRescore = searchWithCollector(searchContext, searcher, query, collectors, timeoutSet); + boolean shouldRescore = searchWithCollector(searchContext, searcher, query, collectors, hasFilterCollector, timeoutSet); ExecutorService executor = searchContext.indexShard().getThreadPool().executor(ThreadPool.Names.SEARCH); assert executor instanceof EWMATrackingEsThreadPoolExecutor || (executor instanceof EsThreadPoolExecutor == false /* in case thread pool is mocked out in tests */) @@ -195,10 +203,11 @@ private static boolean searchWithCollector( ContextIndexSearcher searcher, Query query, LinkedList collectors, + boolean hasFilterCollector, boolean timeoutSet ) throws IOException { // create the top docs collector last when the other collectors are known - final TopDocsCollectorContext topDocsFactory = createTopDocsCollectorContext(searchContext); + final TopDocsCollectorContext topDocsFactory = createTopDocsCollectorContext(searchContext, hasFilterCollector); // add the top docs collector, the first collector context in the chain collectors.addFirst(topDocsFactory); diff --git a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java index 1f49654c9ddf4..5f5b8569da2f2 100644 --- a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java @@ -8,17 +8,30 @@ package org.elasticsearch.search.query; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PointValues; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.Terms; import org.apache.lucene.queries.spans.SpanQuery; import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.Collector; +import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.FieldExistsQuery; +import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MultiCollector; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; import org.apache.lucene.search.TopFieldCollector; @@ -61,6 +74,13 @@ abstract class TopDocsCollectorContext extends QueryCollectorContext { this.numHits = numHits; } + /** + * Returns the number of top docs to retrieve + */ + final int numHits() { + return numHits; + } + /** * Returns true if the top docs should be re-scored after initial search */ @@ -73,7 +93,20 @@ static class EmptyTopDocsCollectorContext extends TopDocsCollectorContext { private final Collector collector; private final Supplier hitCountSupplier; - private EmptyTopDocsCollectorContext(@Nullable SortAndFormats sortAndFormats, int trackTotalHitsUpTo) { + /** + * Ctr + * @param reader The index reader + * @param query The query to execute + * @param trackTotalHitsUpTo True if the total number of hits should be tracked + * @param hasFilterCollector True if the collector chain contains a filter + */ + private EmptyTopDocsCollectorContext( + IndexReader reader, + Query query, + @Nullable SortAndFormats sortAndFormats, + int trackTotalHitsUpTo, + boolean hasFilterCollector + ) throws IOException { super(REASON_SEARCH_COUNT, 0); this.sort = sortAndFormats == null ? null : sortAndFormats.sort; if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_DISABLED) { @@ -83,16 +116,22 @@ private EmptyTopDocsCollectorContext(@Nullable SortAndFormats sortAndFormats, in } else { TotalHitCountCollector hitCountCollector = new TotalHitCountCollector(); // implicit total hit counts are valid only when there is no filter collector in the chain - if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_ACCURATE) { - this.collector = hitCountCollector; - this.hitCountSupplier = () -> new TotalHits(hitCountCollector.getTotalHits(), TotalHits.Relation.EQUAL_TO); + int hitCount = hasFilterCollector ? -1 : shortcutTotalHitCount(reader, query); + if (hitCount == -1) { + if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_ACCURATE) { + this.collector = hitCountCollector; + this.hitCountSupplier = () -> new TotalHits(hitCountCollector.getTotalHits(), TotalHits.Relation.EQUAL_TO); + } else { + EarlyTerminatingCollector col = new EarlyTerminatingCollector(hitCountCollector, trackTotalHitsUpTo, false); + this.collector = col; + this.hitCountSupplier = () -> new TotalHits( + hitCountCollector.getTotalHits(), + col.hasEarlyTerminated() ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO : TotalHits.Relation.EQUAL_TO + ); + } } else { - EarlyTerminatingCollector col = new EarlyTerminatingCollector(hitCountCollector, trackTotalHitsUpTo, false); - this.collector = col; - this.hitCountSupplier = () -> new TotalHits( - hitCountCollector.getTotalHits(), - col.hasEarlyTerminated() ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO : TotalHits.Relation.EQUAL_TO - ); + this.collector = new EarlyTerminatingCollector(hitCountCollector, 0, false); + this.hitCountSupplier = () -> new TotalHits(hitCount, TotalHits.Relation.EQUAL_TO); } } } @@ -152,7 +191,7 @@ private CollapsingTopDocsCollectorContext( } @Override - Collector create(Collector in) { + Collector create(Collector in) throws IOException { assert in == null; return topDocsCollector; } @@ -187,21 +226,25 @@ private static TopDocsCollector createCollector( /** * Ctr + * @param reader The index reader * @param query The Lucene query * @param sortAndFormats The query sort * @param numHits The number of top hits to retrieve * @param searchAfter The doc this request should "search after" * @param trackMaxScore True if max score should be tracked * @param trackTotalHitsUpTo True if the total number of hits should be tracked + * @param hasFilterCollector True if the collector chain contains at least one collector that can filters document */ private SimpleTopDocsCollectorContext( + IndexReader reader, Query query, @Nullable SortAndFormats sortAndFormats, @Nullable ScoreDoc searchAfter, int numHits, boolean trackMaxScore, - int trackTotalHitsUpTo - ) { + int trackTotalHitsUpTo, + boolean hasFilterCollector + ) throws IOException { super(REASON_SEARCH_TOP_HITS, numHits); this.sortAndFormats = sortAndFormats; @@ -219,9 +262,18 @@ private SimpleTopDocsCollectorContext( topDocsSupplier = new CachedSupplier<>(topDocsCollector::topDocs); totalHitsSupplier = () -> new TotalHits(0, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO); } else { - topDocsCollector = createCollector(sortAndFormats, numHits, searchAfter, trackTotalHitsUpTo); - topDocsSupplier = new CachedSupplier<>(topDocsCollector::topDocs); - totalHitsSupplier = () -> topDocsSupplier.get().totalHits; + // implicit total hit counts are valid only when there is no filter collector in the chain + final int hitCount = hasFilterCollector ? -1 : shortcutTotalHitCount(reader, query); + if (hitCount == -1) { + topDocsCollector = createCollector(sortAndFormats, numHits, searchAfter, trackTotalHitsUpTo); + topDocsSupplier = new CachedSupplier<>(topDocsCollector::topDocs); + totalHitsSupplier = () -> topDocsSupplier.get().totalHits; + } else { + // don't compute hit counts via the collector + topDocsCollector = createCollector(sortAndFormats, numHits, searchAfter, 1); + topDocsSupplier = new CachedSupplier<>(topDocsCollector::topDocs); + totalHitsSupplier = () -> new TotalHits(hitCount, TotalHits.Relation.EQUAL_TO); + } } MaxScoreCollector maxScoreCollector = null; if (sortAndFormats == null) { @@ -263,7 +315,7 @@ TopDocsAndMaxScore newTopDocs() { } @Override - void postProcess(QuerySearchResult result) { + void postProcess(QuerySearchResult result) throws IOException { final TopDocsAndMaxScore topDocs = newTopDocs(); result.topDocs(topDocs, sortAndFormats == null ? null : sortAndFormats.formats); } @@ -274,21 +326,32 @@ static class ScrollingTopDocsCollectorContext extends SimpleTopDocsCollectorCont private final int numberOfShards; private ScrollingTopDocsCollectorContext( + IndexReader reader, Query query, ScrollContext scrollContext, @Nullable SortAndFormats sortAndFormats, int numHits, boolean trackMaxScore, int numberOfShards, - int trackTotalHitsUpTo - ) { - super(query, sortAndFormats, scrollContext.lastEmittedDoc, numHits, trackMaxScore, trackTotalHitsUpTo); + int trackTotalHitsUpTo, + boolean hasFilterCollector + ) throws IOException { + super( + reader, + query, + sortAndFormats, + scrollContext.lastEmittedDoc, + numHits, + trackMaxScore, + trackTotalHitsUpTo, + hasFilterCollector + ); this.scrollContext = Objects.requireNonNull(scrollContext); this.numberOfShards = numberOfShards; } @Override - void postProcess(QuerySearchResult result) { + void postProcess(QuerySearchResult result) throws IOException { final TopDocsAndMaxScore topDocs = newTopDocs(); if (scrollContext.totalHits == null) { // first round @@ -311,68 +374,138 @@ void postProcess(QuerySearchResult result) { } } + /** + * Returns query total hit count if the query is a {@link MatchAllDocsQuery} + * or a {@link TermQuery} and the reader has no deletions, + * -1 otherwise. + */ + static int shortcutTotalHitCount(IndexReader reader, Query query) throws IOException { + while (true) { + // remove wrappers that don't matter for counts + // this is necessary so that we don't only optimize match_all + // queries but also match_all queries that are nested in + // a constant_score query + if (query instanceof ConstantScoreQuery) { + query = ((ConstantScoreQuery) query).getQuery(); + } else if (query instanceof BoostQuery) { + query = ((BoostQuery) query).getQuery(); + } else { + break; + } + } + if (query.getClass() == MatchAllDocsQuery.class) { + return reader.numDocs(); + } else if (query.getClass() == TermQuery.class && reader.hasDeletions() == false) { + final Term term = ((TermQuery) query).getTerm(); + int count = 0; + for (LeafReaderContext context : reader.leaves()) { + count += context.reader().docFreq(term); + } + return count; + } else if (query.getClass() == FieldExistsQuery.class && reader.hasDeletions() == false) { + final String field = ((FieldExistsQuery) query).getField(); + int count = 0; + for (LeafReaderContext context : reader.leaves()) { + FieldInfos fieldInfos = context.reader().getFieldInfos(); + FieldInfo fieldInfo = fieldInfos.fieldInfo(field); + if (fieldInfo != null) { + if (fieldInfo.getDocValuesType() == DocValuesType.NONE) { + // no shortcut possible: it's a text field, empty values are counted as no value. + return -1; + } + if (fieldInfo.getPointIndexDimensionCount() > 0) { + PointValues points = context.reader().getPointValues(field); + if (points != null) { + count += points.getDocCount(); + } + } else if (fieldInfo.getIndexOptions() != IndexOptions.NONE) { + Terms terms = context.reader().terms(field); + if (terms != null) { + count += terms.getDocCount(); + } + } else { + return -1; // no shortcut possible for fields that are not indexed + } + } + } + return count; + } else { + return -1; + } + } + /** * Creates a {@link TopDocsCollectorContext} from the provided searchContext. + * @param hasFilterCollector True if the collector chain contains at least one collector that can filters document. */ - static TopDocsCollectorContext createTopDocsCollectorContext(SearchContext searchContext) { + static TopDocsCollectorContext createTopDocsCollectorContext(SearchContext searchContext, boolean hasFilterCollector) + throws IOException { + final IndexReader reader = searchContext.searcher().getIndexReader(); + final Query query = searchContext.rewrittenQuery(); + // top collectors don't like a size of 0 + final int totalNumDocs = Math.max(1, reader.numDocs()); if (searchContext.size() == 0) { // no matter what the value of from is - return new EmptyTopDocsCollectorContext(searchContext.sort(), searchContext.trackTotalHitsUpTo()); + return new EmptyTopDocsCollectorContext( + reader, + query, + searchContext.sort(), + searchContext.trackTotalHitsUpTo(), + hasFilterCollector + ); + } else if (searchContext.scrollContext() != null) { + // we can disable the tracking of total hits after the initial scroll query + // since the total hits is preserved in the scroll context. + int trackTotalHitsUpTo = searchContext.scrollContext().totalHits != null + ? SearchContext.TRACK_TOTAL_HITS_DISABLED + : SearchContext.TRACK_TOTAL_HITS_ACCURATE; + // no matter what the value of from is + int numDocs = Math.min(searchContext.size(), totalNumDocs); + return new ScrollingTopDocsCollectorContext( + reader, + query, + searchContext.scrollContext(), + searchContext.sort(), + numDocs, + searchContext.trackScores(), + searchContext.numberOfShards(), + trackTotalHitsUpTo, + hasFilterCollector + ); + } else if (searchContext.collapse() != null) { + boolean trackScores = searchContext.sort() == null ? true : searchContext.trackScores(); + int numDocs = Math.min(searchContext.from() + searchContext.size(), totalNumDocs); + return new CollapsingTopDocsCollectorContext( + searchContext.collapse(), + searchContext.sort(), + numDocs, + trackScores, + searchContext.searchAfter() + ); } else { - final IndexReader reader = searchContext.searcher().getIndexReader(); - final Query query = searchContext.rewrittenQuery(); - // top collectors don't like a size of 0 - final int totalNumDocs = Math.max(1, reader.numDocs()); - if (searchContext.scrollContext() != null) { - // we can disable the tracking of total hits after the initial scroll query - // since the total hits is preserved in the scroll context. - int trackTotalHitsUpTo = searchContext.scrollContext().totalHits != null - ? SearchContext.TRACK_TOTAL_HITS_DISABLED - : SearchContext.TRACK_TOTAL_HITS_ACCURATE; - // no matter what the value of from is - int numDocs = Math.min(searchContext.size(), totalNumDocs); - return new ScrollingTopDocsCollectorContext( - query, - searchContext.scrollContext(), - searchContext.sort(), - numDocs, - searchContext.trackScores(), - searchContext.numberOfShards(), - trackTotalHitsUpTo - ); - } else if (searchContext.collapse() != null) { - boolean trackScores = searchContext.sort() == null ? true : searchContext.trackScores(); - int numDocs = Math.min(searchContext.from() + searchContext.size(), totalNumDocs); - return new CollapsingTopDocsCollectorContext( - searchContext.collapse(), - searchContext.sort(), - numDocs, - trackScores, - searchContext.searchAfter() - ); - } else { - int numDocs = Math.min(searchContext.from() + searchContext.size(), totalNumDocs); - final boolean rescore = searchContext.rescore().isEmpty() == false; - if (rescore) { - assert searchContext.sort() == null; - for (RescoreContext rescoreContext : searchContext.rescore()) { - numDocs = Math.max(numDocs, rescoreContext.getWindowSize()); - } + int numDocs = Math.min(searchContext.from() + searchContext.size(), totalNumDocs); + final boolean rescore = searchContext.rescore().isEmpty() == false; + if (rescore) { + assert searchContext.sort() == null; + for (RescoreContext rescoreContext : searchContext.rescore()) { + numDocs = Math.max(numDocs, rescoreContext.getWindowSize()); } - return new SimpleTopDocsCollectorContext( - query, - searchContext.sort(), - searchContext.searchAfter(), - numDocs, - searchContext.trackScores(), - searchContext.trackTotalHitsUpTo() - ) { - @Override - boolean shouldRescore() { - return rescore; - } - }; } + return new SimpleTopDocsCollectorContext( + reader, + query, + searchContext.sort(), + searchContext.searchAfter(), + numDocs, + searchContext.trackScores(), + searchContext.trackTotalHitsUpTo(), + hasFilterCollector + ) { + @Override + boolean shouldRescore() { + return rescore; + } + }; } } diff --git a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java index 8c2ba79a81812..2c3a45f491382 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java @@ -634,7 +634,7 @@ public void testDisableTopScoreCollection() throws Exception { context.parsedQuery(new ParsedQuery(q)); context.setSize(3); context.trackTotalHitsUpTo(3); - TopDocsCollectorContext topDocsContext = TopDocsCollectorContext.createTopDocsCollectorContext(context); + TopDocsCollectorContext topDocsContext = TopDocsCollectorContext.createTopDocsCollectorContext(context, false); assertEquals(topDocsContext.create(null).scoreMode(), org.apache.lucene.search.ScoreMode.COMPLETE); QueryPhase.executeInternal(context); assertEquals(5, context.queryResult().topDocs().topDocs.totalHits.value); @@ -642,7 +642,7 @@ public void testDisableTopScoreCollection() throws Exception { assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(3)); context.sort(new SortAndFormats(new Sort(new SortField("other", SortField.Type.INT)), new DocValueFormat[] { DocValueFormat.RAW })); - topDocsContext = TopDocsCollectorContext.createTopDocsCollectorContext(context); + topDocsContext = TopDocsCollectorContext.createTopDocsCollectorContext(context, false); assertEquals(topDocsContext.create(null).scoreMode(), org.apache.lucene.search.ScoreMode.TOP_DOCS); QueryPhase.executeInternal(context); assertEquals(5, context.queryResult().topDocs().topDocs.totalHits.value); diff --git a/server/src/test/java/org/elasticsearch/search/query/TopDocsCollectorContextTests.java b/server/src/test/java/org/elasticsearch/search/query/TopDocsCollectorContextTests.java new file mode 100644 index 0000000000000..783232d3b033c --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/query/TopDocsCollectorContextTests.java @@ -0,0 +1,83 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.query; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.IntPoint; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedDocValuesField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.FieldExistsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +public class TopDocsCollectorContextTests extends ESTestCase { + + public void testShortcutTotalHitCountTextField() throws IOException { + try (Directory dir = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), dir)) { + Document doc = new Document(); + doc.add(new TextField("text", "value", Field.Store.NO)); + iw.addDocument(doc); + doc = new Document(); + doc.add(new TextField("text", "", Field.Store.NO)); + iw.addDocument(doc); + iw.addDocument(new Document()); + iw.commit(); + try (IndexReader reader = iw.getReader()) { + final Query testQuery = new FieldExistsQuery("text"); + int hitCount = TopDocsCollectorContext.shortcutTotalHitCount(reader, testQuery); + assertEquals(-1, hitCount); + } + } + } + + public void testShortcutTotalHitCountStringField() throws IOException { + try (Directory dir = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), dir)) { + Document doc = new Document(); + doc.add(new StringField("string", "value", Field.Store.NO)); + doc.add(new SortedDocValuesField("string", new BytesRef("value"))); + iw.addDocument(doc); + doc = new Document(); + doc.add(new StringField("string", "", Field.Store.NO)); + doc.add(new SortedDocValuesField("string", new BytesRef(""))); + iw.addDocument(doc); + iw.addDocument(new Document()); + iw.commit(); + try (IndexReader reader = iw.getReader()) { + final Query testQuery = new FieldExistsQuery("string"); + int hitCount = TopDocsCollectorContext.shortcutTotalHitCount(reader, testQuery); + assertEquals(2, hitCount); + } + } + } + + public void testShortcutTotalHitCountNumericField() throws IOException { + try (Directory dir = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), dir)) { + Document doc = new Document(); + doc.add(new IntPoint("int", 10)); + doc.add(new NumericDocValuesField("int", 10)); + iw.addDocument(doc); + iw.addDocument(new Document()); + iw.commit(); + try (IndexReader reader = iw.getReader()) { + final Query testQuery = new FieldExistsQuery("int"); + int hitCount = TopDocsCollectorContext.shortcutTotalHitCount(reader, testQuery); + assertEquals(1, hitCount); + } + } + } +}