jpountz commented on a change in pull request #445: URL: https://github.com/apache/lucene/pull/445#discussion_r750295570
########## File path: lucene/core/src/java/org/apache/lucene/search/DocValuesFieldExistsQuery.java ########## @@ -74,6 +74,16 @@ public Scorer scorer(LeafReaderContext context) throws IOException { return new ConstantScoreScorer(this, score(), scoreMode, iterator); } + @Override + public int count(LeafReaderContext context) throws IOException { + final LeafReader reader = context.reader(); + FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field); + if (fieldInfo == null || fieldInfo.getPointDimensionCount() == 0) { + return -1; Review comment: Nit: In these cases we like to fall back to the default implementation, this way we don't have to know about what the contract is for the case when the count cannot be computed. ```suggestion return super.count(context); ``` ########## File path: lucene/core/src/java/org/apache/lucene/search/DocValuesFieldExistsQuery.java ########## @@ -74,6 +74,16 @@ public Scorer scorer(LeafReaderContext context) throws IOException { return new ConstantScoreScorer(this, score(), scoreMode, iterator); } + @Override + public int count(LeafReaderContext context) throws IOException { + final LeafReader reader = context.reader(); + FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field); + if (fieldInfo == null || fieldInfo.getPointDimensionCount() == 0) { Review comment: You also need to take care of the case when there are deletions on the segment, see how `TermWeight` handles it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org