blob: 095f68c501ea131d7a03999f2b57f746b671bf18 [file] [log] [blame]
diff --git a/lucene/highlighter/build.gradle b/lucene/highlighter/build.gradle
index 6e105d59b6e..28c3692a085 100644
--- a/lucene/highlighter/build.gradle
+++ b/lucene/highlighter/build.gradle
@@ -28,4 +28,6 @@ dependencies {
testImplementation project(':lucene:test-framework')
testImplementation project(':lucene:analysis:common')
+ testImplementation project(':lucene:queryparser')
+ testImplementation "org.assertj:assertj-core"
}
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/HitRegionRetriever.java b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/HitRegionRetriever.java
new file mode 100644
index 00000000000..26329faf1a6
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/HitRegionRetriever.java
@@ -0,0 +1,382 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Matches;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PrimitiveIterator;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Predicate;
+
+/**
+ * Utility class to compute a list of "hit regions" for a document.
+ */
+public class HitRegionRetriever {
+ private final List<LeafReaderContext> leaves;
+ private final Weight weight;
+ private final TreeSet<String> affectedFields;
+ private final Map<String, OffsetsFromMatchesStrategy> offsetStrategies;
+ private final Set<String> preloadFields;
+
+ public HitRegionRetriever(IndexSearcher searcher, Query query, Analyzer analyzer)
+ throws IOException {
+ leaves = searcher.getIndexReader().leaves();
+ assert checkOrderConsistency(leaves);
+
+ weight = searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 0);
+
+ // Compute the subset of fields affected by this query so that we don't load or scan
+ // fields that are irrelevant.
+ affectedFields = new TreeSet<>();
+ query.visit(
+ new QueryVisitor() {
+ @Override
+ public boolean acceptField(String field) {
+ affectedFields.add(field);
+ return false;
+ }
+ });
+
+ // Compute value offset retrieval strategy for all affected fields.
+ offsetStrategies =
+ computeOffsetStrategies(affectedFields, searcher.getIndexReader(), analyzer);
+
+ // Ask offset strategies if they'll need field values.
+ preloadFields = new HashSet<>();
+ offsetStrategies.forEach(
+ (field, strategy) -> {
+ if (strategy.requiresDocument()) {
+ preloadFields.add(field);
+ }
+ });
+
+ // Only preload those field values that can be affected by the query and are required
+ // by strategies.
+ preloadFields.retainAll(affectedFields);
+ }
+
+ public void highlightDocuments(PrimitiveIterator.OfInt docIds, DocumentHitsConsumer consumer)
+ throws IOException {
+ if (leaves.isEmpty() || affectedFields.isEmpty()) {
+ return;
+ }
+
+ Iterator<LeafReaderContext> ctx = leaves.iterator();
+ LeafReaderContext currentContext = ctx.next();
+ int previousDocId = -1;
+ Map<String, List<OffsetRange>> highlights = new TreeMap<>();
+ while (docIds.hasNext()) {
+ int docId = docIds.nextInt();
+
+ if (docId < previousDocId) {
+ throw new RuntimeException("Input document IDs must be sorted (increasing).");
+ }
+ previousDocId = docId;
+
+ while (docId >= currentContext.docBase + currentContext.reader().maxDoc()) {
+ currentContext = ctx.next();
+ }
+
+ int contextRelativeDocId = docId - currentContext.docBase;
+
+ // Only preload fields we may potentially need.
+ FieldValueProvider documentSupplier;
+ if (preloadFields.isEmpty()) {
+ documentSupplier = null;
+ } else {
+ Document doc = currentContext.reader().document(contextRelativeDocId, preloadFields);
+ documentSupplier = new DocumentFieldValueProvider(doc);
+ }
+
+ highlightDocument(
+ currentContext, contextRelativeDocId, documentSupplier, highlights, (field) -> true);
+
+ consumer.accept(currentContext.reader(), contextRelativeDocId, highlights);
+ highlights.clear();
+ }
+ }
+
+ public void highlightDocument(
+ LeafReaderContext currentContext,
+ int contextDocId,
+ FieldValueProvider doc,
+ Map<String, List<OffsetRange>> highlights,
+ Predicate<String> acceptField)
+ throws IOException {
+ Matches matches = weight.matches(currentContext, contextDocId);
+ if (matches == null) {
+ return;
+ }
+
+ // TODO: improve no-position field highlighting if this is merged:
+ // https://issues.apache.org/jira/browse/LUCENE-9439
+ for (String field : affectedFields) {
+ if (acceptField.test(field)) {
+ MatchesIterator matchesIterator = matches.getMatches(field);
+ if (matchesIterator == null) {
+ // No matches on this field or the field is not indexed with positions.
+ } else {
+ OffsetsFromMatchesStrategy offsetStrategy = offsetStrategies.get(field);
+ if (offsetStrategy == null) {
+ throw new IOException(
+ "Non-empty matches but no offset retrieval strategy for field: " + field);
+ }
+ List<OffsetRange> ranges = offsetStrategy.get(matchesIterator, doc);
+ if (!ranges.isEmpty()) {
+ highlights.put(field, ranges);
+ }
+ }
+ }
+ }
+ }
+
+ private boolean checkOrderConsistency(List<LeafReaderContext> leaves) {
+ for (int i = 1; i < leaves.size(); i++) {
+ LeafReaderContext prev = leaves.get(i - 1);
+ LeafReaderContext next = leaves.get(i);
+ assert prev.docBase <= next.docBase;
+ assert prev.docBase + prev.reader().maxDoc() == next.docBase;
+ }
+ return true;
+ }
+
+ private static Map<String, OffsetsFromMatchesStrategy> computeOffsetStrategies(
+ Set<String> affectedFields, IndexReader reader, Analyzer analyzer) {
+ Map<String, OffsetsFromMatchesStrategy> offsetStrategies = new HashMap<>();
+ FieldInfos fieldInfos = FieldInfos.getMergedFieldInfos(reader);
+ for (String field : affectedFields) {
+ FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+
+ OffsetsFromMatchesStrategy offsetStrategy;
+ if (fieldInfo != null && fieldInfo.getIndexOptions() != null) {
+ switch (fieldInfo.getIndexOptions()) {
+ case DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS:
+ offsetStrategy = new OffsetsStored(field);
+ break;
+
+ case DOCS_AND_FREQS_AND_POSITIONS:
+ offsetStrategy = new OffsetsFromPositions(field, analyzer);
+ break;
+
+ default:
+ offsetStrategy =
+ (matchesIterator, doc) -> {
+ throw new IOException(
+ "Field is indexed without positions and/or offsets: "
+ + field
+ + ", "
+ + fieldInfo.getIndexOptions());
+ };
+ }
+ offsetStrategies.put(field, offsetStrategy);
+ }
+ }
+ return offsetStrategies;
+ }
+
+ public interface DocumentHitsConsumer {
+ void accept(LeafReader leafReader, int leafDocId, Map<String, List<OffsetRange>> hits)
+ throws IOException;
+ }
+
+ public interface FieldValueProvider {
+ List<CharSequence> getValues(String field);
+ }
+
+ public static final class DocumentFieldValueProvider implements FieldValueProvider {
+ private final Document doc;
+
+ public DocumentFieldValueProvider(Document doc) {
+ this.doc = doc;
+ }
+
+ @Override
+ public List<CharSequence> getValues(String field) {
+ return Arrays.asList(doc.getValues(field));
+ }
+ }
+
+ private interface OffsetsFromMatchesStrategy {
+ List<OffsetRange> get(MatchesIterator matchesIterator, FieldValueProvider doc)
+ throws IOException;
+
+ default boolean requiresDocument() {
+ return false;
+ }
+ }
+
+ private static class OffsetsStored implements OffsetsFromMatchesStrategy {
+ private final String field;
+
+ OffsetsStored(String field) {
+ this.field = field;
+ }
+
+ @Override
+ public List<OffsetRange> get(MatchesIterator matchesIterator, FieldValueProvider doc)
+ throws IOException {
+ ArrayList<OffsetRange> ranges = new ArrayList<>();
+ while (matchesIterator.next()) {
+ int from = matchesIterator.startOffset();
+ int to = matchesIterator.endOffset();
+ if (from < 0 || to < 0) {
+ throw new IOException("Matches API returned negative offsets for field: " + field);
+ }
+ ranges.add(new OffsetRange(from, to));
+ }
+ return ranges;
+ }
+ }
+
+ private static class OffsetsFromPositions implements OffsetsFromMatchesStrategy {
+ private final String field;
+ private final Analyzer analyzer;
+
+ OffsetsFromPositions(String field, Analyzer analyzer) {
+ this.field = field;
+ this.analyzer = analyzer;
+ }
+
+ @Override
+ public List<OffsetRange> get(MatchesIterator matchesIterator, FieldValueProvider doc)
+ throws IOException {
+ ArrayList<OffsetRange> ranges = new ArrayList<>();
+ while (matchesIterator.next()) {
+ int from = matchesIterator.startPosition();
+ int to = matchesIterator.endPosition();
+ if (from < 0 || to < 0) {
+ throw new IOException("Matches API returned negative positions for field: " + field);
+ }
+ ranges.add(new OffsetRange(from, to));
+ }
+
+ // Convert from positions to offsets.
+ ranges = convertPositionsToOffsets(ranges, analyzer, field, doc.getValues(field));
+
+ return ranges;
+ }
+
+ @Override
+ public boolean requiresDocument() {
+ return true;
+ }
+
+ private static ArrayList<OffsetRange> convertPositionsToOffsets(
+ ArrayList<OffsetRange> ranges,
+ Analyzer analyzer,
+ String fieldName,
+ List<CharSequence> values)
+ throws IOException {
+
+ if (ranges.isEmpty()) {
+ return ranges;
+ }
+
+ class LeftRight {
+ int left = Integer.MAX_VALUE;
+ int right = Integer.MIN_VALUE;
+ }
+
+ Map<Integer, LeftRight> requiredPositionSpans = new HashMap<>();
+ int minPosition = Integer.MAX_VALUE;
+ int maxPosition = Integer.MIN_VALUE;
+ for (OffsetRange range : ranges) {
+ requiredPositionSpans.computeIfAbsent(range.from, (key) -> new LeftRight());
+ requiredPositionSpans.computeIfAbsent(range.to, (key) -> new LeftRight());
+ minPosition = Math.min(minPosition, range.from);
+ maxPosition = Math.max(maxPosition, range.to);
+ }
+
+ int position = -1;
+ int valueOffset = 0;
+ for (int valueIndex = 0, max = values.size(); valueIndex < max; valueIndex++) {
+ final String value = values.get(valueIndex).toString();
+ final boolean lastValue = valueIndex + 1 == max;
+
+ TokenStream ts = analyzer.tokenStream(fieldName, value);
+ OffsetAttribute offsetAttr = ts.getAttribute(OffsetAttribute.class);
+ PositionIncrementAttribute posAttr = ts.getAttribute(PositionIncrementAttribute.class);
+ ts.reset();
+ while (ts.incrementToken()) {
+ position += posAttr.getPositionIncrement();
+
+ if (position >= minPosition) {
+ LeftRight leftRight = requiredPositionSpans.get(position);
+ if (leftRight != null) {
+ int startOffset = valueOffset + offsetAttr.startOffset();
+ int endOffset = valueOffset + offsetAttr.endOffset();
+
+ leftRight.left = Math.min(leftRight.left, startOffset);
+ leftRight.right = Math.max(leftRight.right, endOffset);
+ }
+
+ // Only short-circuit if we're on the last value (which should be the common
+ // case since most fields would only have a single value anyway). We need
+ // to make sure of this because otherwise offsetAttr would have incorrect value.
+ if (position > maxPosition && lastValue) {
+ break;
+ }
+ }
+ }
+ ts.end();
+ valueOffset += offsetAttr.endOffset() + analyzer.getOffsetGap(fieldName);
+ ts.close();
+ }
+
+ ArrayList<OffsetRange> converted = new ArrayList<>();
+ for (OffsetRange range : ranges) {
+ LeftRight left = requiredPositionSpans.get(range.from);
+ LeftRight right = requiredPositionSpans.get(range.to);
+ if (left == null
+ || right == null
+ || left.left == Integer.MAX_VALUE
+ || right.right == Integer.MIN_VALUE) {
+ throw new RuntimeException("Position not properly initialized for range: " + range);
+ }
+ converted.add(new OffsetRange(left.left, right.right));
+ }
+
+ return converted;
+ }
+ }
+}
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/OffsetRange.java b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/OffsetRange.java
new file mode 100644
index 00000000000..89b17fc608c
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/OffsetRange.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+import java.util.Objects;
+
+public class OffsetRange {
+ /** Start index, inclusive. */
+ public final int from;
+
+ /** End index, exclusive. */
+ public final int to;
+
+ /**
+ * @param from Start index, inclusive.
+ * @param to End index, exclusive.
+ */
+ public OffsetRange(int from, int to) {
+ assert from <= to : "A non-empty offset range is required: " + from + "-" + to;
+ this.from = from;
+ this.to = to;
+ }
+
+ public int length() {
+ return to - from;
+ }
+
+ @Override
+ public String toString() {
+ return "[from=" + from + ", to=" + to + "]";
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other == this) return true;
+ if (other instanceof OffsetRange) {
+ OffsetRange that = (OffsetRange) other;
+ return from == that.from && to == that.to;
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(from, to);
+ }
+}
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/Passage.java b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/Passage.java
new file mode 100644
index 00000000000..9a4dc4b0a40
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/Passage.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+import java.util.List;
+
+/**
+ * A passage is a fragment of source text, scored and possibly with a list of sub-offsets (markers)
+ * to be highlighted. The markers can be overlapping or nested, but they're always contained within
+ * the passage.
+ */
+public class Passage extends OffsetRange {
+ public List<OffsetRange> markers;
+
+ public Passage(int from, int to, List<OffsetRange> markers) {
+ super(from, to);
+
+ this.markers = markers;
+ }
+
+ @Override
+ public String toString() {
+ return "[" + super.toString() + ", markers=" + markers + "]";
+ }
+}
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageAdjuster.java b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageAdjuster.java
new file mode 100644
index 00000000000..46bb9c6be9b
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageAdjuster.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+public interface PassageAdjuster {
+ public void currentValue(CharSequence value);
+ public OffsetRange adjust(Passage p);
+}
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageFormatter.java b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageFormatter.java
new file mode 100644
index 00000000000..0e2dd16fbfd
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageFormatter.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.RandomAccess;
+import java.util.function.Function;
+
+public class PassageFormatter {
+ private final String ellipsis;
+ private final Function<OffsetRange, String> markerStart;
+ private final Function<OffsetRange, String> markerEnd;
+
+ private final ArrayList<OffsetRange> markerStack = new ArrayList<>();
+
+ public PassageFormatter(String ellipsis, String markerStart, String markerEnd) {
+ this(ellipsis, (m) -> markerStart, (m) -> markerEnd);
+ }
+
+ public PassageFormatter(
+ String ellipsis,
+ Function<OffsetRange, String> markerStart,
+ Function<OffsetRange, String> markerEnd) {
+ this.ellipsis = ellipsis;
+ this.markerStart = markerStart;
+ this.markerEnd = markerEnd;
+ }
+
+ public List<String> format(CharSequence value, List<Passage> passages, List<OffsetRange> ranges) {
+ assert PassageSelector.sortedAndNonOverlapping(passages);
+ assert PassageSelector.sortedAndNonOverlapping(ranges);
+ assert ranges instanceof RandomAccess;
+
+ if (ranges.isEmpty()) {
+ return Collections.emptyList();
+ }
+
+ ArrayList<String> result = new ArrayList<>();
+ StringBuilder buf = new StringBuilder();
+
+ int rangeIndex = 0;
+ OffsetRange range = ranges.get(rangeIndex);
+ passageFormatting:
+ for (Passage passage : passages) {
+ // Move to the range of the current passage.
+ while (passage.from >= range.to) {
+ if (++rangeIndex == ranges.size()) {
+ break passageFormatting;
+ }
+ range = ranges.get(rangeIndex);
+ }
+
+ assert range.from <= passage.from && range.to >= passage.to : range + " ? " + passage;
+
+ buf.setLength(0);
+ if (range.from < passage.from) {
+ buf.append(ellipsis);
+ }
+ format(buf, value, passage);
+ if (range.to > passage.to) {
+ buf.append(ellipsis);
+ }
+ result.add(buf.toString());
+ }
+ return result;
+ }
+
+ public StringBuilder format(StringBuilder buf, CharSequence value, final Passage passage) {
+ switch (passage.markers.size()) {
+ case 0:
+ // No markers, full passage appended.
+ buf.append(value, passage.from, passage.to);
+ break;
+
+ case 1:
+ // One marker, trivial and frequent case so it's handled separately.
+ OffsetRange m = passage.markers.iterator().next();
+ buf.append(value, passage.from, m.from);
+ buf.append(markerStart.apply(m));
+ buf.append(value, m.from, m.to);
+ buf.append(markerEnd.apply(m));
+ buf.append(value, m.to, passage.to);
+ break;
+
+ default:
+ // Multiple markers, possibly overlapping or nested.
+ markerStack.clear();
+ multipleMarkers(value, passage, buf, markerStack);
+ break;
+ }
+
+ return buf;
+ }
+
+ /** Handle multiple markers, possibly overlapping or nested. */
+ private void multipleMarkers(
+ CharSequence value, final Passage p, StringBuilder b, ArrayList<OffsetRange> markerStack) {
+ int at = p.from;
+ int max = p.to;
+ SlicePoint[] slicePoints = slicePoints(p);
+ for (SlicePoint slicePoint : slicePoints) {
+ b.append(value, at, slicePoint.offset);
+ OffsetRange currentMarker = slicePoint.marker;
+ switch (slicePoint.type) {
+ case START:
+ markerStack.add(currentMarker);
+ b.append(markerStart.apply(currentMarker));
+ break;
+
+ case END:
+ int markerIndex = markerStack.lastIndexOf(currentMarker);
+ for (int k = markerIndex; k < markerStack.size(); k++) {
+ b.append(markerEnd.apply(markerStack.get(k)));
+ }
+ markerStack.remove(markerIndex);
+ for (int k = markerIndex; k < markerStack.size(); k++) {
+ b.append(markerStart.apply(markerStack.get(k)));
+ }
+ break;
+
+ default:
+ throw new RuntimeException();
+ }
+
+ at = slicePoint.offset;
+ }
+
+ if (at < max) {
+ b.append(value, at, max);
+ }
+ }
+
+ private static SlicePoint[] slicePoints(Passage p) {
+ SlicePoint[] slicePoints = new SlicePoint[p.markers.size() * 2];
+ int x = 0;
+ for (OffsetRange m : p.markers) {
+ slicePoints[x++] = new SlicePoint(SlicePoint.Type.START, m.from, m);
+ slicePoints[x++] = new SlicePoint(SlicePoint.Type.END, m.to, m);
+ }
+
+ // Order slice points by their offset
+ Comparator<SlicePoint> c =
+ Comparator.<SlicePoint>comparingInt(pt -> pt.offset)
+ .thenComparingInt(pt -> pt.type.ordering)
+ .thenComparing(
+ (a, b) -> {
+ if (a.type == SlicePoint.Type.START) {
+ // Longer start slice points come first.
+ return Integer.compare(b.marker.to, a.marker.to);
+ } else {
+ // Shorter end slice points come first.
+ return Integer.compare(b.marker.from, a.marker.from);
+ }
+ });
+
+ Arrays.sort(slicePoints, c);
+
+ return slicePoints;
+ }
+
+ static class SlicePoint {
+ enum Type {
+ START(2),
+ END(1);
+
+ private final int ordering;
+
+ Type(int ordering) {
+ this.ordering = ordering;
+ }
+ }
+
+ public final int offset;
+ public final Type type;
+ public final OffsetRange marker;
+
+ public SlicePoint(Type t, int offset, OffsetRange m) {
+ this.type = t;
+ this.offset = offset;
+ this.marker = m;
+ }
+
+ @Override
+ public String toString() {
+ return "(" + type + ", " + marker + ")";
+ }
+ }
+}
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageSelector.java b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageSelector.java
new file mode 100644
index 00000000000..95609132a6e
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/PassageSelector.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+import org.apache.lucene.util.PriorityQueue;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.RandomAccess;
+
+/** Selects fragments of text that score best for the given set of highlight markers. */
+public class PassageSelector {
+ public static final Comparator<Passage> DEFAULT_SCORER =
+ (a, b) -> {
+ // Compare the number of highlights first.
+ int v;
+ v = Integer.compare(a.markers.size(), b.markers.size());
+ if (v != 0) {
+ return v;
+ }
+
+ // Total number of characters covered by the highlights.
+ int len1 = 0, len2 = 0;
+ for (OffsetRange o : a.markers) {
+ len1 += o.length();
+ }
+ for (OffsetRange o : b.markers) {
+ len2 += o.length();
+ }
+ if (len1 != len2) {
+ return Integer.compare(len1, len2);
+ }
+
+ return Integer.compare(b.from, a.from);
+ };
+
+ private final Comparator<Passage> passageScorer;
+ private final PassageAdjuster passageAdjuster;
+
+ public PassageSelector() {
+ this(DEFAULT_SCORER, null);
+ }
+
+ public PassageSelector(Comparator<Passage> passageScorer, PassageAdjuster passageAdjuster) {
+ this.passageScorer = passageScorer;
+ this.passageAdjuster = passageAdjuster;
+ }
+
+ public List<Passage> pickBest(
+ CharSequence value,
+ List<? extends OffsetRange> markers,
+ int maxPassageWindow,
+ int maxPassages) {
+ return pickBest(
+ value, markers, maxPassageWindow, maxPassages, List.of(new OffsetRange(0, value.length())));
+ }
+
+ public List<Passage> pickBest(
+ CharSequence value,
+ List<? extends OffsetRange> markers,
+ int maxPassageWindow,
+ int maxPassages,
+ List<OffsetRange> permittedPassageRanges) {
+ assert markers instanceof RandomAccess && permittedPassageRanges instanceof RandomAccess;
+
+ // Handle odd special cases early.
+ if (value.length() == 0 || maxPassageWindow == 0) {
+ return Collections.emptyList();
+ }
+
+ // Sort markers by their start offset, shortest first.
+ markers.sort(
+ (a, b) -> {
+ int v = Integer.compare(a.from, b.from);
+ return v != 0 ? v : Integer.compare(a.to, b.to);
+ });
+
+ // Determine a maximum offset window around each highlight marker and
+ // pick the best scoring passage candidates.
+ PriorityQueue<Passage> pq =
+ new PriorityQueue<>(maxPassages) {
+ @Override
+ protected boolean lessThan(Passage a, Passage b) {
+ return passageScorer.compare(a, b) < 0;
+ }
+ };
+
+ assert sortedAndNonOverlapping(permittedPassageRanges);
+
+ final int max = markers.size();
+ int markerIndex = 0;
+ nextRange:
+ for (OffsetRange range : permittedPassageRanges) {
+ final int rangeTo = Math.min(range.to, value.length());
+
+ // Skip ranges outside of the value window anyway.
+ if (range.from >= rangeTo) {
+ continue;
+ }
+
+ while (markerIndex < max) {
+ OffsetRange m = markers.get(markerIndex);
+
+ // Markers are sorted so if the current marker's start is past the range,
+ // we can advance, but we need to check the same marker against the new range.
+ if (m.from >= rangeTo) {
+ continue nextRange;
+ }
+
+ // Check if current marker falls within the range and is smaller than the largest allowed
+ // passage window.
+ if (m.from >= range.from && m.to <= rangeTo && m.length() <= maxPassageWindow) {
+
+ // Adjust the window range to center the highlight marker.
+ int from = (m.from + m.to - maxPassageWindow) / 2;
+ int to = (m.from + m.to + maxPassageWindow) / 2;
+ if (from < range.from) {
+ to += range.from - from;
+ from = range.from;
+ }
+ if (to > rangeTo) {
+ from -= to - rangeTo;
+ to = rangeTo;
+ if (from < range.from) {
+ from = range.from;
+ }
+ }
+
+ if (from < to && to <= value.length()) {
+ // Find other markers that are completely inside the passage window.
+ ArrayList<OffsetRange> inside = new ArrayList<>();
+ int i = markerIndex;
+ while (i > 0 && markers.get(i - 1).from >= from) {
+ i--;
+ }
+
+ OffsetRange c;
+ for (; i < max && (c = markers.get(i)).from < to; i++) {
+ if (c.to <= to) {
+ inside.add(c);
+ }
+ }
+
+ if (!inside.isEmpty()) {
+ pq.insertWithOverflow(new Passage(from, to, inside));
+ }
+ }
+ }
+
+ // Advance to the next marker.
+ markerIndex++;
+ }
+ }
+
+ // Collect from the priority queue (reverse the order so that highest-scoring are first).
+ Passage[] passages;
+ if (pq.size() > 0) {
+ passages = new Passage[pq.size()];
+ for (int i = pq.size(); --i >= 0; ) {
+ passages[i] = pq.pop();
+ }
+ } else {
+ // Handle the default, no highlighting markers case.
+ passages = pickDefaultPassage(value, maxPassageWindow, permittedPassageRanges);
+ }
+
+ // Correct passage boundaries from maxExclusive window. Typically shrink boundaries until we're
+ // on a proper word/sentence boundary.
+ if (passageAdjuster != null) {
+ passageAdjuster.currentValue(value);
+ for (int x = 0; x < passages.length; x++) {
+ Passage p = passages[x];
+ OffsetRange newRange = passageAdjuster.adjust(p);
+ if (newRange.from != p.from || newRange.to != p.to) {
+ assert newRange.from >= p.from && newRange.to <= p.to
+ : "Adjusters must not expand the passage's range: was "
+ + p
+ + " => changed to "
+ + newRange;
+ passages[x] = new Passage(newRange.from, newRange.to, p.markers);
+ }
+ }
+ }
+
+ // Ensure there are no overlaps on passages. In case of conflicts, better score wins.
+ int last = 0;
+ for (int i = 0; i < passages.length; i++) {
+ Passage a = passages[i];
+ if (a != null && a.length() > 0) {
+ passages[last++] = a;
+ for (int j = i + 1; j < passages.length; j++) {
+ Passage b = passages[j];
+ if (b != null) {
+ if (adjecentOrOverlapping(a, b)) {
+ passages[j] = null;
+ }
+ }
+ }
+ }
+ }
+
+ // Remove nullified slots.
+ if (passages.length != last) {
+ passages = Arrays.copyOf(passages, last);
+ }
+
+ // Sort in the offset order again.
+ Arrays.sort(passages, (a, b) -> Integer.compare(a.from, b.from));
+
+ return Arrays.asList(passages);
+ }
+
+ static boolean sortedAndNonOverlapping(List<? extends OffsetRange> permittedPassageRanges) {
+ if (permittedPassageRanges.size() > 1) {
+ Iterator<? extends OffsetRange> i = permittedPassageRanges.iterator();
+ for (OffsetRange next, previous = i.next(); i.hasNext(); previous = next) {
+ next = i.next();
+ if (previous.to > next.from) {
+ throw new AssertionError(
+ "Ranges must be sorted and non-overlapping: " + permittedPassageRanges);
+ }
+ }
+ }
+
+ return true;
+ }
+
+ /**
+ * Invoked when no passages could be selected (due to constraints or lack of highlight markers).
+ */
+ protected Passage[] pickDefaultPassage(
+ CharSequence value, int maxCharacterWindow, List<OffsetRange> permittedPassageRanges) {
+ // Search for the first range that is not empty.
+ for (OffsetRange o : permittedPassageRanges) {
+ int to = Math.min(value.length(), o.to);
+ if (o.from < to) {
+ return new Passage[] {
+ new Passage(
+ o.from, o.from + Math.min(maxCharacterWindow, o.length()), Collections.emptyList())
+ };
+ }
+ }
+
+ return new Passage[] {};
+ }
+
+ private static boolean adjecentOrOverlapping(Passage a, Passage b) {
+ if (a.from >= b.from) {
+ return a.from <= b.to - 1;
+ } else {
+ return a.to - 1 >= b.from;
+ }
+ }
+}
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/package-info.java b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/package-info.java
new file mode 100644
index 00000000000..9e7554a5f91
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/matchhighlight/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Highlighter that works on top of {@link org.apache.lucene.search.Matches} API.
+ */
+package org.apache.lucene.search.matchhighlight;
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/HitRegionRetrieverTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/HitRegionRetrieverTest.java
new file mode 100644
index 00000000000..15f5c4d8cd1
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/HitRegionRetrieverTest.java
@@ -0,0 +1,614 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+import com.carrotsearch.randomizedtesting.RandomizedTest;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
+import org.apache.lucene.analysis.synonym.SynonymGraphFilter;
+import org.apache.lucene.analysis.synonym.SynonymMap;
+import org.apache.lucene.analysis.util.CharTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.standard.StandardQueryParser;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class HitRegionRetrieverTest extends LuceneTestCase {
+ private static final String FLD_ID = "field_id";
+
+ private static final String FLD_TEXT_POS_OFFS1 = "field_text_offs1";
+ private static final String FLD_TEXT_POS_OFFS2 = "field_text_offs2";
+
+ private static final String FLD_TEXT_POS_OFFS = "field_text_offs";
+ private static final String FLD_TEXT_POS = "field_text";
+
+ private static final String FLD_TEXT_SYNONYMS_POS_OFFS = "field_text_syns_offs";
+ private static final String FLD_TEXT_SYNONYMS_POS = "field_text_syns";
+
+ private static final String FLD_TEXT_NOPOS = "field_text_nopos";
+
+ private static final String FLD_NON_EXISTING = "field_missing";
+
+ private FieldType TYPE_STORED_WITH_OFFSETS;
+ private FieldType TYPE_STORED_NO_POSITIONS;
+
+ private Analyzer analyzer;
+
+ @Before
+ public void setup() {
+ TYPE_STORED_WITH_OFFSETS = new FieldType(TextField.TYPE_STORED);
+ TYPE_STORED_WITH_OFFSETS.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+ TYPE_STORED_WITH_OFFSETS.freeze();
+
+ TYPE_STORED_NO_POSITIONS = new FieldType(TextField.TYPE_STORED);
+ TYPE_STORED_NO_POSITIONS.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+ TYPE_STORED_NO_POSITIONS.freeze();
+
+ Analyzer whitespaceAnalyzer =
+ new Analyzer() {
+ int offsetGap = RandomizedTest.randomIntBetween(0, 2);
+
+ @Override
+ protected TokenStreamComponents createComponents(String fieldName) {
+ int maxTokenLength = Integer.MAX_VALUE;
+ return new TokenStreamComponents(
+ new WhitespaceTokenizer(CharTokenizer.DEFAULT_MAX_WORD_LEN));
+ }
+
+ @Override
+ public int getOffsetGap(String fieldName) {
+ return offsetGap;
+ }
+ };
+
+ Map<String, Analyzer> fieldAnalyzers = new HashMap<>();
+ fieldAnalyzers.put(FLD_TEXT_POS, whitespaceAnalyzer);
+ fieldAnalyzers.put(FLD_TEXT_POS_OFFS, whitespaceAnalyzer);
+ fieldAnalyzers.put(FLD_TEXT_POS_OFFS1, whitespaceAnalyzer);
+ fieldAnalyzers.put(FLD_TEXT_POS_OFFS2, whitespaceAnalyzer);
+ fieldAnalyzers.put(FLD_TEXT_NOPOS, whitespaceAnalyzer);
+
+ try {
+ SynonymMap.Builder b = new SynonymMap.Builder();
+ b.add(new CharsRef("foo\u0000bar"), new CharsRef("syn1"), true);
+ b.add(new CharsRef("baz"), new CharsRef("syn2\u0000syn3"), true);
+ SynonymMap synonymMap = b.build();
+ Analyzer synonymsAnalyzer =
+ new Analyzer() {
+ @Override
+ protected TokenStreamComponents createComponents(String fieldName) {
+ Tokenizer tokenizer = new WhitespaceTokenizer();
+ TokenStream tokenStream = new SynonymGraphFilter(tokenizer, synonymMap, true);
+ return new TokenStreamComponents(tokenizer, tokenStream);
+ }
+ };
+ fieldAnalyzers.put(FLD_TEXT_SYNONYMS_POS_OFFS, synonymsAnalyzer);
+ fieldAnalyzers.put(FLD_TEXT_SYNONYMS_POS, synonymsAnalyzer);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+
+ analyzer = new PerFieldAnalyzerWrapper(new MissingAnalyzer(), fieldAnalyzers);
+ }
+
+ BiFunction<String, String, Query> stdQueryParser =
+ (query, defField) -> {
+ try {
+ StandardQueryParser parser = new StandardQueryParser(analyzer);
+ parser.setDefaultOperator(StandardQueryConfigHandler.Operator.AND);
+ return parser.parse(query, defField);
+ } catch (QueryNodeException e) {
+ throw new RuntimeException(e);
+ }
+ };
+
+ @Test
+ public void testTermQueryWithOffsets() throws IOException {
+ checkTermQuery(FLD_TEXT_POS_OFFS);
+ }
+
+ @Test
+ public void testTermQueryWithPositions() throws IOException {
+ checkTermQuery(FLD_TEXT_POS);
+ }
+
+ private void checkTermQuery(String field) throws IOException {
+ withReader(
+ List.of(
+ Map.of(field, values("foo bar baz")),
+ Map.of(field, values("bar foo baz")),
+ Map.of(field, values("bar baz foo")),
+ Map.of(field, values("bar bar bar irrelevant"))),
+ reader -> {
+ Assertions.assertThat(highlights(reader, new TermQuery(new Term(field, "foo"))))
+ .containsOnly(
+ fmt("0: (%s: '>foo< bar baz')", field),
+ fmt("1: (%s: 'bar >foo< baz')", field),
+ fmt("2: (%s: 'bar baz >foo<')", field));
+ });
+ }
+
+ @Test
+ public void testBooleanMultifieldQueryWithOffsets() throws IOException {
+ checkBooleanMultifieldQuery(FLD_TEXT_POS_OFFS);
+ }
+
+ @Test
+ public void testBooleanMultifieldQueryWithPositions() throws IOException {
+ checkBooleanMultifieldQuery(FLD_TEXT_POS);
+ }
+
+ private void checkBooleanMultifieldQuery(String field) throws IOException {
+ Query query =
+ new BooleanQuery.Builder()
+ .add(new PhraseQuery(1, field, "foo", "baz"), BooleanClause.Occur.SHOULD)
+ .add(new TermQuery(new Term(FLD_NON_EXISTING, "abc")), BooleanClause.Occur.SHOULD)
+ .add(new TermQuery(new Term(field, "xyz")), BooleanClause.Occur.MUST_NOT)
+ .build();
+
+ withReader(
+ List.of(
+ Map.of(field, values("foo bar baz abc")),
+ Map.of(field, values("bar foo baz def")),
+ Map.of(field, values("bar baz foo xyz"))),
+ reader -> {
+ Assertions.assertThat(highlights(reader, query))
+ .containsOnly(
+ fmt("0: (%s: '>foo bar baz< abc')", field),
+ fmt("1: (%s: 'bar >foo baz< def')", field));
+ });
+ }
+
+ @Test
+ public void testVariousQueryTypesWithOffsets() throws IOException {
+ checkVariousQueryTypes(FLD_TEXT_POS_OFFS);
+ }
+
+ @Test
+ public void testVariousQueryTypesWithPositions() throws IOException {
+ checkVariousQueryTypes(FLD_TEXT_POS);
+ }
+
+ private void checkVariousQueryTypes(String field) throws IOException {
+ withReader(
+ List.of(
+ Map.of(field, values("foo bar baz abc")),
+ Map.of(field, values("bar foo baz def")),
+ Map.of(field, values("bar baz foo xyz"))),
+ reader -> {
+ Assertions.assertThat(highlights(reader, stdQueryParser.apply("foo baz", field)))
+ .containsOnly(
+ fmt("0: (%s: '>foo< bar >baz< abc')", field),
+ fmt("1: (%s: 'bar >foo< >baz< def')", field),
+ fmt("2: (%s: 'bar >baz< >foo< xyz')", field));
+
+ Assertions.assertThat(highlights(reader, stdQueryParser.apply("foo OR xyz", field)))
+ .containsOnly(
+ fmt("0: (%s: '>foo< bar baz abc')", field),
+ fmt("1: (%s: 'bar >foo< baz def')", field),
+ fmt("2: (%s: 'bar baz >foo< >xyz<')", field));
+
+ Assertions.assertThat(highlights(reader, stdQueryParser.apply("bas~2", field)))
+ .containsOnly(
+ fmt("0: (%s: 'foo >bar< >baz< >abc<')", field),
+ fmt("1: (%s: '>bar< foo >baz< def')", field),
+ fmt("2: (%s: '>bar< >baz< foo xyz')", field));
+
+ Assertions.assertThat(highlights(reader, stdQueryParser.apply("\"foo bar\"", field)))
+ .containsOnly(fmt("0: (%s: '>foo bar< baz abc')", field));
+
+ Assertions.assertThat(highlights(reader, stdQueryParser.apply("\"foo bar\"~3", field)))
+ .containsOnly(
+ fmt("0: (%s: '>foo bar< baz abc')", field),
+ fmt("1: (%s: '>bar foo< baz def')", field),
+ fmt("2: (%s: '>bar baz foo< xyz')", field));
+
+ Assertions.assertThat(highlights(reader, stdQueryParser.apply("ba*", field)))
+ .containsOnly(
+ fmt("0: (%s: 'foo >bar< >baz< abc')", field),
+ fmt("1: (%s: '>bar< foo >baz< def')", field),
+ fmt("2: (%s: '>bar< >baz< foo xyz')", field));
+
+ Assertions.assertThat(highlights(reader, stdQueryParser.apply("[bar TO bas]", field)))
+ .containsOnly(
+ fmt("0: (%s: 'foo >bar< baz abc')", field),
+ fmt("1: (%s: '>bar< foo baz def')", field),
+ fmt("2: (%s: '>bar< baz foo xyz')", field));
+
+ // Note how document '2' has 'bar' that isn't highlighted (because this
+ // document is excluded in the first clause).
+ Assertions.assertThat(
+ highlights(reader, stdQueryParser.apply("([bar TO baz] -xyz) OR baz", field)))
+ .containsOnly(
+ fmt("0: (%s: 'foo >bar< >>baz<< abc')", field),
+ fmt("1: (%s: '>bar< foo >>baz<< def')", field),
+ fmt("2: (%s: 'bar >baz< foo xyz')", field));
+
+ Assertions.assertThat(highlights(reader, new MatchAllDocsQuery())).isEmpty();
+ });
+
+ withReader(
+ List.of(
+ Map.of(field, values("foo baz foo")),
+ Map.of(field, values("bas baz foo")),
+ Map.of(field, values("bar baz foo xyz"))),
+ reader -> {
+ Assertions.assertThat(
+ highlights(reader, stdQueryParser.apply("[bar TO baz] -bar", field)))
+ .containsOnly(
+ fmt("0: (%s: 'foo >baz< foo')", field), fmt("1: (%s: '>bas< >baz< foo')", field));
+ });
+ }
+
+ @Test
+ public void testMultivaluedFieldsWithOffsets() throws IOException {
+ checkMultivaluedFields(FLD_TEXT_POS_OFFS);
+ }
+
+ @Test
+ public void testMultivaluedFieldsWithPositions() throws IOException {
+ checkMultivaluedFields(FLD_TEXT_POS);
+ }
+
+ public void checkMultivaluedFields(String field) throws IOException {
+ withReader(
+ List.of(
+ Map.of(field, values("foo bar", "baz abc", "bad baz")),
+ Map.of(field, values("bar foo", "baz def")),
+ Map.of(field, values("bar baz", "foo xyz"))),
+ reader -> {
+ Assertions.assertThat(highlights(reader, stdQueryParser.apply("baz", field)))
+ .containsOnly(
+ fmt("0: (%s: '>baz< abc | bad >baz<')", field),
+ fmt("1: (%s: '>baz< def')", field),
+ fmt("2: (%s: 'bar >baz<')", field));
+ });
+ }
+
+ @Test
+ public void testMultiFieldHighlights() throws IOException {
+ for (String[] fields :
+ new String[][] {
+ {FLD_TEXT_POS_OFFS1, FLD_TEXT_POS_OFFS2},
+ {FLD_TEXT_POS, FLD_TEXT_POS_OFFS2},
+ {FLD_TEXT_POS_OFFS1, FLD_TEXT_POS}
+ }) {
+ String field1 = fields[0];
+ String field2 = fields[1];
+ withReader(
+ List.of(
+ Map.of(
+ field1, values("foo bar", "baz abc"),
+ field2, values("foo baz", "loo bar"))),
+ reader -> {
+ String ordered =
+ Stream.of(fmt("(%s: '>baz< abc')", field1), fmt("(%s: 'loo >bar<')", field2))
+ .sorted()
+ .collect(Collectors.joining(""));
+
+ Assertions.assertThat(
+ highlights(
+ reader,
+ stdQueryParser.apply(field1 + ":baz" + " OR " + field2 + ":bar", field1)))
+ .containsOnly(fmt("0: %s", ordered));
+ });
+ }
+ }
+
+ @Test
+ public void testNestedQueryHitsWithOffsets() throws IOException {
+ checkNestedQueryHits(FLD_TEXT_POS_OFFS);
+ }
+
+ @Test
+ public void testNestedQueryHitsWithPositions() throws IOException {
+ checkNestedQueryHits(FLD_TEXT_POS);
+ }
+
+ public void checkNestedQueryHits(String field) throws IOException {
+ withReader(
+ List.of(Map.of(field, values("foo bar baz abc"))),
+ reader -> {
+ Assertions.assertThat(
+ highlights(
+ reader,
+ new BooleanQuery.Builder()
+ .add(new PhraseQuery(1, field, "foo", "baz"), BooleanClause.Occur.SHOULD)
+ .add(new TermQuery(new Term(field, "bar")), BooleanClause.Occur.SHOULD)
+ .build()))
+ .containsOnly(fmt("0: (%s: '>foo >bar< baz< abc')", field));
+
+ Assertions.assertThat(
+ highlights(
+ reader,
+ new BooleanQuery.Builder()
+ .add(new PhraseQuery(1, field, "foo", "baz"), BooleanClause.Occur.SHOULD)
+ .add(new TermQuery(new Term(field, "bar")), BooleanClause.Occur.SHOULD)
+ .add(new TermQuery(new Term(field, "baz")), BooleanClause.Occur.SHOULD)
+ .build()))
+ .containsOnly(fmt("0: (%s: '>foo >bar< >baz<< abc')", field));
+ });
+ }
+
+ @Test
+ public void testGraphQueryWithOffsets() throws Exception {
+ checkGraphQuery(FLD_TEXT_SYNONYMS_POS_OFFS);
+ }
+
+ @Test
+ public void testGraphQueryWithPositions() throws Exception {
+ checkGraphQuery(FLD_TEXT_SYNONYMS_POS);
+ }
+
+ private void checkGraphQuery(String field) throws IOException {
+ withReader(
+ List.of(
+ Map.of(field, values("foo bar baz")),
+ Map.of(field, values("bar foo baz")),
+ Map.of(field, values("bar baz foo")),
+ Map.of(field, values("bar bar bar irrelevant"))),
+ reader -> {
+ Assertions.assertThat(highlights(reader, new TermQuery(new Term(field, "syn1"))))
+ .containsOnly(fmt("0: (%s: '>foo bar< baz')", field));
+
+ // [syn2 syn3] = baz
+ // so both these queries highlight baz.
+ Assertions.assertThat(highlights(reader, new TermQuery(new Term(field, "syn3"))))
+ .containsOnly(
+ fmt("0: (%s: 'foo bar >baz<')", field),
+ fmt("1: (%s: 'bar foo >baz<')", field),
+ fmt("2: (%s: 'bar >baz< foo')", field));
+ Assertions.assertThat(
+ highlights(reader, stdQueryParser.apply(field + ":\"syn2 syn3\"", field)))
+ .containsOnly(
+ fmt("0: (%s: 'foo bar >baz<')", field),
+ fmt("1: (%s: 'bar foo >baz<')", field),
+ fmt("2: (%s: 'bar >baz< foo')", field));
+ Assertions.assertThat(
+ highlights(reader, stdQueryParser.apply(field + ":\"foo syn2 syn3\"", field)))
+ .containsOnly(fmt("1: (%s: 'bar >foo baz<')", field));
+ });
+ }
+
+ @Test
+ public void testSpanQueryWithOffsets() throws Exception {
+ checkSpanQueries(FLD_TEXT_POS_OFFS);
+ }
+
+ @Test
+ public void testSpanQueryWithPositions() throws Exception {
+ checkSpanQueries(FLD_TEXT_POS);
+ }
+
+ private void checkSpanQueries(String field) throws IOException {
+ withReader(
+ List.of(
+ Map.of(field, values("foo bar baz")),
+ Map.of(field, values("bar foo baz")),
+ Map.of(field, values("bar baz foo")),
+ Map.of(field, values("bar bar bar irrelevant"))),
+ reader -> {
+ Assertions.assertThat(
+ highlights(
+ reader,
+ SpanNearQuery.newOrderedNearQuery(field)
+ .addClause(new SpanTermQuery(new Term(field, "bar")))
+ .addClause(new SpanTermQuery(new Term(field, "foo")))
+ .build()))
+ .containsOnly(fmt("1: (%s: '>bar foo< baz')", field));
+
+ Assertions.assertThat(
+ highlights(
+ reader,
+ SpanNearQuery.newOrderedNearQuery(field)
+ .addClause(new SpanTermQuery(new Term(field, "bar")))
+ .addGap(1)
+ .addClause(new SpanTermQuery(new Term(field, "foo")))
+ .build()))
+ .containsOnly(fmt("2: (%s: '>bar baz foo<')", field));
+
+ Assertions.assertThat(
+ highlights(
+ reader,
+ SpanNearQuery.newUnorderedNearQuery(field)
+ .addClause(new SpanTermQuery(new Term(field, "foo")))
+ .addClause(new SpanTermQuery(new Term(field, "bar")))
+ .build()))
+ .containsOnly(
+ fmt("0: (%s: '>foo bar< baz')", field), fmt("1: (%s: '>bar foo< baz')", field));
+
+ Assertions.assertThat(
+ highlights(
+ reader,
+ SpanNearQuery.newUnorderedNearQuery(field)
+ .addClause(new SpanTermQuery(new Term(field, "foo")))
+ .addClause(new SpanTermQuery(new Term(field, "bar")))
+ .setSlop(1)
+ .build()))
+ .containsOnly(
+ fmt("0: (%s: '>foo bar< baz')", field),
+ fmt("1: (%s: '>bar foo< baz')", field),
+ fmt("2: (%s: '>bar baz foo<')", field));
+ });
+ }
+
+ /**
+ * This test runs a term query against a field with no stored
+ * positions or offsets. Ideally, the highlighter should return the field
+ * that caused the document to be included - perhaps with the full
+ * range of the field's value.
+ *
+ * Such field structure is often useful for multivalued "keyword-like"
+ * fields.
+ */
+ @Test
+ public void testTextFieldNoPositions() throws Exception {
+ String field = FLD_TEXT_NOPOS;
+ withReader(
+ List.of(
+ Map.of(
+ FLD_TEXT_NOPOS, values("foo bar"))
+ ),
+ reader -> {
+ Assertions.assertThat(
+ highlights(
+ reader,
+ new TermQuery(new Term(field, "bar"))))
+ .containsOnly(
+ fmt("0: (%s: '>foo bar<')", field));
+ });
+ }
+
+ private List<String> highlights(IndexReader reader, Query query) throws IOException {
+ IndexSearcher searcher = new IndexSearcher(reader);
+ int maxDocs = 1000;
+
+ Query rewrittenQuery = searcher.rewrite(query);
+ TopDocs topDocs = searcher.search(rewrittenQuery, maxDocs);
+
+ ArrayList<String> highlights = new ArrayList<>();
+
+ SimpleHighlightFormatter formatter = new SimpleHighlightFormatter(analyzer);
+
+ HitRegionRetriever.DocumentHitsConsumer highlightCollector =
+ (leafReader, docId, fieldHighlights) -> {
+ StringBuilder sb = new StringBuilder();
+
+ Document document = leafReader.document(docId);
+ sb.append(document.get(FLD_ID)).append(": ");
+ formatter
+ .apply(document, new TreeMap<>(fieldHighlights))
+ .forEach(
+ (field, snippets) -> {
+ sb.append(
+ String.format(
+ Locale.ROOT, "(%s: '%s')", field, String.join(" | ", snippets)));
+ });
+
+ highlights.add(sb.toString());
+ };
+
+ HitRegionRetriever highlighter = new HitRegionRetriever(searcher, rewrittenQuery, analyzer);
+ highlighter.highlightDocuments(
+ Arrays.stream(topDocs.scoreDocs).mapToInt(scoreDoc -> scoreDoc.doc).sorted().iterator(),
+ highlightCollector);
+
+ return highlights;
+ }
+
+ private String[] values(String... values) {
+ Assertions.assertThat(values).isNotEmpty();
+ return values;
+ }
+
+ private void withReader(
+ Collection<Map<String, String[]>> docs, IOUtils.IOConsumer<DirectoryReader> block)
+ throws IOException {
+ IndexWriterConfig config = new IndexWriterConfig(analyzer);
+
+ try (Directory directory = new ByteBuffersDirectory()) {
+ IndexWriter iw = new IndexWriter(directory, config);
+
+ int seq = 0;
+ for (Map<String, String[]> fields : docs) {
+ Document doc = new Document();
+ doc.add(new StringField(FLD_ID, Integer.toString(seq++), Field.Store.YES));
+ for (Map.Entry<String, String[]> field : fields.entrySet()) {
+ for (String value : field.getValue()) {
+ doc.add(toField(field.getKey(), value));
+ }
+ }
+ iw.addDocument(doc);
+ if (RandomizedTest.randomBoolean()) {
+ iw.commit();
+ }
+ }
+ iw.flush();
+
+ try (DirectoryReader reader = DirectoryReader.open(iw)) {
+ block.accept(reader);
+ }
+ }
+ }
+
+ private IndexableField toField(String name, String value) {
+ switch (name) {
+ case FLD_TEXT_NOPOS:
+ return new Field(name, value, TYPE_STORED_NO_POSITIONS);
+ case FLD_TEXT_POS:
+ case FLD_TEXT_SYNONYMS_POS:
+ return new TextField(name, value, Field.Store.YES);
+ case FLD_TEXT_POS_OFFS:
+ case FLD_TEXT_POS_OFFS1:
+ case FLD_TEXT_POS_OFFS2:
+ case FLD_TEXT_SYNONYMS_POS_OFFS:
+ return new Field(name, value, TYPE_STORED_WITH_OFFSETS);
+ default:
+ throw new AssertionError("Don't know how to handle this field: " + name);
+ }
+ }
+
+ private static String fmt(String string, Object... args) {
+ return String.format(Locale.ROOT, string, args);
+ }
+}
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/MissingAnalyzer.java b/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/MissingAnalyzer.java
new file mode 100644
index 00000000000..fbe340f7aae
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/MissingAnalyzer.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+import org.apache.lucene.analysis.Analyzer;
+
+import java.io.Reader;
+
+/** An {@link Analyzer} that throws a runtime exception when used for anything. */
+public final class MissingAnalyzer extends Analyzer {
+ @Override
+ protected Reader initReader(String fieldName, Reader reader) {
+ throw new RuntimeException("Field must have an explicit Analyzer: " + fieldName);
+ }
+
+ @Override
+ protected TokenStreamComponents createComponents(String fieldName) {
+ throw new RuntimeException("Field must have an explicit Analyzer: " + fieldName);
+ }
+
+ @Override
+ public int getOffsetGap(String fieldName) {
+ return 0;
+ }
+}
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/SimpleHighlightFormatter.java b/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/SimpleHighlightFormatter.java
new file mode 100644
index 00000000000..7d8c7690fec
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/matchhighlight/SimpleHighlightFormatter.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.matchhighlight;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.document.Document;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A simple ASCII hit range highlighter for tests.
+ */
+final class SimpleHighlightFormatter {
+ private final Analyzer analyzer;
+ private final PassageFormatter passageFormatter;
+ private final PassageSelector selector;
+
+ private int maxPassageWindow = 160;
+ private int maxPassages = 10;
+
+ public SimpleHighlightFormatter(Analyzer analyzer) {
+ this.passageFormatter = new PassageFormatter("...", ">", "<");
+ this.selector = new PassageSelector();
+ this.analyzer = analyzer;
+ }
+
+ public Map<String, List<String>> apply(Document document, Map<String, List<OffsetRange>> fieldHighlights) {
+ ArrayList<OffsetRange> valueRanges = new ArrayList<>();
+ Map<String, List<String>> fieldSnippets = new LinkedHashMap<>();
+
+ fieldHighlights.forEach(
+ (field, matchRanges) -> {
+ int offsetGap = analyzer.getOffsetGap(field);
+
+ String[] values = document.getValues(field);
+ String value;
+ if (values.length == 1) {
+ value = values[0];
+ } else {
+ String fieldGapPadding = " ".repeat(offsetGap);
+ value = String.join(fieldGapPadding, values);
+ }
+
+ // Create permitted range windows for passages so that they don't cross
+ // multi-value boundary.
+ valueRanges.clear();
+ int offset = 0;
+ for (CharSequence v : values) {
+ valueRanges.add(new OffsetRange(offset, offset + v.length()));
+ offset += v.length();
+ offset += offsetGap;
+ }
+
+ List<Passage> passages =
+ selector.pickBest(value, matchRanges, maxPassageWindow, maxPassages, valueRanges);
+
+ fieldSnippets.put(field, passageFormatter.format(value, passages, valueRanges));
+ });
+
+ return fieldSnippets;
+ }
+}
diff --git a/lucene/licenses/assertj-core-3.14.0.jar.sha1 b/lucene/licenses/assertj-core-3.14.0.jar.sha1
new file mode 100644
index 00000000000..7c54f4cbdb0
--- /dev/null
+++ b/lucene/licenses/assertj-core-3.14.0.jar.sha1
@@ -0,0 +1 @@
+3b7b0fcac821f3d167764e9926573cd64f78f9e9
diff --git a/lucene/licenses/assertj-core-LICENSE-ASL.txt b/lucene/licenses/assertj-core-LICENSE-ASL.txt
new file mode 100644
index 00000000000..d6456956733
--- /dev/null
+++ b/lucene/licenses/assertj-core-LICENSE-ASL.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
diff --git a/lucene/licenses/assertj-core-NOTICE.txt b/lucene/licenses/assertj-core-NOTICE.txt
new file mode 100644
index 00000000000..e69de29bb2d
diff --git a/versions.lock b/versions.lock
index eea9b9cda3d..86e2cc8f92c 100644
--- a/versions.lock
+++ b/versions.lock
@@ -227,6 +227,7 @@ org.apache.kerby:kerb-server:1.0.1 (1 constraints: 0405f135)
org.apache.kerby:kerb-simplekdc:1.0.1 (1 constraints: 0405f135)
org.apache.kerby:kerby-kdc:1.0.1 (1 constraints: 0405f135)
org.apache.logging.log4j:log4j-1.2-api:2.13.2 (1 constraints: 3a053a3b)
+org.assertj:assertj-core:3.14.0 (1 constraints: 3a05413b)
org.hsqldb:hsqldb:2.4.0 (1 constraints: 08050136)
org.locationtech.jts:jts-core:1.15.0 (1 constraints: 3905383b)
org.mockito:mockito-core:2.23.4 (1 constraints: 3d05403b)
diff --git a/versions.props b/versions.props
index 35f9bbf7982..8b775b074c6 100644
--- a/versions.props
+++ b/versions.props
@@ -77,6 +77,7 @@ org.apache.xmlbeans:xmlbeans=3.1.0
org.apache.zookeeper:*=3.6.1
org.asciidoctor:asciidoctorj=1.6.2
org.aspectj:aspectjrt=1.8.0
+org.assertj:*=3.14.0
org.bitbucket.b_c:jose4j=0.6.5
org.bouncycastle:*=1.64
org.brotli:dec=0.1.2