diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 79b984a4ed6..9016a503511 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -134,6 +134,15 @@ New Features soft deletes if the reader is opened form a directory. (Simon Willnauer, Mike McCandless, Uwe Schindler, Adrien Grand) +* LUCENE-8229: Add a method Weight.matches(LeafReaderContext, doc) that returns + an iterator over matching positions for a given query and document. This + allows exact hit extraction and will enable implementation of accurate + highlighters. (Alan Woodward, Adrien Grand, David Smiley) + +* LUCENE-8246: Allow to customize the number of deletes a merge claims. This + helps merge policies in the soft-delete case to correctly implement retention + policies without triggering uncessary merges. (Simon Willnauer, Mike McCandless) + Bug Fixes * LUCENE-8234: Fixed bug in how spatial relationship is computed for diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index 8ba460d70b5..e973b91f47e 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -5221,4 +5221,24 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { return false; } + + /** + * Returns the number of deletes a merge would claim back if the given segment is merged. + * @see MergePolicy#numDeletesToMerge(SegmentCommitInfo, int, org.apache.lucene.util.IOSupplier) + * @param info the segment to get the number of deletes for + * @lucene.experimental + */ + public final int numDeletesToMerge(SegmentCommitInfo info) throws IOException { + MergePolicy mergePolicy = config.getMergePolicy(); + final ReadersAndUpdates rld = readerPool.get(info, false); + int numDeletesToMerge; + if (rld != null) { + numDeletesToMerge = rld.numDeletesToMerge(mergePolicy); + } else { + numDeletesToMerge = mergePolicy.numDeletesToMerge(info, 0, null); + } + assert numDeletesToMerge <= info.info.maxDoc() : + "numDeletesToMerge: " + numDeletesToMerge + " > maxDoc: " + info.info.maxDoc(); + return numDeletesToMerge; + } } diff --git a/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java index 78025634b4a..134f6ff848f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java @@ -150,7 +150,7 @@ public abstract class LogMergePolicy extends MergePolicy { * #setCalibrateSizeByDeletes} is set. */ protected long sizeDocs(SegmentCommitInfo info, IndexWriter writer) throws IOException { if (calibrateSizeByDeletes) { - int delCount = writer.numDeletedDocs(info); + int delCount = writer.numDeletesToMerge(info); assert delCount <= info.info.maxDoc(); return (info.info.maxDoc() - (long)delCount); } else { @@ -388,7 +388,7 @@ public abstract class LogMergePolicy extends MergePolicy { assert writer != null; for(int i=0;i 0) { if (verbose(writer)) { message(" segment " + info.info.name + " has deletions", writer); diff --git a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java index c0d9748b6fd..8212c4a3f2c 100644 --- a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java @@ -30,8 +30,10 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.BooleanSupplier; import java.util.stream.Collectors; +import org.apache.lucene.document.Field; import org.apache.lucene.store.Directory; import org.apache.lucene.store.MergeInfo; +import org.apache.lucene.util.IOSupplier; /** *

Expert: a MergePolicy determines the sequence of @@ -310,7 +312,7 @@ public abstract class MergePolicy { * input total size. This is only set once the merge is * initialized by IndexWriter. */ - public long totalBytesSize() throws IOException { + public long totalBytesSize() { return totalMergeBytes; } @@ -318,7 +320,7 @@ public abstract class MergePolicy { * Returns the total number of documents that are included with this merge. * Note that this does not indicate the number of documents after the merge. * */ - public int totalNumDocs() throws IOException { + public int totalNumDocs() { int total = 0; for (SegmentCommitInfo info : segments) { total += info.info.maxDoc(); @@ -551,7 +553,7 @@ public abstract class MergePolicy { * non-deleted documents is set. */ protected long size(SegmentCommitInfo info, IndexWriter writer) throws IOException { long byteSize = info.sizeInBytes(); - int delCount = writer.numDeletedDocs(info); + int delCount = writer.numDeletesToMerge(info); double delRatio = info.info.maxDoc() <= 0 ? 0.0f : (float) delCount / (float) info.info.maxDoc(); assert delRatio <= 1.0; return (info.info.maxDoc() <= 0 ? byteSize : (long) (byteSize * (1.0 - delRatio))); @@ -562,7 +564,7 @@ public abstract class MergePolicy { * writer, and matches the current compound file setting */ protected final boolean isMerged(SegmentInfos infos, SegmentCommitInfo info, IndexWriter writer) throws IOException { assert writer != null; - boolean hasDeletions = writer.numDeletedDocs(info) > 0; + boolean hasDeletions = writer.numDeletesToMerge(info) > 0; return !hasDeletions && info.info.dir == writer.getDirectory() && useCompoundFile(infos, info, writer) == info.info.getUseCompoundFile(); @@ -612,4 +614,21 @@ public abstract class MergePolicy { public boolean keepFullyDeletedSegment(CodecReader reader) throws IOException { return false; } + + /** + * Returns the number of deletes that a merge would claim on the given segment. This method will by default return + * the sum of the del count on disk and the pending delete count. Yet, subclasses that wrap merge readers + * might modify this to reflect deletes that are carried over to the target segment in the case of soft deletes. + * + * Soft deletes all deletes to survive across merges in order to control when the soft-deleted data is claimed. + * @see IndexWriter#softUpdateDocument(Term, Iterable, Field...) + * @see IndexWriterConfig#setSoftDeletesField(String) + * @param info the segment info that identifies the segment + * @param pendingDeleteCount the number of pending deletes for this segment + * @param readerSupplier a supplier that allows to obtain a {@link CodecReader} for this segment + */ + public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, + IOSupplier readerSupplier) throws IOException { + return info.getDelCount() + pendingDeleteCount; + } } diff --git a/lucene/core/src/java/org/apache/lucene/index/MergePolicyWrapper.java b/lucene/core/src/java/org/apache/lucene/index/MergePolicyWrapper.java index 606f3c2bd45..c7124add4a4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/MergePolicyWrapper.java +++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicyWrapper.java @@ -19,6 +19,8 @@ package org.apache.lucene.index; import java.io.IOException; import java.util.Map; +import org.apache.lucene.util.IOSupplier; + /** * A wrapper for {@link MergePolicy} instances. * @@ -90,4 +92,9 @@ public class MergePolicyWrapper extends MergePolicy { public boolean keepFullyDeletedSegment(CodecReader reader) throws IOException { return in.keepFullyDeletedSegment(reader); } + + @Override + public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, IOSupplier readerSupplier) throws IOException { + return in.numDeletesToMerge(info, pendingDeleteCount, readerSupplier); + } } diff --git a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java index 4387f250513..08f900a32b8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java @@ -20,6 +20,8 @@ package org.apache.lucene.index; import java.io.IOException; import java.util.Map; +import org.apache.lucene.util.IOSupplier; + /** * A {@link MergePolicy} which never returns merges to execute. Use it if you * want to prevent segment merges. @@ -73,6 +75,11 @@ public final class NoMergePolicy extends MergePolicy { return super.keepFullyDeletedSegment(reader); } + @Override + public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, IOSupplier readerSupplier) throws IOException { + return super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier); + } + @Override public String toString() { return "NoMergePolicy"; diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java index 3e06acad173..4851d45ccc4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java @@ -41,6 +41,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.TrackingDirectoryWrapper; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.InfoStream; @@ -87,6 +88,10 @@ final class ReadersAndUpdates { final AtomicLong ramBytesUsed = new AtomicLong(); + // if set to true the pending deletes must be marked as shared next time the reader is + // returned from #getReader() + private boolean liveDocsSharedPending = false; + ReadersAndUpdates(int indexCreatedVersionMajor, SegmentCommitInfo info, PendingDeletes pendingDeletes) { this.info = info; @@ -196,12 +201,15 @@ final class ReadersAndUpdates { // We steal returned ref: reader = new SegmentReader(info, indexCreatedVersionMajor, context); pendingDeletes.onNewReader(reader, info); + } else if (liveDocsSharedPending) { + markAsShared(); } + // Ref for caller reader.incRef(); return reader; } - + public synchronized void release(SegmentReader sr) throws IOException { assert info == sr.getSegmentInfo(); sr.decRef(); @@ -221,6 +229,7 @@ final class ReadersAndUpdates { } finally { reader = null; } + liveDocsSharedPending = false; } decRef(); @@ -237,7 +246,7 @@ final class ReadersAndUpdates { } // force new liveDocs Bits liveDocs = pendingDeletes.getLiveDocs(); - pendingDeletes.liveDocsShared(); + markAsShared(); if (liveDocs != null) { return new SegmentReader(reader.getSegmentInfo(), reader, liveDocs, info.info.maxDoc() - info.getDelCount() - pendingDeletes.numPendingDeletes()); @@ -249,6 +258,22 @@ final class ReadersAndUpdates { } } + synchronized int numDeletesToMerge(MergePolicy policy) throws IOException { + IOSupplier readerSupplier = () -> { + if (this.reader == null) { + // get a reader and dec the ref right away we just make sure we have a reader + getReader(IOContext.READ).decRef(); + } + if (reader.getLiveDocs() != pendingDeletes.getLiveDocs() + || reader.numDeletedDocs() != info.getDelCount() - pendingDeletes.numPendingDeletes()) { + // we have a reader but its live-docs are out of sync. let's create a temporary one that we never share + swapNewReaderWithLatestLiveDocs(); + } + return reader; + }; + return policy.numDeletesToMerge(info, pendingDeletes.numPendingDeletes(), readerSupplier); + } + public synchronized Bits getLiveDocs() { return pendingDeletes.getLiveDocs(); @@ -676,18 +701,7 @@ final class ReadersAndUpdates { // if there is a reader open, reopen it to reflect the updates if (reader != null) { - SegmentReader newReader = new SegmentReader(info, reader, pendingDeletes.getLiveDocs(), info.info.maxDoc() - info.getDelCount() - pendingDeletes.numPendingDeletes()); - boolean success2 = false; - try { - pendingDeletes.onNewReader(newReader, info); - reader.decRef(); - reader = newReader; - success2 = true; - } finally { - if (success2 == false) { - newReader.decRef(); - } - } + swapNewReaderWithLatestLiveDocs(); } // writing field updates succeeded @@ -713,6 +727,28 @@ final class ReadersAndUpdates { return true; } + private SegmentReader createNewReaderWithLatestLiveDocs(SegmentReader reader) throws IOException { + assert reader != null; + SegmentReader newReader = new SegmentReader(info, reader, pendingDeletes.getLiveDocs(), + info.info.maxDoc() - info.getDelCount() - pendingDeletes.numPendingDeletes()); + boolean success2 = false; + try { + pendingDeletes.onNewReader(newReader, info); + reader.decRef(); + success2 = true; + } finally { + if (success2 == false) { + newReader.decRef(); + } + } + return newReader; + } + + private void swapNewReaderWithLatestLiveDocs() throws IOException { + reader = createNewReaderWithLatestLiveDocs(reader); + liveDocsSharedPending = true; + } + synchronized public void setIsMerging() { // This ensures any newly resolved doc value updates while we are merging are // saved for re-applying after this segment is done merging: @@ -743,26 +779,11 @@ final class ReadersAndUpdates { if (delCount != reader.numDeletedDocs()) { // beware of zombies: assert delCount > reader.numDeletedDocs(): "delCount=" + delCount + " reader.numDeletedDocs()=" + reader.numDeletedDocs(); - Bits liveDocs = pendingDeletes.getLiveDocs(); - assert liveDocs != null; - - // Create a new reader with the latest live docs: - SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - delCount); - boolean success = false; - try { - reader.decRef(); - pendingDeletes.onNewReader(newReader, info); - success = true; - } finally { - if (success == false) { - newReader.close(); - } - } - reader = newReader; + assert pendingDeletes.getLiveDocs() != null; + reader = createNewReaderWithLatestLiveDocs(reader); } - pendingDeletes.liveDocsShared(); - + markAsShared(); assert verifyDocCounts(); return reader; @@ -794,5 +815,11 @@ final class ReadersAndUpdates { public synchronized boolean isFullyDeleted() { return pendingDeletes.isFullyDeleted(); } + + private final void markAsShared() { + assert Thread.holdsLock(this); + liveDocsSharedPending = false; + pendingDeletes.liveDocsShared(); // this is not costly we can just call it even if it's already marked as shared + } } diff --git a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java index b0887557bed..51578f98729 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/SoftDeletesRetentionMergePolicy.java @@ -33,6 +33,7 @@ import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; import org.apache.lucene.util.Bits; import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.IOSupplier; /** * This {@link MergePolicy} allows to carry over soft deleted documents across merges. The policy wraps @@ -167,4 +168,30 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge return numDocs; } }; - }} + } + + @Override + public int numDeletesToMerge(SegmentCommitInfo info, int pendingDeleteCount, IOSupplier readerSupplier) throws IOException { + int numDeletesToMerge = super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier); + if (numDeletesToMerge != 0) { + final CodecReader reader = readerSupplier.get(); + if (reader.getLiveDocs() != null) { + Scorer scorer = getScorer(field, retentionQuerySupplier.get(), wrapLiveDocs(reader, null, reader.maxDoc())); + if (scorer != null) { + DocIdSetIterator iterator = scorer.iterator(); + Bits liveDocs = reader.getLiveDocs(); + int numDeletedDocs = reader.numDeletedDocs(); + while (iterator.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { + if (liveDocs.get(iterator.docID()) == false) { + numDeletedDocs--; + } + } + return numDeletedDocs; + } + } + } + assert numDeletesToMerge >= 0 : "numDeletesToMerge: " + numDeletesToMerge; + assert numDeletesToMerge <= info.info.maxDoc() : "numDeletesToMerge: " + numDeletesToMerge + " maxDoc:" + info.info.maxDoc(); + return numDeletesToMerge; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java index cdfdea6e5af..bdd3a8a6ac4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java @@ -597,7 +597,7 @@ public class TieredMergePolicy extends MergePolicy { final List eligible = new ArrayList<>(); final Set merging = writer.getMergingSegments(); for(SegmentCommitInfo info : infos) { - double pctDeletes = 100.*((double) writer.numDeletedDocs(info))/info.info.maxDoc(); + double pctDeletes = 100.*((double) writer.numDeletesToMerge(info))/info.info.maxDoc(); if (pctDeletes > forceMergeDeletesPctAllowed && !merging.contains(info)) { eligible.add(info); } diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java index fffdd09093f..1fddd7c3370 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java @@ -119,6 +119,41 @@ final class BooleanWeight extends Weight { } } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + final int minShouldMatch = query.getMinimumNumberShouldMatch(); + List matches = new ArrayList<>(); + int shouldMatchCount = 0; + Iterator wIt = weights.iterator(); + Iterator cIt = query.clauses().iterator(); + while (wIt.hasNext()) { + Weight w = wIt.next(); + BooleanClause bc = cIt.next(); + Matches m = w.matches(context, doc); + if (bc.isProhibited()) { + if (m != null) { + return null; + } + } + if (bc.isRequired()) { + if (m == null) { + return null; + } + matches.add(m); + } + if (bc.getOccur() == Occur.SHOULD) { + if (m != null) { + matches.add(m); + shouldMatchCount++; + } + } + } + if (shouldMatchCount < minShouldMatch) { + return null; + } + return Matches.fromSubMatches(matches); + } + static BulkScorer disableScoring(final BulkScorer scorer) { return new BulkScorer() { diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java index 464cde6a45f..d696c215f1b 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java @@ -158,6 +158,11 @@ public final class ConstantScoreQuery extends Query { }; } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + return innerWeight.matches(context, doc); + } + @Override public Scorer scorer(LeafReaderContext context) throws IOException { ScorerSupplier scorerSupplier = scorerSupplier(context); diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java new file mode 100644 index 00000000000..37770d2f401 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.lucene.search; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; +import org.apache.lucene.util.PriorityQueue; + +/** + * A {@link MatchesIterator} that combines matches from a set of sub-iterators + * + * Matches are sorted by their start positions, and then by their end positions, so that + * prefixes sort first. Matches may overlap, or be duplicated if they appear in more + * than one of the sub-iterators. + */ +final class DisjunctionMatchesIterator implements MatchesIterator { + + /** + * Create a {@link DisjunctionMatchesIterator} over a list of terms + * + * Only terms that have at least one match in the given document will be included + */ + static MatchesIterator fromTerms(LeafReaderContext context, int doc, String field, List terms) throws IOException { + Objects.requireNonNull(field); + for (Term term : terms) { + if (Objects.equals(field, term.field()) == false) { + throw new IllegalArgumentException("Tried to generate iterator from terms in multiple fields: expected [" + field + "] but got [" + term.field() + "]"); + } + } + return fromTermsEnum(context, doc, field, asBytesRefIterator(terms)); + } + + private static BytesRefIterator asBytesRefIterator(List terms) { + return new BytesRefIterator() { + int i = 0; + @Override + public BytesRef next() { + if (i >= terms.size()) + return null; + return terms.get(i++).bytes(); + } + }; + } + + /** + * Create a {@link DisjunctionMatchesIterator} over a list of terms extracted from a {@link BytesRefIterator} + * + * Only terms that have at least one match in the given document will be included + */ + static MatchesIterator fromTermsEnum(LeafReaderContext context, int doc, String field, BytesRefIterator terms) throws IOException { + Objects.requireNonNull(field); + List mis = new ArrayList<>(); + Terms t = context.reader().terms(field); + if (t == null) + return null; + TermsEnum te = t.iterator(); + PostingsEnum reuse = null; + for (BytesRef term = terms.next(); term != null; term = terms.next()) { + if (te.seekExact(term)) { + PostingsEnum pe = te.postings(reuse, PostingsEnum.OFFSETS); + if (pe.advance(doc) == doc) { + // TODO do we want to use the copied term here, or instead create a label that associates all of the TMIs with a single term? + mis.add(new TermMatchesIterator(BytesRef.deepCopyOf(term), pe)); + reuse = null; + } + else { + reuse = pe; + } + } + } + if (mis.size() == 0) + return null; + if (mis.size() == 1) + return mis.get(0); + return new DisjunctionMatchesIterator(mis); + } + + static MatchesIterator fromSubIterators(List mis) throws IOException { + if (mis.size() == 0) + return null; + if (mis.size() == 1) + return mis.get(0); + return new DisjunctionMatchesIterator(mis); + } + + private final PriorityQueue queue; + + private boolean started = false; + + private DisjunctionMatchesIterator(List matches) throws IOException { + queue = new PriorityQueue(matches.size()){ + @Override + protected boolean lessThan(MatchesIterator a, MatchesIterator b) { + return a.startPosition() < b.startPosition() || + (a.startPosition() == b.startPosition() && a.endPosition() < b.endPosition()) || + (a.startPosition() == b.startPosition() && a.endPosition() == b.endPosition() && a.term().compareTo(b.term()) < 0); + } + }; + for (MatchesIterator mi : matches) { + if (mi.next()) { + queue.add(mi); + } + } + } + + @Override + public boolean next() throws IOException { + if (started == false) { + return started = true; + } + if (queue.top().next() == false) { + queue.pop(); + } + if (queue.size() > 0) { + queue.updateTop(); + return true; + } + return false; + } + + @Override + public int startPosition() { + return queue.top().startPosition(); + } + + @Override + public int endPosition() { + return queue.top().endPosition(); + } + + @Override + public int startOffset() throws IOException { + return queue.top().startOffset(); + } + + @Override + public int endOffset() throws IOException { + return queue.top().endOffset(); + } + + @Override + public BytesRef term() { + return queue.top().term(); + } + +} diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java index 1e67cb15046..5b99c3ce24b 100644 --- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java @@ -118,6 +118,18 @@ public final class DisjunctionMaxQuery extends Query implements Iterable } } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + List mis = new ArrayList<>(); + for (Weight weight : weights) { + Matches mi = weight.matches(context, doc); + if (mi != null) { + mis.add(mi); + } + } + return Matches.fromSubMatches(mis); + } + /** Create the scorer used to score our associated DisjunctionMaxQuery */ @Override public Scorer scorer(LeafReaderContext context) throws IOException { diff --git a/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java b/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java index 5d591983fab..99720b7e586 100644 --- a/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java +++ b/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java @@ -74,11 +74,16 @@ public final class DocValuesRewriteMethod extends MultiTermQuery.RewriteMethod { @Override public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { return new ConstantScoreWeight(this, boost) { + @Override - public Scorer scorer(LeafReaderContext context) throws IOException { + public Matches matches(LeafReaderContext context, int doc) throws IOException { final SortedSetDocValues fcsi = DocValues.getSortedSet(context.reader(), query.field); - TermsEnum termsEnum = query.getTermsEnum(new Terms() { - + return Matches.forField(query.field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, query.field, getTermsEnum(fcsi))); + } + + private TermsEnum getTermsEnum(SortedSetDocValues fcsi) throws IOException { + return query.getTermsEnum(new Terms() { + @Override public TermsEnum iterator() throws IOException { return fcsi.termsEnum(); @@ -118,13 +123,18 @@ public final class DocValuesRewriteMethod extends MultiTermQuery.RewriteMethod { public boolean hasPositions() { return false; } - + @Override public boolean hasPayloads() { return false; } }); - + } + + @Override + public Scorer scorer(LeafReaderContext context) throws IOException { + final SortedSetDocValues fcsi = DocValues.getSortedSet(context.reader(), query.field); + TermsEnum termsEnum = getTermsEnum(fcsi); assert termsEnum != null; if (termsEnum.next() == null) { // no matching terms diff --git a/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java b/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java index 925c9534f89..8a2b57b41ea 100644 --- a/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/FilterWeight.java @@ -75,4 +75,8 @@ public abstract class FilterWeight extends Weight { return in.scorer(context); } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + return in.matches(context, doc); + } } diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java b/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java index f89924d1605..d69421ec4d6 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java @@ -119,6 +119,12 @@ public final class IndexOrDocValuesQuery extends Query { indexWeight.extractTerms(terms); } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + // We need to check a single doc, so the dv query should perform better + return dvWeight.matches(context, doc); + } + @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { // We need to check a single doc, so the dv query should perform better diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java index 72239e876cb..9391afdbf96 100644 --- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java @@ -678,6 +678,11 @@ public class LRUQueryCache implements QueryCache, Accountable { in.extractTerms(terms); } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + return in.matches(context, doc); + } + private boolean cacheEntryHasReasonableWorstCaseSize(int maxDoc) { // The worst-case (dense) is a bit set which needs one bit per document final long worstCaseRamUsage = maxDoc / 8; diff --git a/lucene/core/src/java/org/apache/lucene/search/Matches.java b/lucene/core/src/java/org/apache/lucene/search/Matches.java new file mode 100644 index 00000000000..367056373b7 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/Matches.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.lucene.search; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Reports the positions and optionally offsets of all matching terms in a query + * for a single document + * + * To obtain a {@link MatchesIterator} for a particular field, call {@link #getMatches(String)}. + * Note that you can call {@link #getMatches(String)} multiple times to retrieve new + * iterators, but it is not thread-safe. + */ +public interface Matches extends Iterable { + + /** + * Returns a {@link MatchesIterator} over the matches for a single field, + * or {@code null} if there are no matches in that field. + */ + MatchesIterator getMatches(String field) throws IOException; + + /** + * Indicates a match with no term positions, for example on a Point or DocValues field, + * or a field indexed as docs and freqs only + */ + Matches MATCH_WITH_NO_TERMS = new Matches() { + @Override + public Iterator iterator() { + return Collections.emptyIterator(); + } + + @Override + public MatchesIterator getMatches(String field) { + return null; + } + }; + + /** + * Amalgamate a collection of {@link Matches} into a single object + */ + static Matches fromSubMatches(List subMatches) { + if (subMatches == null || subMatches.size() == 0) { + return null; + } + List sm = subMatches.stream().filter(m -> m != MATCH_WITH_NO_TERMS).collect(Collectors.toList()); + if (sm.size() == 0) { + return MATCH_WITH_NO_TERMS; + } + if (sm.size() == 1) { + return sm.get(0); + } + Set fields = new HashSet<>(); + for (Matches m : sm) { + for (String field : m) { + fields.add(field); + } + } + return new Matches() { + @Override + public MatchesIterator getMatches(String field) throws IOException { + List subIterators = new ArrayList<>(); + for (Matches m : sm) { + MatchesIterator it = m.getMatches(field); + if (it != null) { + subIterators.add(it); + } + } + return DisjunctionMatchesIterator.fromSubIterators(subIterators); + } + + @Override + public Iterator iterator() { + return fields.iterator(); + } + }; + } + + /** + * A functional interface that supplies a {@link MatchesIterator} + */ + @FunctionalInterface + interface MatchesIteratorSupplier { + /** Return a new {@link MatchesIterator} */ + MatchesIterator get() throws IOException; + } + + /** + * Create a Matches for a single field + */ + static Matches forField(String field, MatchesIteratorSupplier mis) throws IOException { + + // The indirection here, using a Supplier object rather than a MatchesIterator + // directly, is to allow for multiple calls to Matches.getMatches() to return + // new iterators. We still need to call MatchesIteratorSupplier.get() eagerly + // to work out if we have a hit or not. + + MatchesIterator mi = mis.get(); + if (mi == null) { + return null; + } + return new Matches() { + boolean cached = true; + @Override + public MatchesIterator getMatches(String f) throws IOException { + if (Objects.equals(field, f) == false) { + return null; + } + if (cached == false) { + return mis.get(); + } + cached = false; + return mi; + } + + @Override + public Iterator iterator() { + return Collections.singleton(field).iterator(); + } + }; + } + +} diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java new file mode 100644 index 00000000000..b874263915b --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.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; + +import java.io.IOException; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.util.BytesRef; + +/** + * An iterator over match positions (and optionally offsets) for a single document and field + * + * To iterate over the matches, call {@link #next()} until it returns {@code false}, retrieving + * positions and/or offsets after each call. You should not call the position or offset methods + * before {@link #next()} has been called, or after {@link #next()} has returned {@code false}. + * + * Matches are ordered by start position, and then by end position. Match intervals may overlap. + * + * @see Weight#matches(LeafReaderContext, int) + */ +public interface MatchesIterator { + + /** + * Advance the iterator to the next match position + * @return {@code true} if matches have not been exhausted + */ + boolean next() throws IOException; + + /** + * The start position of the current match + * + * Should only be called after {@link #next()} has returned {@code true} + */ + int startPosition(); + + /** + * The end position of the current match + * + * Should only be called after {@link #next()} has returned {@code true} + */ + int endPosition(); + + /** + * The starting offset of the current match, or {@code -1} if offsets are not available + * + * Should only be called after {@link #next()} has returned {@code true} + */ + int startOffset() throws IOException; + + /** + * The ending offset of the current match, or {@code -1} if offsets are not available + * + * Should only be called after {@link #next()} has returned {@code true} + */ + int endOffset() throws IOException; + + /** + * The underlying term of the current match + * + * Should only be called after {@link #next()} has returned {@code true} + */ + BytesRef term(); + +} diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java b/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java index 3a46b96411c..22997762119 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java +++ b/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java @@ -25,8 +25,8 @@ import java.util.Objects; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermStates; import org.apache.lucene.index.TermState; +import org.apache.lucene.index.TermStates; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.BooleanClause.Occur; @@ -202,6 +202,18 @@ final class MultiTermQueryConstantScoreWrapper extends } } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + final Terms terms = context.reader().terms(query.field); + if (terms == null) { + return null; + } + if (terms.hasPositions() == false) { + return super.matches(context, doc); + } + return Matches.forField(query.field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, query.field, query.getTermsEnum(terms))); + } + @Override public Scorer scorer(LeafReaderContext context) throws IOException { final WeightOrDocIdSet weightOrBitSet = rewrite(context); diff --git a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java index 2a7c450805d..a364c9aacb0 100644 --- a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java @@ -30,8 +30,9 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermStates; import org.apache.lucene.index.TermState; +import org.apache.lucene.index.TermStates; +import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.util.BytesRef; @@ -159,6 +160,16 @@ public final class SynonymQuery extends Query { } } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + String field = terms[0].field(); + Terms terms = context.reader().terms(field); + if (terms == null || terms.hasPositions() == false) { + return super.matches(context, doc); + } + return Matches.forField(field, () -> DisjunctionMatchesIterator.fromTerms(context, doc, field, Arrays.asList(SynonymQuery.this.terms))); + } + @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { Scorer scorer = scorer(context); diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java index a8bf5b0679c..7145a83db30 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java @@ -220,6 +220,15 @@ public class TermInSetQuery extends Query implements Accountable { // order to protect highlighters } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + Terms terms = context.reader().terms(field); + if (terms == null || terms.hasPositions() == false) { + return super.matches(context, doc); + } + return Matches.forField(field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, field, termData.iterator())); + } + /** * On the given leaf context, try to either rewrite to a disjunction if * there are few matching terms, or build a bitset containing matching docs. diff --git a/lucene/core/src/java/org/apache/lucene/search/TermMatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/TermMatchesIterator.java new file mode 100644 index 00000000000..051699651db --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/TermMatchesIterator.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.lucene.search; + +import java.io.IOException; + +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.util.BytesRef; + +/** + * A {@link MatchesIterator} over a single term's postings list + */ +class TermMatchesIterator implements MatchesIterator { + + private int upto; + private int pos; + private final PostingsEnum pe; + private final BytesRef term; + + /** + * Create a new {@link TermMatchesIterator} for the given term and postings list + */ + TermMatchesIterator(BytesRef term, PostingsEnum pe) throws IOException { + this.pe = pe; + this.upto = pe.freq(); + this.term = term; + } + + @Override + public boolean next() throws IOException { + if (upto-- > 0) { + pos = pe.nextPosition(); + return true; + } + return false; + } + + @Override + public int startPosition() { + return pos; + } + + @Override + public int endPosition() { + return pos; + } + + @Override + public int startOffset() throws IOException { + return pe.startOffset(); + } + + @Override + public int endOffset() throws IOException { + return pe.endOffset(); + } + + @Override + public BytesRef term() { + return term; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java index f1f44154f55..b86f340f334 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java @@ -25,10 +25,11 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.ReaderUtil; import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermStates; import org.apache.lucene.index.TermState; +import org.apache.lucene.index.TermStates; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.similarities.Similarity; @@ -80,6 +81,24 @@ public class TermQuery extends Query { terms.add(getTerm()); } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + TermsEnum te = getTermsEnum(context); + if (te == null) { + return null; + } + if (context.reader().terms(term.field()).hasPositions() == false) { + return super.matches(context, doc); + } + return Matches.forField(term.field(), () -> { + PostingsEnum pe = te.postings(null, PostingsEnum.OFFSETS); + if (pe.advance(doc) != doc) { + return null; + } + return new TermMatchesIterator(term.bytes(), pe); + }); + } + @Override public String toString() { return "weight(" + TermQuery.this + ")"; diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java index 7853ccf2465..3281b4167ee 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Weight.java +++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java @@ -69,6 +69,35 @@ public abstract class Weight implements SegmentCacheable { */ public abstract void extractTerms(Set terms); + /** + * Returns {@link Matches} for a specific document, or {@code null} if the document + * does not match the parent query + * + * A query match that contains no position information (for example, a Point or + * DocValues query) will return {@link Matches#MATCH_WITH_NO_TERMS} + * + * @param context the reader's context to create the {@link Matches} for + * @param doc the document's id relative to the given context's reader + */ + public Matches matches(LeafReaderContext context, int doc) throws IOException { + Scorer scorer = scorer(context); + if (scorer == null) { + return null; + } + final TwoPhaseIterator twoPhase = scorer.twoPhaseIterator(); + if (twoPhase == null) { + if (scorer.iterator().advance(doc) != doc) { + return null; + } + } + else { + if (twoPhase.approximation().advance(doc) != doc || twoPhase.matches() == false) { + return null; + } + } + return Matches.MATCH_WITH_NO_TERMS; + } + /** * An explanation of the score computation for the named document. * diff --git a/lucene/core/src/java/org/apache/lucene/util/IOSupplier.java b/lucene/core/src/java/org/apache/lucene/util/IOSupplier.java new file mode 100644 index 00000000000..04758278665 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/util/IOSupplier.java @@ -0,0 +1,37 @@ +/* + * 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.util; + +import java.io.IOException; + +/** + * This is a result supplier that is allowed to throw an IOException. + * + * @see java.util.function.Supplier + * @param the suppliers result type. + */ +@FunctionalInterface +public interface IOSupplier{ + + /** + * Gets the result. + * @return the result + * @throws IOException if producing the result throws an {@link IOException} + */ + T get() throws IOException; +} diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java index 3d8ffe3b158..061d006f249 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -37,6 +38,7 @@ import org.apache.lucene.document.StringField; import org.apache.lucene.search.DocValuesFieldExistsQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; @@ -47,6 +49,60 @@ import org.apache.lucene.util.LuceneTestCase; public class TestSoftDeletesRetentionMergePolicy extends LuceneTestCase { + public void testForceMergeFullyDeleted() throws IOException { + Directory dir = newDirectory(); + AtomicBoolean letItGo = new AtomicBoolean(false); + MergePolicy policy = new SoftDeletesRetentionMergePolicy("soft_delete", + () -> letItGo.get() ? new MatchNoDocsQuery() : new MatchAllDocsQuery(), new LogDocMergePolicy()); + IndexWriterConfig indexWriterConfig = newIndexWriterConfig().setMergePolicy(policy) + .setSoftDeletesField("soft_delete"); + IndexWriter writer = new IndexWriter(dir, indexWriterConfig); + + Document doc = new Document(); + doc.add(new StringField("id", "1", Field.Store.YES)); + doc.add(new NumericDocValuesField("soft_delete", 1)); + writer.addDocument(doc); + writer.commit(); + doc = new Document(); + doc.add(new StringField("id", "2", Field.Store.YES)); + doc.add(new NumericDocValuesField("soft_delete", 1)); + writer.addDocument(doc); + DirectoryReader reader = writer.getReader(); + { + assertEquals(2, reader.leaves().size()); + final SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader(); + assertTrue(policy.keepFullyDeletedSegment(segmentReader)); + assertEquals(0, policy.numDeletesToMerge(segmentReader.getSegmentInfo(), 0, () -> segmentReader)); + } + { + SegmentReader segmentReader = (SegmentReader) reader.leaves().get(1).reader(); + assertTrue(policy.keepFullyDeletedSegment(segmentReader)); + assertEquals(0, policy.numDeletesToMerge(segmentReader.getSegmentInfo(), 0, () -> segmentReader)); + writer.forceMerge(1); + reader.close(); + } + reader = writer.getReader(); + { + assertEquals(1, reader.leaves().size()); + SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader(); + assertEquals(2, reader.maxDoc()); + assertTrue(policy.keepFullyDeletedSegment(segmentReader)); + assertEquals(0, policy.numDeletesToMerge(segmentReader.getSegmentInfo(), 0, () -> segmentReader)); + } + writer.forceMerge(1); // make sure we don't merge this + assertNull(DirectoryReader.openIfChanged(reader)); + + writer.forceMergeDeletes(); // make sure we don't merge this + assertNull(DirectoryReader.openIfChanged(reader)); + letItGo.set(true); + writer.forceMergeDeletes(); // make sure we don't merge this + DirectoryReader directoryReader = DirectoryReader.openIfChanged(reader); + assertNotNull(directoryReader); + assertEquals(0, directoryReader.numDeletedDocs()); + assertEquals(0, directoryReader.maxDoc()); + IOUtils.close(directoryReader, reader, writer, dir); + } + public void testKeepFullyDeletedSegments() throws IOException { Directory dir = newDirectory(); IndexWriterConfig indexWriterConfig = newIndexWriterConfig(); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java b/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java new file mode 100644 index 00000000000..3b3dd322516 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java @@ -0,0 +1,440 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.lucene.search; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.ReaderUtil; +import org.apache.lucene.index.Term; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.LuceneTestCase; + +public class TestMatchesIterator extends LuceneTestCase { + + protected IndexSearcher searcher; + protected Directory directory; + protected IndexReader reader; + + private static final String FIELD_WITH_OFFSETS = "field_offsets"; + private static final String FIELD_NO_OFFSETS = "field_no_offsets"; + private static final String FIELD_DOCS_ONLY = "field_docs_only"; + private static final String FIELD_FREQS = "field_freqs"; + + private static final FieldType OFFSETS = new FieldType(TextField.TYPE_STORED); + static { + OFFSETS.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS); + } + + private static final FieldType DOCS = new FieldType(TextField.TYPE_STORED); + static { + DOCS.setIndexOptions(IndexOptions.DOCS); + } + + private static final FieldType DOCS_AND_FREQS = new FieldType(TextField.TYPE_STORED); + static { + DOCS_AND_FREQS.setIndexOptions(IndexOptions.DOCS_AND_FREQS); + } + + @Override + public void tearDown() throws Exception { + reader.close(); + directory.close(); + super.tearDown(); + } + + @Override + public void setUp() throws Exception { + super.setUp(); + directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), directory, + newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy())); + for (int i = 0; i < docFields.length; i++) { + Document doc = new Document(); + doc.add(newField(FIELD_WITH_OFFSETS, docFields[i], OFFSETS)); + doc.add(newField(FIELD_NO_OFFSETS, docFields[i], TextField.TYPE_STORED)); + doc.add(newField(FIELD_DOCS_ONLY, docFields[i], DOCS)); + doc.add(newField(FIELD_FREQS, docFields[i], DOCS_AND_FREQS)); + doc.add(new NumericDocValuesField("id", i)); + doc.add(newField("id", Integer.toString(i), TextField.TYPE_STORED)); + writer.addDocument(doc); + } + writer.forceMerge(1); + reader = writer.getReader(); + writer.close(); + searcher = newSearcher(getOnlyLeafReader(reader)); + } + + protected String[] docFields = { + "w1 w2 w3 w4 w5", + "w1 w3 w2 w3 zz", + "w1 xx w2 yy w4", + "w1 w2 w1 w4 w2 w3", + "nothing matches this document" + }; + + void checkMatches(Query q, String field, int[][] expected) throws IOException { + Weight w = searcher.createWeight(searcher.rewrite(q), ScoreMode.COMPLETE_NO_SCORES, 1); + for (int i = 0; i < expected.length; i++) { + LeafReaderContext ctx = searcher.leafContexts.get(ReaderUtil.subIndex(expected[i][0], searcher.leafContexts)); + int doc = expected[i][0] - ctx.docBase; + Matches matches = w.matches(ctx, doc); + if (matches == null) { + assertEquals(expected[i].length, 1); + continue; + } + MatchesIterator it = matches.getMatches(field); + if (expected[i].length == 1) { + assertNull(it); + return; + } + checkFieldMatches(it, expected[i]); + checkFieldMatches(matches.getMatches(field), expected[i]); // test multiple calls + } + } + + void checkFieldMatches(MatchesIterator it, int[] expected) throws IOException { + int pos = 1; + while (it.next()) { + //System.out.println(expected[i][pos] + "->" + expected[i][pos + 1] + "[" + expected[i][pos + 2] + "->" + expected[i][pos + 3] + "]"); + assertEquals(expected[pos], it.startPosition()); + assertEquals(expected[pos + 1], it.endPosition()); + assertEquals(expected[pos + 2], it.startOffset()); + assertEquals(expected[pos + 3], it.endOffset()); + pos += 4; + } + assertEquals(expected.length, pos); + } + + void checkNoPositionsMatches(Query q, String field, boolean[] expected) throws IOException { + Weight w = searcher.createWeight(searcher.rewrite(q), ScoreMode.COMPLETE_NO_SCORES, 1); + for (int i = 0; i < expected.length; i++) { + LeafReaderContext ctx = searcher.leafContexts.get(ReaderUtil.subIndex(i, searcher.leafContexts)); + int doc = i - ctx.docBase; + Matches matches = w.matches(ctx, doc); + if (expected[i]) { + MatchesIterator mi = matches.getMatches(field); + assertNull(mi); + } + else { + assertNull(matches); + } + } + } + + void checkTerms(Query q, String field, String[][] expected) throws IOException { + Weight w = searcher.createWeight(searcher.rewrite(q), ScoreMode.COMPLETE_NO_SCORES, 1); + for (int i = 0; i < expected.length; i++) { + LeafReaderContext ctx = searcher.leafContexts.get(ReaderUtil.subIndex(i, searcher.leafContexts)); + int doc = i - ctx.docBase; + Matches matches = w.matches(ctx, doc); + if (matches == null) { + assertEquals(expected[i].length, 0); + continue; + } + MatchesIterator it = matches.getMatches(field); + if (it == null) { + assertEquals(expected[i].length, 0); + continue; + } + int pos = 0; + while (it.next()) { + assertEquals(expected[i][pos], it.term().utf8ToString()); + pos += 1; + } + assertEquals(expected[i].length, pos); + } + } + + public void testTermQuery() throws IOException { + Query q = new TermQuery(new Term(FIELD_WITH_OFFSETS, "w1")); + checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{ + { 0, 0, 0, 0, 2 }, + { 1, 0, 0, 0, 2 }, + { 2, 0, 0, 0, 2 }, + { 3, 0, 0, 0, 2, 2, 2, 6, 8 }, + { 4 } + }); + } + + public void testTermQueryNoStoredOffsets() throws IOException { + Query q = new TermQuery(new Term(FIELD_NO_OFFSETS, "w1")); + checkMatches(q, FIELD_NO_OFFSETS, new int[][]{ + { 0, 0, 0, -1, -1 }, + { 1, 0, 0, -1, -1 }, + { 2, 0, 0, -1, -1 }, + { 3, 0, 0, -1, -1, 2, 2, -1, -1 }, + { 4 } + }); + checkTerms(q, FIELD_NO_OFFSETS, new String[][]{ + { "w1" }, + { "w1" }, + { "w1" }, + { "w1", "w1" }, + {} + }); + } + + public void testTermQueryNoPositions() throws IOException { + for (String field : new String[]{ FIELD_DOCS_ONLY, FIELD_FREQS }) { + Query q = new TermQuery(new Term(field, "w1")); + checkNoPositionsMatches(q, field, new boolean[]{ true, true, true, true, false }); + } + } + + public void testDisjunction() throws IOException { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w1")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.SHOULD) + .build(); + checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{ + { 0, 0, 0, 0, 2, 2, 2, 6, 8 }, + { 1, 0, 0, 0, 2, 1, 1, 3, 5, 3, 3, 9, 11 }, + { 2, 0, 0, 0, 2 }, + { 3, 0, 0, 0, 2, 2, 2, 6, 8, 5, 5, 15, 17 }, + { 4 } + }); + checkTerms(q, FIELD_WITH_OFFSETS, new String[][]{ + { "w1", "w3" }, + { "w1", "w3", "w3" }, + { "w1" }, + { "w1", "w1", "w3" }, + {} + }); + } + + public void testDisjunctionNoPositions() throws IOException { + for (String field : new String[]{ FIELD_DOCS_ONLY, FIELD_FREQS }) { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(field, "w1")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD) + .build(); + checkNoPositionsMatches(q, field, new boolean[]{ true, true, true, true, false }); + } + } + + public void testReqOpt() throws IOException { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w1")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.MUST) + .build(); + checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{ + { 0, 0, 0, 0, 2, 2, 2, 6, 8 }, + { 1, 0, 0, 0, 2, 1, 1, 3, 5, 3, 3, 9, 11 }, + { 2 }, + { 3, 0, 0, 0, 2, 2, 2, 6, 8, 5, 5, 15, 17 }, + { 4 } + }); + } + + public void testReqOptNoPositions() throws IOException { + for (String field : new String[]{ FIELD_DOCS_ONLY, FIELD_FREQS }) { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(field, "w1")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST) + .build(); + checkNoPositionsMatches(q, field, new boolean[]{ true, true, false, true, false }); + } + } + + public void testMinShouldMatch() throws IOException { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.SHOULD) + .add(new BooleanQuery.Builder() + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w1")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w4")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "xx")), BooleanClause.Occur.SHOULD) + .setMinimumNumberShouldMatch(2) + .build(), BooleanClause.Occur.SHOULD) + .build(); + checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{ + { 0, 0, 0, 0, 2, 2, 2, 6, 8, 3, 3, 9, 11 }, + { 1, 1, 1, 3, 5, 3, 3, 9, 11 }, + { 2, 0, 0, 0, 2, 1, 1, 3, 5, 4, 4, 12, 14 }, + { 3, 0, 0, 0, 2, 2, 2, 6, 8, 3, 3, 9, 11, 5, 5, 15, 17 }, + { 4 } + }); + checkTerms(q, FIELD_WITH_OFFSETS, new String[][]{ + { "w1", "w3", "w4" }, + { "w3", "w3" }, + { "w1", "xx", "w4" }, + { "w1", "w1", "w4", "w3" }, + {} + }); + } + + public void testMinShouldMatchNoPositions() throws IOException { + for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD) + .add(new BooleanQuery.Builder() + .add(new TermQuery(new Term(field, "w1")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(field, "w4")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(field, "xx")), BooleanClause.Occur.SHOULD) + .setMinimumNumberShouldMatch(2) + .build(), BooleanClause.Occur.SHOULD) + .build(); + checkNoPositionsMatches(q, field, new boolean[]{ true, true, true, true, false }); + } + } + + public void testExclusion() throws IOException { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "zz")), BooleanClause.Occur.MUST_NOT) + .build(); + checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{ + { 0, 2, 2, 6, 8 }, + { 1 }, + { 2 }, + { 3, 5, 5, 15, 17 }, + { 4 } + }); + } + + public void testExclusionNoPositions() throws IOException { + for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(field, "zz")), BooleanClause.Occur.MUST_NOT) + .build(); + checkNoPositionsMatches(q, field, new boolean[]{ true, false, false, true, false }); + } + } + + public void testConjunction() throws IOException { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.MUST) + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w4")), BooleanClause.Occur.MUST) + .build(); + checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{ + { 0, 2, 2, 6, 8, 3, 3, 9, 11 }, + { 1 }, + { 2 }, + { 3, 3, 3, 9, 11, 5, 5, 15, 17 }, + { 4 } + }); + } + + public void testConjunctionNoPositions() throws IOException { + for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term(field, "w3")), BooleanClause.Occur.MUST) + .add(new TermQuery(new Term(field, "w4")), BooleanClause.Occur.MUST) + .build(); + checkNoPositionsMatches(q, field, new boolean[]{ true, false, false, true, false }); + } + } + + public void testWildcards() throws IOException { + Query q = new PrefixQuery(new Term(FIELD_WITH_OFFSETS, "x")); + checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{ + { 0 }, + { 1 }, + { 2, 1, 1, 3, 5 }, + { 3 }, + { 4 } + }); + checkTerms(q, FIELD_WITH_OFFSETS, new String[][]{ + {}, {}, { "xx" }, {} + }); + + Query rq = new RegexpQuery(new Term(FIELD_WITH_OFFSETS, "w[1-2]")); + checkMatches(rq, FIELD_WITH_OFFSETS, new int[][]{ + { 0, 0, 0, 0, 2, 1, 1, 3, 5 }, + { 1, 0, 0, 0, 2, 2, 2, 6, 8 }, + { 2, 0, 0, 0, 2, 2, 2, 6, 8 }, + { 3, 0, 0, 0, 2, 1, 1, 3, 5, 2, 2, 6, 8, 4, 4, 12, 14 }, + { 4 } + }); + + } + + public void testNoMatchWildcards() throws IOException { + Query nomatch = new PrefixQuery(new Term(FIELD_WITH_OFFSETS, "wibble")); + Matches matches = searcher.createWeight(searcher.rewrite(nomatch), ScoreMode.COMPLETE_NO_SCORES, 1) + .matches(searcher.leafContexts.get(0), 0); + assertNull(matches); + } + + public void testWildcardsNoPositions() throws IOException { + for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) { + Query q = new PrefixQuery(new Term(field, "x")); + checkNoPositionsMatches(q, field, new boolean[]{ false, false, true, false, false }); + } + } + + public void testSynonymQuery() throws IOException { + Query q = new SynonymQuery(new Term(FIELD_WITH_OFFSETS, "w1"), new Term(FIELD_WITH_OFFSETS, "w2")); + checkMatches(q, FIELD_WITH_OFFSETS, new int[][]{ + { 0, 0, 0, 0, 2, 1, 1, 3, 5 }, + { 1, 0, 0, 0, 2, 2, 2, 6, 8 }, + { 2, 0, 0, 0, 2, 2, 2, 6, 8 }, + { 3, 0, 0, 0, 2, 1, 1, 3, 5, 2, 2, 6, 8, 4, 4, 12, 14 }, + { 4 } + }); + } + + public void testSynonymQueryNoPositions() throws IOException { + for (String field : new String[]{ FIELD_FREQS, FIELD_DOCS_ONLY }) { + Query q = new SynonymQuery(new Term(field, "w1"), new Term(field, "w2")); + checkNoPositionsMatches(q, field, new boolean[]{ true, true, true, true, false }); + } + } + + public void testMultipleFields() throws IOException { + Query q = new BooleanQuery.Builder() + .add(new TermQuery(new Term("id", "1")), BooleanClause.Occur.SHOULD) + .add(new TermQuery(new Term(FIELD_WITH_OFFSETS, "w3")), BooleanClause.Occur.MUST) + .build(); + Weight w = searcher.createWeight(searcher.rewrite(q), ScoreMode.COMPLETE, 1); + + LeafReaderContext ctx = searcher.leafContexts.get(ReaderUtil.subIndex(1, searcher.leafContexts)); + Matches m = w.matches(ctx, 1 - ctx.docBase); + assertNotNull(m); + checkFieldMatches(m.getMatches("id"), new int[]{ -1, 0, 0, -1, -1 }); + checkFieldMatches(m.getMatches(FIELD_WITH_OFFSETS), new int[]{ -1, 1, 1, 3, 5, 3, 3, 9, 11 }); + assertNull(m.getMatches("bogus")); + + Set fields = new HashSet<>(); + for (String field : m) { + fields.add(field); + } + assertEquals(2, fields.size()); + assertTrue(fields.contains(FIELD_WITH_OFFSETS)); + assertTrue(fields.contains("id")); + } + + protected String[] doc1Fields = { + "w1 w2 w3 w4 w5", + "w1 w3 w2 w3 zz", + "w1 xx w2 yy w4", + "w1 w2 w1 w4 w2 w3" + }; + +} diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java index 9f2bd905506..3b99ccf3947 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java @@ -28,6 +28,7 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.FilterWeight; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Matches; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.ScorerSupplier; @@ -151,6 +152,28 @@ public class ToParentBlockJoinQuery extends Query { } return Explanation.noMatch("Not a match"); } + + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + // The default implementation would delegate to the joinQuery's Weight, which + // matches on children. We need to match on the parent instead + Scorer scorer = scorer(context); + if (scorer == null) { + return null; + } + final TwoPhaseIterator twoPhase = scorer.twoPhaseIterator(); + if (twoPhase == null) { + if (scorer.iterator().advance(doc) != doc) { + return null; + } + } + else { + if (twoPhase.approximation().advance(doc) != doc || twoPhase.matches() == false) { + return null; + } + } + return Matches.MATCH_WITH_NO_TERMS; + } } private static class ParentApproximation extends DocIdSetIterator { diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java index 0d39e8b2589..de0e6d409f7 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionScoreQuery.java @@ -29,6 +29,7 @@ import org.apache.lucene.search.DoubleValuesSource; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.FilterScorer; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Matches; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Scorer; @@ -148,6 +149,11 @@ public final class FunctionScoreQuery extends Query { this.inner.extractTerms(terms); } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + return inner.matches(context, doc); + } + @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { Explanation scoreExplanation = inner.explain(context, doc); diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java index 8f821922c6d..8d6836b6105 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java +++ b/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java @@ -136,6 +136,28 @@ public final class CoveringQuery extends Query { } } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + LongValues minMatchValues = minimumNumberMatch.getValues(context, null); + if (minMatchValues.advanceExact(doc) == false) { + return null; + } + final long minimumNumberMatch = Math.max(1, minMatchValues.longValue()); + long matchCount = 0; + List subMatches = new ArrayList<>(); + for (Weight weight : weights) { + Matches matches = weight.matches(context, doc); + if (matches != null) { + matchCount++; + subMatches.add(matches); + } + } + if (matchCount < minimumNumberMatch) { + return null; + } + return Matches.fromSubMatches(subMatches); + } + @Override public Explanation explain(LeafReaderContext context, int doc) throws IOException { LongValues minMatchValues = minimumNumberMatch.getValues(context, null); diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatches.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatches.java new file mode 100644 index 00000000000..c5c6e981e1c --- /dev/null +++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatches.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.lucene.search; + +import java.io.IOException; +import java.util.Iterator; + +class AssertingMatches implements Matches { + + private final Matches in; + + AssertingMatches(Matches matches) { + this.in = matches; + } + + @Override + public MatchesIterator getMatches(String field) throws IOException { + MatchesIterator mi = in.getMatches(field); + if (mi == null) + return null; + return new AssertingMatchesIterator(mi); + } + + @Override + public Iterator iterator() { + return in.iterator(); + } +} diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatchesIterator.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatchesIterator.java new file mode 100644 index 00000000000..52fb184250c --- /dev/null +++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingMatchesIterator.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.lucene.search; + +import java.io.IOException; + +import org.apache.lucene.util.BytesRef; + +class AssertingMatchesIterator implements MatchesIterator { + + private final MatchesIterator in; + private State state = State.UNPOSITIONED; + + private enum State { UNPOSITIONED, ITERATING, EXHAUSTED } + + AssertingMatchesIterator(MatchesIterator in) { + this.in = in; + } + + @Override + public boolean next() throws IOException { + assert state != State.EXHAUSTED : state; + boolean more = in.next(); + if (more == false) { + state = State.EXHAUSTED; + } + else { + state = State.ITERATING; + } + return more; + } + + @Override + public int startPosition() { + assert state == State.ITERATING : state; + return in.startPosition(); + } + + @Override + public int endPosition() { + assert state == State.ITERATING : state; + return in.endPosition(); + } + + @Override + public int startOffset() throws IOException { + assert state == State.ITERATING : state; + return in.startOffset(); + } + + @Override + public int endOffset() throws IOException { + assert state == State.ITERATING : state; + return in.endOffset(); + } + + @Override + public BytesRef term() { + assert state == State.ITERATING : state; + return in.term(); + } +} diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java index 8e3a29fb002..55fda238ce7 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java @@ -31,6 +31,14 @@ class AssertingWeight extends FilterWeight { this.scoreMode = scoreMode; } + @Override + public Matches matches(LeafReaderContext context, int doc) throws IOException { + Matches matches = in.matches(context, doc); + if (matches == null) + return null; + return new AssertingMatches(matches); + } + @Override public Scorer scorer(LeafReaderContext context) throws IOException { if (random.nextBoolean()) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java b/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java index 4452c1965e2..a918078aeef 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java @@ -28,7 +28,9 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.util.LuceneTestCase; +import static junit.framework.Assert.assertNotNull; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; /** @@ -56,7 +58,7 @@ public class CheckHits { if (ignore.contains(Integer.valueOf(doc))) continue; Explanation exp = searcher.explain(q, doc); - Assert.assertNotNull("Explanation of [["+d+"]] for #"+doc+" is null", + assertNotNull("Explanation of [["+d+"]] for #"+doc+" is null", exp); Assert.assertFalse("Explanation of [["+d+"]] for #"+doc+ " doesn't indicate non-match: " + exp.toString(), @@ -300,6 +302,16 @@ public class CheckHits { (query, defaultFieldName, searcher, deep)); } + + /** + * Asserts that the result of calling {@link Weight#matches(LeafReaderContext, int)} + * for every document matching a query returns a non-null {@link Matches} + * @param query the query to test + * @param searcher the search to test against + */ + public static void checkMatches(Query query, IndexSearcher searcher) throws IOException { + searcher.search(query, new MatchesAsserter(query, searcher)); + } private static final Pattern COMPUTED_FROM_PATTERN = Pattern.compile(".*, computed as .* from:"); @@ -505,7 +517,7 @@ public class CheckHits { ("exception in hitcollector of [["+d+"]] for #"+doc, e); } - Assert.assertNotNull("Explanation of [["+d+"]] for #"+doc+" is null", exp); + assertNotNull("Explanation of [["+d+"]] for #"+doc+" is null", exp); verifyExplanation(d,doc,scorer.score(),deep,exp); Assert.assertTrue("Explanation of [["+d+"]] for #"+ doc + " does not indicate match: " + exp.toString(), @@ -522,6 +534,45 @@ public class CheckHits { } } + /** + * Asserts that the {@link Matches} from a query is non-null whenever + * the document its created for is a hit. + * + * Also checks that the previous non-matching document has a {@code null} {@link Matches} + */ + public static class MatchesAsserter extends SimpleCollector { + + private final Weight weight; + private LeafReaderContext context; + int lastCheckedDoc = -1; + + public MatchesAsserter(Query query, IndexSearcher searcher) throws IOException { + this.weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1); + } + + @Override + protected void doSetNextReader(LeafReaderContext context) throws IOException { + this.context = context; + this.lastCheckedDoc = -1; + } + + @Override + public void collect(int doc) throws IOException { + Matches matches = this.weight.matches(context, doc); + assertNotNull("Unexpected null Matches object in doc" + doc + " for query " + this.weight.getQuery(), matches); + if (lastCheckedDoc != doc - 1) { + assertNull("Unexpected non-null Matches object in non-matching doc" + doc + " for query " + this.weight.getQuery(), + this.weight.matches(context, doc - 1)); + } + lastCheckedDoc = doc; + } + + @Override + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE_NO_SCORES; + } + } + public static void checkTopScores(Random random, Query query, IndexSearcher searcher) throws IOException { // Check it computed the top hits correctly doCheckTopScores(query, searcher, 1); diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java index 139cb9d16ed..f68b7176bc9 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java @@ -108,6 +108,7 @@ public class QueryUtils { * @see #checkSkipTo * @see #checkExplanations * @see #checkEqual + * @see CheckHits#checkMatches(Query, IndexSearcher) */ public static void check(Random random, Query q1, IndexSearcher s) { check(random, q1, s, true); @@ -125,6 +126,7 @@ public class QueryUtils { check(random, q1, wrapUnderlyingReader(random, s, +1), false); } checkExplanations(q1,s); + CheckHits.checkMatches(q1, s); } } catch (IOException e) { throw new RuntimeException(e); diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 9cefce2c5f2..84cac13ed55 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -88,6 +88,8 @@ New Features * SOLR-12151: Add abstract MultiSolrCloudTestCase class. (Christine Poerschke) +* SOLR-12181: Add index size autoscaling trigger, based on document count or size in bytes. (ab) + Bug Fixes ---------------------- diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanAction.java index febd6bda2c0..4189aa4329e 100644 --- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanAction.java +++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoAddReplicasPlanAction.java @@ -33,7 +33,7 @@ import org.apache.solr.common.cloud.ZkStateReader; public class AutoAddReplicasPlanAction extends ComputePlanAction { @Override - protected Suggester getSuggester(Policy.Session session, TriggerEvent event, SolrCloudManager cloudManager) { + protected Suggester getSuggester(Policy.Session session, TriggerEvent event, ActionContext context, SolrCloudManager cloudManager) { // for backward compatibility ClusterStateProvider stateProvider = cloudManager.getClusterStateProvider(); String autoAddReplicas = stateProvider.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, (String) null); @@ -41,7 +41,7 @@ public class AutoAddReplicasPlanAction extends ComputePlanAction { return NoneSuggester.get(session); } - Suggester suggester = super.getSuggester(session, event, cloudManager); + Suggester suggester = super.getSuggester(session, event, context, cloudManager); ClusterState clusterState; try { clusterState = stateProvider.getClusterState(); diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java index 68282a7f1e7..93f449a7e76 100644 --- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java +++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java @@ -180,6 +180,9 @@ public class AutoScaling { case SCHEDULED: t = new ScheduledTrigger(name); break; + case INDEXSIZE: + t = new IndexSizeTrigger(name); + break; default: throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name); } diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java index 8f3175c3052..4a9c7442774 100644 --- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java +++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java @@ -34,6 +34,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.Policy; import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper; import org.apache.solr.client.solrj.cloud.SolrCloudManager; import org.apache.solr.client.solrj.cloud.autoscaling.Suggester; +import org.apache.solr.client.solrj.cloud.autoscaling.UnsupportedSuggester; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.params.AutoScalingParams; @@ -88,7 +89,7 @@ public class ComputePlanAction extends TriggerActionBase { log.trace("-- state: {}", clusterState); } try { - Suggester intialSuggester = getSuggester(session, event, cloudManager); + Suggester intialSuggester = getSuggester(session, event, context, cloudManager); Suggester suggester = intialSuggester; int maxOperations = getMaxNumOps(event, autoScalingConf, clusterState); int requestedOperations = getRequestedNumOps(event); @@ -112,7 +113,7 @@ public class ComputePlanAction extends TriggerActionBase { if (suggester.getSession() != null) { session = suggester.getSession(); } - suggester = getSuggester(session, event, cloudManager); + suggester = getSuggester(session, event, context, cloudManager); // break on first null op // unless a specific number of ops was requested @@ -190,7 +191,7 @@ public class ComputePlanAction extends TriggerActionBase { private static final String START = "__start__"; - protected Suggester getSuggester(Policy.Session session, TriggerEvent event, SolrCloudManager cloudManager) { + protected Suggester getSuggester(Policy.Session session, TriggerEvent event, ActionContext context, SolrCloudManager cloudManager) { Suggester suggester; switch (event.getEventType()) { case NODEADDED: @@ -203,6 +204,7 @@ public class ComputePlanAction extends TriggerActionBase { break; case SEARCHRATE: case METRIC: + case INDEXSIZE: List ops = (List)event.getProperty(TriggerEvent.REQUESTED_OPS, Collections.emptyList()); int start = (Integer)event.getProperty(START, 0); if (ops.isEmpty() || start >= ops.size()) { @@ -210,14 +212,15 @@ public class ComputePlanAction extends TriggerActionBase { } TriggerEvent.Op op = ops.get(start); suggester = session.getSuggester(op.getAction()); + if (suggester instanceof UnsupportedSuggester) { + List unsupportedOps = (List)context.getProperties().computeIfAbsent("unsupportedOps", k -> new ArrayList()); + unsupportedOps.add(op); + } for (Map.Entry e : op.getHints().entrySet()) { suggester = suggester.hint(e.getKey(), e.getValue()); } - if (++start >= ops.size()) { - event.getProperties().remove(START); - } else { - event.getProperties().put(START, start); - } + start++; + event.getProperties().put(START, start); break; case SCHEDULED: String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower()); @@ -225,7 +228,7 @@ public class ComputePlanAction extends TriggerActionBase { suggester = session.getSuggester(action); break; default: - throw new UnsupportedOperationException("No support for events other than nodeAdded, nodeLost, searchRate and metric. Received: " + event.getEventType()); + throw new UnsupportedOperationException("No support for events other than nodeAdded, nodeLost, searchRate, metric and indexSize. Received: " + event.getEventType()); } return suggester; } diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java new file mode 100644 index 00000000000..756f88ffd88 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java @@ -0,0 +1,408 @@ +/* + * 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.solr.cloud.autoscaling; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.solr.client.solrj.cloud.SolrCloudManager; +import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo; +import org.apache.solr.client.solrj.cloud.autoscaling.Suggester; +import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.cloud.ClusterState; +import org.apache.solr.common.cloud.DocCollection; +import org.apache.solr.common.cloud.Replica; +import org.apache.solr.common.cloud.Slice; +import org.apache.solr.common.params.CollectionParams; +import org.apache.solr.common.util.Pair; +import org.apache.solr.common.util.StrUtils; +import org.apache.solr.common.util.Utils; +import org.apache.solr.core.SolrResourceLoader; +import org.apache.solr.metrics.SolrCoreMetricManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + */ +public class IndexSizeTrigger extends TriggerBase { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + public static final String ABOVE_BYTES_PROP = "aboveBytes"; + public static final String ABOVE_DOCS_PROP = "aboveDocs"; + public static final String ABOVE_OP_PROP = "aboveOp"; + public static final String BELOW_BYTES_PROP = "belowBytes"; + public static final String BELOW_DOCS_PROP = "belowDocs"; + public static final String BELOW_OP_PROP = "belowOp"; + public static final String COLLECTIONS_PROP = "collections"; + + public static final String BYTES_SIZE_PROP = "__bytes__"; + public static final String DOCS_SIZE_PROP = "__docs__"; + public static final String ABOVE_SIZE_PROP = "aboveSize"; + public static final String BELOW_SIZE_PROP = "belowSize"; + public static final String VIOLATION_PROP = "violationType"; + + public enum Unit { bytes, docs } + + private long aboveBytes, aboveDocs, belowBytes, belowDocs; + private CollectionParams.CollectionAction aboveOp, belowOp; + private final Set collections = new HashSet<>(); + private final Map lastEventMap = new ConcurrentHashMap<>(); + + public IndexSizeTrigger(String name) { + super(TriggerEventType.INDEXSIZE, name); + TriggerUtils.validProperties(validProperties, + ABOVE_BYTES_PROP, ABOVE_DOCS_PROP, BELOW_BYTES_PROP, BELOW_DOCS_PROP, COLLECTIONS_PROP); + } + + @Override + public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map properties) throws TriggerValidationException { + super.configure(loader, cloudManager, properties); + String aboveStr = String.valueOf(properties.getOrDefault(ABOVE_BYTES_PROP, Long.MAX_VALUE)); + String belowStr = String.valueOf(properties.getOrDefault(BELOW_BYTES_PROP, -1)); + try { + aboveBytes = Long.parseLong(aboveStr); + if (aboveBytes <= 0) { + throw new Exception("value must be > 0"); + } + } catch (Exception e) { + throw new TriggerValidationException(getName(), ABOVE_BYTES_PROP, "invalid value '" + aboveStr + "': " + e.toString()); + } + try { + belowBytes = Long.parseLong(belowStr); + if (belowBytes < 0) { + belowBytes = -1; + } + } catch (Exception e) { + throw new TriggerValidationException(getName(), BELOW_BYTES_PROP, "invalid value '" + belowStr + "': " + e.toString()); + } + // below must be at least 2x smaller than above, otherwise splitting a shard + // would immediately put the shard below the threshold and cause the mergeshards action + if (belowBytes > 0 && (belowBytes * 2 > aboveBytes)) { + throw new TriggerValidationException(getName(), BELOW_BYTES_PROP, + "invalid value " + belowBytes + ", should be less than half of '" + ABOVE_BYTES_PROP + "' value, which is " + aboveBytes); + } + // do the same for docs bounds + aboveStr = String.valueOf(properties.getOrDefault(ABOVE_DOCS_PROP, Long.MAX_VALUE)); + belowStr = String.valueOf(properties.getOrDefault(BELOW_DOCS_PROP, -1)); + try { + aboveDocs = Long.parseLong(aboveStr); + if (aboveDocs <= 0) { + throw new Exception("value must be > 0"); + } + } catch (Exception e) { + throw new TriggerValidationException(getName(), ABOVE_DOCS_PROP, "invalid value '" + aboveStr + "': " + e.toString()); + } + try { + belowDocs = Long.parseLong(belowStr); + if (belowDocs < 0) { + belowDocs = -1; + } + } catch (Exception e) { + throw new TriggerValidationException(getName(), BELOW_DOCS_PROP, "invalid value '" + belowStr + "': " + e.toString()); + } + // below must be at least 2x smaller than above, otherwise splitting a shard + // would immediately put the shard below the threshold and cause the mergeshards action + if (belowDocs > 0 && (belowDocs * 2 > aboveDocs)) { + throw new TriggerValidationException(getName(), BELOW_DOCS_PROP, + "invalid value " + belowDocs + ", should be less than half of '" + ABOVE_DOCS_PROP + "' value, which is " + aboveDocs); + } + + String collectionsString = (String) properties.get(COLLECTIONS_PROP); + if (collectionsString != null && !collectionsString.isEmpty()) { + collections.addAll(StrUtils.splitSmart(collectionsString, ',')); + } + String aboveOpStr = String.valueOf(properties.getOrDefault(ABOVE_OP_PROP, CollectionParams.CollectionAction.SPLITSHARD.toLower())); + // TODO: this is a placeholder until SOLR-9407 is implemented + String belowOpStr = String.valueOf(properties.getOrDefault(BELOW_OP_PROP, CollectionParams.CollectionAction.MERGESHARDS.toLower())); + aboveOp = CollectionParams.CollectionAction.get(aboveOpStr); + if (aboveOp == null) { + throw new TriggerValidationException(getName(), ABOVE_OP_PROP, "unrecognized value of " + ABOVE_OP_PROP + ": '" + aboveOpStr + "'"); + } + belowOp = CollectionParams.CollectionAction.get(belowOpStr); + if (belowOp == null) { + throw new TriggerValidationException(getName(), BELOW_OP_PROP, "unrecognized value of " + BELOW_OP_PROP + ": '" + belowOpStr + "'"); + } + } + + @Override + protected Map getState() { + Map state = new HashMap<>(); + state.put("lastEventMap", lastEventMap); + return state; + } + + @Override + protected void setState(Map state) { + this.lastEventMap.clear(); + Map replicaVsTime = (Map)state.get("lastEventMap"); + if (replicaVsTime != null) { + this.lastEventMap.putAll(replicaVsTime); + } + } + + @Override + public void restoreState(AutoScaling.Trigger old) { + assert old.isClosed(); + if (old instanceof IndexSizeTrigger) { + } else { + throw new SolrException(SolrException.ErrorCode.INVALID_STATE, + "Unable to restore state from an unknown type of trigger"); + } + } + + @Override + public void run() { + synchronized(this) { + if (isClosed) { + log.warn(getName() + " ran but was already closed"); + return; + } + } + AutoScaling.TriggerEventProcessor processor = processorRef.get(); + if (processor == null) { + return; + } + + // replica name / info + size, retrieved from leaders only + Map currentSizes = new HashMap<>(); + + try { + ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState(); + for (String node : clusterState.getLiveNodes()) { + Map metricTags = new HashMap<>(); + // coll, shard, replica + Map>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList()); + infos.forEach((coll, shards) -> { + if (!collections.isEmpty() && !collections.contains(coll)) { + return; + } + DocCollection docCollection = clusterState.getCollection(coll); + + shards.forEach((sh, replicas) -> { + // check only the leader of a replica in active shard + Slice s = docCollection.getSlice(sh); + if (s.getState() != Slice.State.ACTIVE) { + return; + } + Replica r = s.getLeader(); + // no leader - don't do anything + if (r == null) { + return; + } + // find ReplicaInfo + ReplicaInfo info = null; + for (ReplicaInfo ri : replicas) { + if (r.getCoreName().equals(ri.getCore())) { + info = ri; + break; + } + } + if (info == null) { + // probably replica is not on this node? + return; + } + // we have to translate to the metrics registry name, which uses "_replica_nN" as suffix + String replicaName = Utils.parseMetricsReplicaName(coll, info.getCore()); + if (replicaName == null) { // should never happen??? + replicaName = info.getName(); // which is actually coreNode name... + } + String registry = SolrCoreMetricManager.createRegistryName(true, coll, sh, replicaName, null); + String tag = "metrics:" + registry + ":INDEX.sizeInBytes"; + metricTags.put(tag, info); + tag = "metrics:" + registry + ":SEARCHER.searcher.numDocs"; + metricTags.put(tag, info); + }); + }); + if (metricTags.isEmpty()) { + continue; + } + Map sizes = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet()); + sizes.forEach((tag, size) -> { + final ReplicaInfo info = metricTags.get(tag); + if (info == null) { + log.warn("Missing replica info for response tag " + tag); + } else { + // verify that it's a Number + if (!(size instanceof Number)) { + log.warn("invalid size value - not a number: '" + size + "' is " + size.getClass().getName()); + return; + } + + ReplicaInfo currentInfo = currentSizes.computeIfAbsent(info.getCore(), k -> (ReplicaInfo)info.clone()); + if (tag.contains("INDEX")) { + currentInfo.getVariables().put(BYTES_SIZE_PROP, ((Number) size).longValue()); + } else { + currentInfo.getVariables().put(DOCS_SIZE_PROP, ((Number) size).longValue()); + } + } + }); + } + } catch (IOException e) { + log.warn("Error running trigger " + getName(), e); + return; + } + + long now = cloudManager.getTimeSource().getTimeNs(); + + // now check thresholds + + // collection / list(info) + Map> aboveSize = new HashMap<>(); + currentSizes.entrySet().stream() + .filter(e -> ( + (Long)e.getValue().getVariable(BYTES_SIZE_PROP) > aboveBytes || + (Long)e.getValue().getVariable(DOCS_SIZE_PROP) > aboveDocs + ) && waitForElapsed(e.getKey(), now, lastEventMap)) + .forEach(e -> { + ReplicaInfo info = e.getValue(); + List infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>()); + if (!infos.contains(info)) { + if ((Long)e.getValue().getVariable(BYTES_SIZE_PROP) > aboveBytes) { + info.getVariables().put(VIOLATION_PROP, ABOVE_BYTES_PROP); + } else { + info.getVariables().put(VIOLATION_PROP, ABOVE_DOCS_PROP); + } + infos.add(info); + } + }); + // collection / list(info) + Map> belowSize = new HashMap<>(); + currentSizes.entrySet().stream() + .filter(e -> ( + (Long)e.getValue().getVariable(BYTES_SIZE_PROP) < belowBytes || + (Long)e.getValue().getVariable(DOCS_SIZE_PROP) < belowDocs + ) && waitForElapsed(e.getKey(), now, lastEventMap)) + .forEach(e -> { + ReplicaInfo info = e.getValue(); + List infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>()); + if (!infos.contains(info)) { + if ((Long)e.getValue().getVariable(BYTES_SIZE_PROP) < belowBytes) { + info.getVariables().put(VIOLATION_PROP, BELOW_BYTES_PROP); + } else { + info.getVariables().put(VIOLATION_PROP, BELOW_DOCS_PROP); + } + infos.add(info); + } + }); + + if (aboveSize.isEmpty() && belowSize.isEmpty()) { + return; + } + + // find the earliest time when a condition was exceeded + final AtomicLong eventTime = new AtomicLong(now); + + // calculate ops + final List ops = new ArrayList<>(); + aboveSize.forEach((coll, replicas) -> { + replicas.forEach(r -> { + TriggerEvent.Op op = new TriggerEvent.Op(aboveOp); + op.addHint(Suggester.Hint.COLL_SHARD, new Pair<>(coll, r.getShard())); + ops.add(op); + Long time = lastEventMap.get(r.getCore()); + if (time != null && eventTime.get() > time) { + eventTime.set(time); + } + }); + }); + belowSize.forEach((coll, replicas) -> { + if (replicas.size() < 2) { + return; + } + // sort by increasing size + replicas.sort((r1, r2) -> { + // XXX this is not quite correct - if BYTES_SIZE_PROP decided that replica got here + // then we should be sorting by BYTES_SIZE_PROP. However, since DOCS and BYTES are + // loosely correlated it's simpler to sort just by docs (which better reflects the "too small" + // condition than index size, due to possibly existing deleted docs that still occupy space) + long delta = (Long) r1.getVariable(DOCS_SIZE_PROP) - (Long) r2.getVariable(DOCS_SIZE_PROP); + if (delta > 0) { + return 1; + } else if (delta < 0) { + return -1; + } else { + return 0; + } + }); + + // TODO: MERGESHARDS is not implemented yet. For now take the top two smallest shards + // TODO: but in the future we probably need to get ones with adjacent ranges. + + // TODO: generate as many MERGESHARDS as needed to consume all belowSize shards + TriggerEvent.Op op = new TriggerEvent.Op(belowOp); + op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(0).getShard())); + op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(1).getShard())); + ops.add(op); + Long time = lastEventMap.get(replicas.get(0).getCore()); + if (time != null && eventTime.get() > time) { + eventTime.set(time); + } + time = lastEventMap.get(replicas.get(1).getCore()); + if (time != null && eventTime.get() > time) { + eventTime.set(time); + } + }); + + if (ops.isEmpty()) { + return; + } + if (processor.process(new IndexSizeEvent(getName(), eventTime.get(), ops, aboveSize, belowSize))) { + // update last event times + aboveSize.forEach((coll, replicas) -> { + replicas.forEach(r -> lastEventMap.put(r.getCore(), now)); + }); + belowSize.forEach((coll, replicas) -> { + lastEventMap.put(replicas.get(0).getCore(), now); + lastEventMap.put(replicas.get(1).getCore(), now); + }); + } + } + + private boolean waitForElapsed(String name, long now, Map lastEventMap) { + Long lastTime = lastEventMap.computeIfAbsent(name, s -> now); + long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS); + log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed); + if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) { + return false; + } + return true; + } + + public static class IndexSizeEvent extends TriggerEvent { + public IndexSizeEvent(String source, long eventTime, List ops, Map> aboveSize, + Map> belowSize) { + super(TriggerEventType.INDEXSIZE, source, eventTime, null); + properties.put(TriggerEvent.REQUESTED_OPS, ops); + properties.put(ABOVE_SIZE_PROP, aboveSize); + properties.put(BELOW_SIZE_PROP, belowSize); + } + } + +} diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java index 9fdf8dc354b..9058a9a67e0 100644 --- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java +++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java @@ -203,12 +203,12 @@ public class MetricTrigger extends TriggerBase { List ops = new ArrayList<>(hotNodes.size()); for (String n : hotNodes.keySet()) { Op op = new Op(CollectionParams.CollectionAction.get(preferredOp)); - op.setHint(Suggester.Hint.SRC_NODE, n); + op.addHint(Suggester.Hint.SRC_NODE, n); if (!collection.equals(Policy.ANY)) { if (!shard.equals(Policy.ANY)) { - op.setHint(Suggester.Hint.COLL_SHARD, new Pair<>(collection, shard)); + op.addHint(Suggester.Hint.COLL_SHARD, new Pair<>(collection, shard)); } else { - op.setHint(Suggester.Hint.COLL, collection); + op.addHint(Suggester.Hint.COLL, collection); } } ops.add(op); diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java index 00bc6d878d2..02a2d0cdff4 100644 --- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java +++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java @@ -181,10 +181,11 @@ public class SearchRateTrigger extends TriggerBase { } else { Map> perCollection = collectionRates.computeIfAbsent(info.getCollection(), s -> new HashMap<>()); List perShard = perCollection.computeIfAbsent(info.getShard(), s -> new ArrayList<>()); - info.getVariables().put(AutoScalingParams.RATE, rate); + info = (ReplicaInfo)info.clone(); + info.getVariables().put(AutoScalingParams.RATE, ((Number)rate).doubleValue()); perShard.add(info); AtomicDouble perNode = nodeRates.computeIfAbsent(node, s -> new AtomicDouble()); - perNode.addAndGet((Double)rate); + perNode.addAndGet(((Number)rate).doubleValue()); } }); } diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java index e4a4b3d0b35..907309dce8f 100644 --- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java +++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java @@ -17,9 +17,13 @@ package org.apache.solr.cloud.autoscaling; import java.io.IOException; +import java.util.Collection; +import java.util.Collections; import java.util.EnumMap; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import org.apache.solr.client.solrj.cloud.autoscaling.Suggester; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; @@ -49,11 +53,17 @@ public class TriggerEvent implements MapWriter { public Op(CollectionParams.CollectionAction action, Suggester.Hint hint, Object hintValue) { this.action = action; - this.hints.put(hint, hintValue); + addHint(hint, hintValue); } - public void setHint(Suggester.Hint hint, Object value) { - hints.put(hint, value); + public void addHint(Suggester.Hint hint, Object value) { + hint.validator.accept(value); + if (hint.multiValued) { + Collection values = value instanceof Collection ? (Collection) value : Collections.singletonList(value); + ((Set) hints.computeIfAbsent(hint, h -> new HashSet<>())).addAll(values); + } else { + hints.put(hint, value == null ? null : String.valueOf(value)); + } } public CollectionParams.CollectionAction getAction() { diff --git a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java b/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java index 04c90b1301e..55902521a2e 100644 --- a/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java +++ b/solr/core/src/test/org/apache/solr/cloud/CloudTestUtils.java @@ -19,6 +19,7 @@ package org.apache.solr.cloud; import java.io.IOException; import java.lang.invoke.MethodHandles; +import java.util.Collection; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -113,19 +114,30 @@ public class CloudTestUtils { * number of shards and replicas */ public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas) { + return clusterShape(expectedShards, expectedReplicas, false); + } + + public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas, boolean withInactive) { return (liveNodes, collectionState) -> { - if (collectionState == null) + if (collectionState == null) { + log.debug("-- null collection"); return false; - if (collectionState.getSlices().size() != expectedShards) + } + Collection slices = withInactive ? collectionState.getSlices() : collectionState.getActiveSlices(); + if (slices.size() != expectedShards) { + log.debug("-- wrong number of active slices, expected=" + expectedShards + ", found=" + collectionState.getSlices().size()); return false; - for (Slice slice : collectionState) { + } + for (Slice slice : slices) { int activeReplicas = 0; for (Replica replica : slice) { if (replica.isActive(liveNodes)) activeReplicas++; } - if (activeReplicas != expectedReplicas) + if (activeReplicas != expectedReplicas) { + log.debug("-- wrong number of active replicas in slice " + slice.getName() + ", expected=" + expectedReplicas + ", found=" + activeReplicas); return false; + } } return true; }; diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerTest.java new file mode 100644 index 00000000000..79dd0197ba6 --- /dev/null +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerTest.java @@ -0,0 +1,647 @@ +/* + * 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.solr.cloud.autoscaling; + +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.lucene.util.TestUtil; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.cloud.SolrCloudManager; +import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; +import org.apache.solr.client.solrj.cloud.autoscaling.Suggester; +import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.cloud.CloudTestUtils; +import org.apache.solr.cloud.SolrCloudTestCase; +import org.apache.solr.cloud.autoscaling.sim.SimCloudManager; +import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.cloud.ZkNodeProps; +import org.apache.solr.common.params.CollectionParams; +import org.apache.solr.common.util.NamedList; +import org.apache.solr.common.util.Pair; +import org.apache.solr.common.util.TimeSource; +import org.apache.solr.common.util.Utils; +import org.apache.solr.core.SolrResourceLoader; +import org.apache.solr.util.LogLevel; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest; +import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH; + +/** + * + */ +@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG") +public class IndexSizeTriggerTest extends SolrCloudTestCase { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private static SolrCloudManager cloudManager; + private static SolrClient solrClient; + private static TimeSource timeSource; + private static SolrResourceLoader loader; + + private static int SPEED; + + private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> { + fail("Did not expect the processor to fire on first run! event=" + event); + return true; + }; + private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(2); + + static Map> listenerEvents = new ConcurrentHashMap<>(); + static CountDownLatch listenerCreated = new CountDownLatch(1); + static CountDownLatch finished = new CountDownLatch(1); + + @BeforeClass + public static void setupCluster() throws Exception { + configureCluster(2) + .addConfig("conf", configset("cloud-minimal")) + .configure(); + if (random().nextBoolean()) { + cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager(); + solrClient = cluster.getSolrClient(); + loader = cluster.getJettySolrRunner(0).getCoreContainer().getResourceLoader(); + SPEED = 1; + } else { + SPEED = 50; + cloudManager = SimCloudManager.createCluster(2, TimeSource.get("simTime:" + SPEED)); + // wait for defaults to be applied - due to accelerated time sometimes we may miss this + cloudManager.getTimeSource().sleep(10000); + AutoScalingConfig cfg = cloudManager.getDistribStateManager().getAutoScalingConfig(); + assertFalse("autoscaling config is empty", cfg.isEmpty()); + solrClient = ((SimCloudManager)cloudManager).simGetSolrClient(); + loader = ((SimCloudManager) cloudManager).getLoader(); + } + timeSource = cloudManager.getTimeSource(); + } + + @After + public void restoreDefaults() throws Exception { + if (cloudManager instanceof SimCloudManager) { + log.info(((SimCloudManager) cloudManager).dumpClusterState(true)); + ((SimCloudManager) cloudManager).getSimClusterStateProvider().simDeleteAllCollections(); + ((SimCloudManager) cloudManager).simResetOpCounts(); + } else { + cluster.deleteAllCollections(); + } + cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), -1); + cloudManager.getTimeSource().sleep(5000); + listenerEvents.clear(); + listenerCreated = new CountDownLatch(1); + finished = new CountDownLatch(1); + } + + @AfterClass + public static void teardown() throws Exception { + if (cloudManager instanceof SimCloudManager) { + cloudManager.close(); + } + solrClient = null; + cloudManager = null; + } + + @Test + public void testTrigger() throws Exception { + String collectionName = "testTrigger_collection"; + CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, + "conf", 2, 2).setMaxShardsPerNode(2); + create.process(solrClient); + CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName, + CloudTestUtils.clusterShape(2, 2)); + + long waitForSeconds = 3 + random().nextInt(5); + Map props = createTriggerProps(waitForSeconds); + try (IndexSizeTrigger trigger = new IndexSizeTrigger("index_size_trigger")) { + trigger.configure(loader, cloudManager, props); + trigger.init(); + trigger.setProcessor(noFirstRunProcessor); + trigger.run(); + + for (int i = 0; i < 25; i++) { + SolrInputDocument doc = new SolrInputDocument("id", "id-" + i); + solrClient.add(collectionName, doc); + } + solrClient.commit(collectionName); + + AtomicBoolean fired = new AtomicBoolean(false); + AtomicReference eventRef = new AtomicReference<>(); + trigger.setProcessor(event -> { + if (fired.compareAndSet(false, true)) { + eventRef.set(event); + long currentTimeNanos = timeSource.getTimeNs(); + long eventTimeNanos = event.getEventTime(); + long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS; + if (currentTimeNanos - eventTimeNanos <= waitForNanos) { + fail("processor was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos); + } + } else { + fail("IndexSizeTrigger was fired more than once!"); + } + return true; + }); + trigger.run(); + TriggerEvent ev = eventRef.get(); + // waitFor delay - should not produce any event yet + assertNull("waitFor not elapsed but produced an event", ev); + timeSource.sleep(TimeUnit.MILLISECONDS.convert(waitForSeconds + 1, TimeUnit.SECONDS)); + trigger.run(); + ev = eventRef.get(); + assertNotNull("should have fired an event", ev); + List ops = (List) ev.getProperty(TriggerEvent.REQUESTED_OPS); + assertNotNull("should contain requestedOps", ops); + assertEquals("number of ops", 2, ops.size()); + boolean shard1 = false; + boolean shard2 = false; + for (TriggerEvent.Op op : ops) { + assertEquals(CollectionParams.CollectionAction.SPLITSHARD, op.getAction()); + Set> hints = (Set>)op.getHints().get(Suggester.Hint.COLL_SHARD); + assertNotNull("hints", hints); + assertEquals("hints", 1, hints.size()); + Pair p = hints.iterator().next(); + assertEquals(collectionName, p.first()); + if (p.second().equals("shard1")) { + shard1 = true; + } else if (p.second().equals("shard2")) { + shard2 = true; + } else { + fail("unexpected shard name " + p.second()); + } + } + assertTrue("shard1 should be split", shard1); + assertTrue("shard2 should be split", shard2); + } + } + + public static class CapturingTriggerListener extends TriggerListenerBase { + @Override + public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException { + super.configure(loader, cloudManager, config); + listenerCreated.countDown(); + } + + @Override + public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, + ActionContext context, Throwable error, String message) { + List lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>()); + CapturedEvent ev = new CapturedEvent(timeSource.getTimeNs(), context, config, stage, actionName, event, message); + log.info("=======> " + ev); + lst.add(ev); + } + } + + public static class FinishedProcessingListener extends TriggerListenerBase { + + @Override + public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context, Throwable error, String message) throws Exception { + finished.countDown(); + } + } + + @Test + public void testSplitIntegration() throws Exception { + String collectionName = "testSplitIntegration_collection"; + CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, + "conf", 2, 2).setMaxShardsPerNode(2); + create.process(solrClient); + CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName, + CloudTestUtils.clusterShape(2, 2)); + + long waitForSeconds = 3 + random().nextInt(5); + String setTriggerCommand = "{" + + "'set-trigger' : {" + + "'name' : 'index_size_trigger'," + + "'event' : 'indexSize'," + + "'waitFor' : '" + waitForSeconds + "s'," + + "'aboveDocs' : 10," + + "'belowDocs' : 4," + + "'enabled' : true," + + "'actions' : [{'name' : 'compute_plan', 'class' : 'solr.ComputePlanAction'}," + + "{'name' : 'execute_plan', 'class' : '" + ExecutePlanAction.class.getName() + "'}]" + + "}}"; + SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand); + NamedList response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + String setListenerCommand = "{" + + "'set-listener' : " + + "{" + + "'name' : 'capturing'," + + "'trigger' : 'index_size_trigger'," + + "'stage' : ['STARTED','ABORTED','SUCCEEDED','FAILED']," + + "'beforeAction' : ['compute_plan','execute_plan']," + + "'afterAction' : ['compute_plan','execute_plan']," + + "'class' : '" + CapturingTriggerListener.class.getName() + "'" + + "}" + + "}"; + req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + setListenerCommand = "{" + + "'set-listener' : " + + "{" + + "'name' : 'finished'," + + "'trigger' : 'index_size_trigger'," + + "'stage' : ['SUCCEEDED']," + + "'class' : '" + FinishedProcessingListener.class.getName() + "'" + + "}" + + "}"; + req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + + for (int i = 0; i < 25; i++) { + SolrInputDocument doc = new SolrInputDocument("id", "id-" + i); + solrClient.add(collectionName, doc); + } + solrClient.commit(collectionName); + + timeSource.sleep(TimeUnit.MILLISECONDS.convert(waitForSeconds + 1, TimeUnit.SECONDS)); + + boolean await = finished.await(60000 / SPEED, TimeUnit.MILLISECONDS); + assertTrue("did not finish processing in time", await); + CloudTestUtils.waitForState(cloudManager, collectionName, 10, TimeUnit.SECONDS, CloudTestUtils.clusterShape(4, 2)); + assertEquals(1, listenerEvents.size()); + List events = listenerEvents.get("capturing"); + assertNotNull("'capturing' events not found", events); + assertEquals("events: " + events, 6, events.size()); + assertEquals(TriggerEventProcessorStage.STARTED, events.get(0).stage); + assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, events.get(1).stage); + assertEquals(TriggerEventProcessorStage.AFTER_ACTION, events.get(2).stage); + assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, events.get(3).stage); + assertEquals(TriggerEventProcessorStage.AFTER_ACTION, events.get(4).stage); + assertEquals(TriggerEventProcessorStage.SUCCEEDED, events.get(5).stage); + // check ops + List ops = (List) events.get(4).event.getProperty(TriggerEvent.REQUESTED_OPS); + assertNotNull("should contain requestedOps", ops); + assertEquals("number of ops", 2, ops.size()); + boolean shard1 = false; + boolean shard2 = false; + for (TriggerEvent.Op op : ops) { + assertEquals(CollectionParams.CollectionAction.SPLITSHARD, op.getAction()); + Set> hints = (Set>)op.getHints().get(Suggester.Hint.COLL_SHARD); + assertNotNull("hints", hints); + assertEquals("hints", 1, hints.size()); + Pair p = hints.iterator().next(); + assertEquals(collectionName, p.first()); + if (p.second().equals("shard1")) { + shard1 = true; + } else if (p.second().equals("shard2")) { + shard2 = true; + } else { + fail("unexpected shard name " + p.second()); + } + } + assertTrue("shard1 should be split", shard1); + assertTrue("shard2 should be split", shard2); + + } + + @Test + public void testMergeIntegration() throws Exception { + String collectionName = "testMergeIntegration_collection"; + CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, + "conf", 2, 2).setMaxShardsPerNode(2); + create.process(solrClient); + CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName, + CloudTestUtils.clusterShape(2, 2)); + + for (int i = 0; i < 10; i++) { + SolrInputDocument doc = new SolrInputDocument("id", "id-" + (i * 100)); + solrClient.add(collectionName, doc); + } + solrClient.commit(collectionName); + + long waitForSeconds = 3 + random().nextInt(5); + String setTriggerCommand = "{" + + "'set-trigger' : {" + + "'name' : 'index_size_trigger'," + + "'event' : 'indexSize'," + + "'waitFor' : '" + waitForSeconds + "s'," + + "'aboveDocs' : 40," + + "'belowDocs' : 4," + + "'enabled' : true," + + "'actions' : [{'name' : 'compute_plan', 'class' : 'solr.ComputePlanAction'}," + + "{'name' : 'execute_plan', 'class' : '" + ExecutePlanAction.class.getName() + "'}]" + + "}}"; + SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand); + NamedList response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + String setListenerCommand = "{" + + "'set-listener' : " + + "{" + + "'name' : 'capturing'," + + "'trigger' : 'index_size_trigger'," + + "'stage' : ['STARTED','ABORTED','SUCCEEDED','FAILED']," + + "'beforeAction' : ['compute_plan','execute_plan']," + + "'afterAction' : ['compute_plan','execute_plan']," + + "'class' : '" + CapturingTriggerListener.class.getName() + "'" + + "}" + + "}"; + req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + setListenerCommand = "{" + + "'set-listener' : " + + "{" + + "'name' : 'finished'," + + "'trigger' : 'index_size_trigger'," + + "'stage' : ['SUCCEEDED']," + + "'class' : '" + FinishedProcessingListener.class.getName() + "'" + + "}" + + "}"; + req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + // delete some docs to trigger a merge + for (int i = 0; i < 5; i++) { + solrClient.deleteById(collectionName, "id-" + (i * 100)); + } + solrClient.commit(collectionName); + + timeSource.sleep(TimeUnit.MILLISECONDS.convert(waitForSeconds + 1, TimeUnit.SECONDS)); + + boolean await = finished.await(60000 / SPEED, TimeUnit.MILLISECONDS); + assertTrue("did not finish processing in time", await); + assertEquals(1, listenerEvents.size()); + List events = listenerEvents.get("capturing"); + assertNotNull("'capturing' events not found", events); + assertEquals("events: " + events, 6, events.size()); + assertEquals(TriggerEventProcessorStage.STARTED, events.get(0).stage); + assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, events.get(1).stage); + assertEquals(TriggerEventProcessorStage.AFTER_ACTION, events.get(2).stage); + assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, events.get(3).stage); + assertEquals(TriggerEventProcessorStage.AFTER_ACTION, events.get(4).stage); + assertEquals(TriggerEventProcessorStage.SUCCEEDED, events.get(5).stage); + // check ops + List ops = (List) events.get(4).event.getProperty(TriggerEvent.REQUESTED_OPS); + assertNotNull("should contain requestedOps", ops); + assertTrue("number of ops: " + ops, ops.size() > 0); + for (TriggerEvent.Op op : ops) { + assertEquals(CollectionParams.CollectionAction.MERGESHARDS, op.getAction()); + Set> hints = (Set>)op.getHints().get(Suggester.Hint.COLL_SHARD); + assertNotNull("hints", hints); + assertEquals("hints", 2, hints.size()); + Pair p = hints.iterator().next(); + assertEquals(collectionName, p.first()); + } + + // TODO: fix this once MERGESHARDS is supported + List unsupportedOps = (List)events.get(2).context.get("properties.unsupportedOps"); + assertNotNull("should have unsupportedOps", unsupportedOps); + assertEquals(unsupportedOps.toString() + "\n" + ops, ops.size(), unsupportedOps.size()); + unsupportedOps.forEach(op -> assertEquals(CollectionParams.CollectionAction.MERGESHARDS, op.getAction())); + } + + @Test + public void testMixedBounds() throws Exception { + if (cloudManager instanceof SimCloudManager) { + log.warn("Requires SOLR-12208"); + return; + } + + String collectionName = "testMixedBounds_collection"; + CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, + "conf", 2, 2).setMaxShardsPerNode(2); + create.process(solrClient); + CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName, + CloudTestUtils.clusterShape(2, 2)); + + for (int j = 0; j < 10; j++) { + UpdateRequest ureq = new UpdateRequest(); + ureq.setParam("collection", collectionName); + for (int i = 0; i < 100; i++) { + SolrInputDocument doc = new SolrInputDocument("id", "id-" + (i * 100) + "-" + j); + doc.addField("foo", TestUtil.randomSimpleString(random(), 130, 130)); + ureq.add(doc); + } + solrClient.request(ureq); + } + solrClient.commit(collectionName); + + long waitForSeconds = 3 + random().nextInt(5); + + // the trigger is initially disabled so that we have time to add listeners + // and have them capture all events once the trigger is enabled + String setTriggerCommand = "{" + + "'set-trigger' : {" + + "'name' : 'index_size_trigger'," + + "'event' : 'indexSize'," + + "'waitFor' : '" + waitForSeconds + "s'," + + // don't hit this limit when indexing + "'aboveDocs' : 10000," + + // hit this limit when deleting + "'belowDocs' : 100," + + // hit this limit when indexing + "'aboveBytes' : 150000," + + // don't hit this limit when deleting + "'belowBytes' : 10," + + "'enabled' : false," + + "'actions' : [{'name' : 'compute_plan', 'class' : 'solr.ComputePlanAction'}," + + "{'name' : 'execute_plan', 'class' : '" + ExecutePlanAction.class.getName() + "'}]" + + "}}"; + SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand); + NamedList response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + String setListenerCommand = "{" + + "'set-listener' : " + + "{" + + "'name' : 'capturing'," + + "'trigger' : 'index_size_trigger'," + + "'stage' : ['STARTED','ABORTED','SUCCEEDED','FAILED']," + + "'beforeAction' : ['compute_plan','execute_plan']," + + "'afterAction' : ['compute_plan','execute_plan']," + + "'class' : '" + CapturingTriggerListener.class.getName() + "'" + + "}" + + "}"; + req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + setListenerCommand = "{" + + "'set-listener' : " + + "{" + + "'name' : 'finished'," + + "'trigger' : 'index_size_trigger'," + + "'stage' : ['SUCCEEDED']," + + "'class' : '" + FinishedProcessingListener.class.getName() + "'" + + "}" + + "}"; + req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + // now enable the trigger + String resumeTriggerCommand = "{" + + "'resume-trigger' : {" + + "'name' : 'index_size_trigger'" + + "}" + + "}"; + req = createAutoScalingRequest(SolrRequest.METHOD.POST, resumeTriggerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + timeSource.sleep(TimeUnit.MILLISECONDS.convert(waitForSeconds + 1, TimeUnit.SECONDS)); + + boolean await = finished.await(90000 / SPEED, TimeUnit.MILLISECONDS); + assertTrue("did not finish processing in time", await); + assertEquals(1, listenerEvents.size()); + List events = listenerEvents.get("capturing"); + assertNotNull("'capturing' events not found", events); + assertEquals("events: " + events, 6, events.size()); + assertEquals(TriggerEventProcessorStage.STARTED, events.get(0).stage); + assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, events.get(1).stage); + assertEquals(TriggerEventProcessorStage.AFTER_ACTION, events.get(2).stage); + assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, events.get(3).stage); + assertEquals(TriggerEventProcessorStage.AFTER_ACTION, events.get(4).stage); + assertEquals(TriggerEventProcessorStage.SUCCEEDED, events.get(5).stage); + + // collection should have 2 inactive and 4 active shards + CloudTestUtils.waitForState(cloudManager, "failed to create " + collectionName, collectionName, + CloudTestUtils.clusterShape(6, 2, true)); + + // check ops + List ops = (List) events.get(4).event.getProperty(TriggerEvent.REQUESTED_OPS); + assertNotNull("should contain requestedOps", ops); + assertEquals("number of ops", 2, ops.size()); + boolean shard1 = false; + boolean shard2 = false; + for (TriggerEvent.Op op : ops) { + assertEquals(CollectionParams.CollectionAction.SPLITSHARD, op.getAction()); + Set> hints = (Set>)op.getHints().get(Suggester.Hint.COLL_SHARD); + assertNotNull("hints", hints); + assertEquals("hints", 1, hints.size()); + Pair p = hints.iterator().next(); + assertEquals(collectionName, p.first()); + if (p.second().equals("shard1")) { + shard1 = true; + } else if (p.second().equals("shard2")) { + shard2 = true; + } else { + fail("unexpected shard name " + p.second()); + } + } + assertTrue("shard1 should be split", shard1); + assertTrue("shard2 should be split", shard2); + + // now delete most of docs to trigger belowDocs condition + listenerEvents.clear(); + finished = new CountDownLatch(1); + + // suspend the trigger first so that we can safely delete all docs + String suspendTriggerCommand = "{" + + "'suspend-trigger' : {" + + "'name' : 'index_size_trigger'" + + "}" + + "}"; + req = createAutoScalingRequest(SolrRequest.METHOD.POST, suspendTriggerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + for (int j = 0; j < 8; j++) { + UpdateRequest ureq = new UpdateRequest(); + ureq.setParam("collection", collectionName); + for (int i = 0; i < 95; i++) { + ureq.deleteById("id-" + (i * 100) + "-" + j); + } + solrClient.request(ureq); + } + solrClient.commit(collectionName); + + // resume trigger + req = createAutoScalingRequest(SolrRequest.METHOD.POST, resumeTriggerCommand); + response = solrClient.request(req); + assertEquals(response.get("result").toString(), "success"); + + timeSource.sleep(TimeUnit.MILLISECONDS.convert(waitForSeconds + 1, TimeUnit.SECONDS)); + + await = finished.await(90000 / SPEED, TimeUnit.MILLISECONDS); + assertTrue("did not finish processing in time", await); + assertEquals(1, listenerEvents.size()); + events = listenerEvents.get("capturing"); + assertNotNull("'capturing' events not found", events); + assertEquals("events: " + events, 6, events.size()); + assertEquals(TriggerEventProcessorStage.STARTED, events.get(0).stage); + assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, events.get(1).stage); + assertEquals(TriggerEventProcessorStage.AFTER_ACTION, events.get(2).stage); + assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, events.get(3).stage); + assertEquals(TriggerEventProcessorStage.AFTER_ACTION, events.get(4).stage); + assertEquals(TriggerEventProcessorStage.SUCCEEDED, events.get(5).stage); + + // check ops + ops = (List) events.get(4).event.getProperty(TriggerEvent.REQUESTED_OPS); + assertNotNull("should contain requestedOps", ops); + assertTrue("number of ops: " + ops, ops.size() > 0); + for (TriggerEvent.Op op : ops) { + assertEquals(CollectionParams.CollectionAction.MERGESHARDS, op.getAction()); + Set> hints = (Set>)op.getHints().get(Suggester.Hint.COLL_SHARD); + assertNotNull("hints", hints); + assertEquals("hints", 2, hints.size()); + Pair p = hints.iterator().next(); + assertEquals(collectionName, p.first()); + } + + // TODO: fix this once MERGESHARDS is supported + List unsupportedOps = (List)events.get(2).context.get("properties.unsupportedOps"); + assertNotNull("should have unsupportedOps", unsupportedOps); + assertEquals(unsupportedOps.toString() + "\n" + ops, ops.size(), unsupportedOps.size()); + unsupportedOps.forEach(op -> assertEquals(CollectionParams.CollectionAction.MERGESHARDS, op.getAction())); + } + + private Map createTriggerProps(long waitForSeconds) { + Map props = new HashMap<>(); + props.put("event", "indexSize"); + props.put("waitFor", waitForSeconds); + props.put("enabled", true); + props.put(IndexSizeTrigger.ABOVE_DOCS_PROP, 10); + props.put(IndexSizeTrigger.BELOW_DOCS_PROP, 2); + List> actions = new ArrayList<>(3); + Map map = new HashMap<>(2); + map.put("name", "compute_plan"); + map.put("class", "solr.ComputePlanAction"); + actions.add(map); + map = new HashMap<>(2); + map.put("name", "execute_plan"); + map.put("class", "solr.ExecutePlanAction"); + actions.add(map); + props.put("actions", actions); + return props; + } +} diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java index 2d084b8a214..cd52785de35 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java @@ -91,7 +91,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase { long eventTimeNanos = event.getEventTime(); long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS; if (currentTimeNanos - eventTimeNanos <= waitForNanos) { - fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos); + fail("processor was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos); } } else { fail("NodeAddedTrigger was fired more than once!"); diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java index ffcab4dd8fd..164db8ff674 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ScheduledMaintenanceTriggerTest.java @@ -165,7 +165,7 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase { .setShardName("shard1"); split1.process(solrClient); CloudTestUtils.waitForState(cloudManager, "failed to split " + collection1, collection1, - CloudTestUtils.clusterShape(3, 1)); + CloudTestUtils.clusterShape(3, 1, true)); String setListenerCommand = "{" + "'set-listener' : " + diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java index 04dc96f9c97..9641552e1fa 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java @@ -24,8 +24,10 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Random; +import java.util.Set; import java.util.TreeMap; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentSkipListMap; @@ -42,8 +44,11 @@ import org.apache.solr.client.solrj.cloud.DistributedQueueFactory; import org.apache.solr.client.solrj.cloud.DistribStateManager; import org.apache.solr.client.solrj.cloud.NodeStateProvider; import org.apache.solr.client.solrj.cloud.SolrCloudManager; +import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo; import org.apache.solr.client.solrj.impl.ClusterStateProvider; +import org.apache.solr.client.solrj.request.AbstractUpdateRequest; import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.QueryRequest; import org.apache.solr.client.solrj.request.RequestWriter; import org.apache.solr.client.solrj.request.UpdateRequest; import org.apache.solr.client.solrj.response.RequestStatusState; @@ -55,6 +60,7 @@ import org.apache.solr.cloud.autoscaling.OverseerTriggerThread; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.cloud.ClusterState; +import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.rule.ImplicitSnitch; @@ -240,6 +246,67 @@ public class SimCloudManager implements SolrCloudManager { return values; } + public String dumpClusterState(boolean withCollections) throws Exception { + StringBuilder sb = new StringBuilder(); + sb.append("#######################################\n"); + sb.append("############ CLUSTER STATE ############\n"); + sb.append("#######################################\n"); + sb.append("## Live nodes:\t\t" + getLiveNodesSet().size() + "\n"); + int emptyNodes = 0; + int maxReplicas = 0; + int minReplicas = Integer.MAX_VALUE; + Map> replicaStates = new TreeMap<>(); + int numReplicas = 0; + for (String node : getLiveNodesSet().get()) { + List replicas = getSimClusterStateProvider().simGetReplicaInfos(node); + numReplicas += replicas.size(); + if (replicas.size() > maxReplicas) { + maxReplicas = replicas.size(); + } + if (minReplicas > replicas.size()) { + minReplicas = replicas.size(); + } + for (ReplicaInfo ri : replicas) { + replicaStates.computeIfAbsent(ri.getCollection(), c -> new TreeMap<>()) + .computeIfAbsent(ri.getState(), s -> new AtomicInteger()) + .incrementAndGet(); + } + if (replicas.isEmpty()) { + emptyNodes++; + } + } + if (minReplicas == Integer.MAX_VALUE) { + minReplicas = 0; + } + sb.append("## Empty nodes:\t" + emptyNodes + "\n"); + Set deadNodes = getSimNodeStateProvider().simGetDeadNodes(); + sb.append("## Dead nodes:\t\t" + deadNodes.size() + "\n"); + deadNodes.forEach(n -> sb.append("##\t\t" + n + "\n")); + sb.append("## Collections:\t" + getSimClusterStateProvider().simListCollections() + "\n"); + if (withCollections) { + ClusterState state = clusterStateProvider.getClusterState(); + state.forEachCollection(coll -> sb.append(coll.toString() + "\n")); + } + sb.append("## Max replicas per node:\t" + maxReplicas + "\n"); + sb.append("## Min replicas per node:\t" + minReplicas + "\n"); + sb.append("## Total replicas:\t\t" + numReplicas + "\n"); + replicaStates.forEach((c, map) -> { + AtomicInteger repCnt = new AtomicInteger(); + map.forEach((s, cnt) -> repCnt.addAndGet(cnt.get())); + sb.append("## * " + c + "\t\t" + repCnt.get() + "\n"); + map.forEach((s, cnt) -> sb.append("##\t\t- " + String.format(Locale.ROOT, "%-12s %4d", s, cnt.get()) + "\n")); + }); + sb.append("######### Solr op counts ##########\n"); + simGetOpCounts().forEach((k, cnt) -> sb.append("##\t\t- " + String.format(Locale.ROOT, "%-14s %4d", k, cnt.get()) + "\n")); + sb.append("######### Autoscaling event counts ###########\n"); + Map> counts = simGetEventCounts(); + counts.forEach((trigger, map) -> { + sb.append("## * Trigger: " + trigger + "\n"); + map.forEach((s, cnt) -> sb.append("##\t\t- " + String.format(Locale.ROOT, "%-11s %4d", s, cnt.get()) + "\n")); + }); + return sb.toString(); + } + /** * Get the instance of {@link SolrResourceLoader} that is used by the cluster components. */ @@ -333,6 +400,17 @@ public class SimCloudManager implements SolrCloudManager { return new SolrClient() { @Override public NamedList request(SolrRequest request, String collection) throws SolrServerException, IOException { + if (collection != null) { + if (request instanceof AbstractUpdateRequest) { + ((AbstractUpdateRequest)request).setParam("collection", collection); + } else if (request instanceof QueryRequest) { + ModifiableSolrParams params = new ModifiableSolrParams(request.getParams()); + params.set("collection", collection); + request = new QueryRequest(params); + } else { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "when collection != null only UpdateRequest and QueryRequest are supported: request=" + request + ", collection=" + collection); + } + } SolrResponse rsp = SimCloudManager.this.request(request); return rsp.getResponse(); } @@ -508,14 +586,17 @@ public class SimCloudManager implements SolrCloudManager { incrementCount("update"); // support only updates to the system collection UpdateRequest ureq = (UpdateRequest)req; - if (ureq.getCollection() == null || !ureq.getCollection().equals(CollectionAdminParams.SYSTEM_COLL)) { - throw new UnsupportedOperationException("Only .system updates are supported but got: " + req); + String collection = ureq.getCollection(); + if (collection != null && !collection.equals(CollectionAdminParams.SYSTEM_COLL)) { + // simulate an update + return clusterStateProvider.simUpdate(ureq); + } else { + List docs = ureq.getDocuments(); + if (docs != null) { + systemColl.addAll(docs); + } + return new UpdateResponse(); } - List docs = ureq.getDocuments(); - if (docs != null) { - systemColl.addAll(docs); - } - return new UpdateResponse(); } // support only a specific subset of collection admin ops if (!(req instanceof CollectionAdminRequest)) { @@ -560,8 +641,12 @@ public class SimCloudManager implements SolrCloudManager { } break; case DELETE: - clusterStateProvider.simDeleteCollection(req.getParams().get(CommonParams.NAME), - req.getParams().get(CommonAdminParams.ASYNC), results); + try { + clusterStateProvider.simDeleteCollection(req.getParams().get(CommonParams.NAME), + req.getParams().get(CommonAdminParams.ASYNC), results); + } catch (Exception e) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); + } break; case LIST: results.add("collections", clusterStateProvider.simListCollections()); diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java index f6762fcc844..9b3782ad5ca 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java @@ -47,6 +47,8 @@ import org.apache.solr.client.solrj.cloud.autoscaling.Suggestion; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData; import org.apache.solr.client.solrj.impl.ClusterStateProvider; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.client.solrj.response.UpdateResponse; import org.apache.solr.cloud.ActionThrottle; import org.apache.solr.cloud.api.collections.AddReplicaCmd; import org.apache.solr.cloud.api.collections.Assign; @@ -57,6 +59,7 @@ import org.apache.solr.cloud.overseer.ClusterStateMutator; import org.apache.solr.cloud.overseer.CollectionMutator; import org.apache.solr.cloud.overseer.ZkWriteCommand; import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.DocRouter; @@ -241,7 +244,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @return true if a node existed and was removed */ public boolean simRemoveNode(String nodeId) throws Exception { - lock.lock(); + lock.lockInterruptibly(); try { Set collections = new HashSet<>(); // mark every replica on that node as down @@ -296,14 +299,14 @@ public class SimClusterStateProvider implements ClusterStateProvider { liveNodes.add(nodeId); createEphemeralLiveNode(nodeId); Set collections = new HashSet<>(); - lock.lock(); + lock.lockInterruptibly(); try { setReplicaStates(nodeId, Replica.State.RECOVERING, collections); } finally { lock.unlock(); } cloudManager.getTimeSource().sleep(1000); - lock.lock(); + lock.lockInterruptibly(); try { setReplicaStates(nodeId, Replica.State.ACTIVE, collections); } finally { @@ -389,7 +392,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { throw new Exception("Wrong node (not " + nodeId + "): " + replicaInfo); } - lock.lock(); + lock.lockInterruptibly(); try { opDelay(replicaInfo.getCollection(), CollectionParams.CollectionAction.ADDREPLICA.name()); @@ -435,7 +438,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { */ public void simRemoveReplica(String nodeId, String coreNodeName) throws Exception { List replicas = nodeReplicaMap.computeIfAbsent(nodeId, n -> new ArrayList<>()); - lock.lock(); + lock.lockInterruptibly(); try { for (int i = 0; i < replicas.size(); i++) { if (coreNodeName.equals(replicas.get(i).getName())) { @@ -638,6 +641,9 @@ public class SimClusterStateProvider implements ClusterStateProvider { replicaNum.getAndIncrement()); try { replicaProps.put(ZkStateReader.CORE_NAME_PROP, coreName); + replicaProps.put("SEARCHER.searcher.deletedDocs", 0); + replicaProps.put("SEARCHER.searcher.numDocs", 0); + replicaProps.put("SEARCHER.searcher.maxDoc", 0); ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0), coreName, collectionName, pos.shard, pos.type, pos.node, replicaProps); cloudManager.submit(() -> { @@ -662,6 +668,8 @@ public class SimClusterStateProvider implements ClusterStateProvider { } }); }); + // force recreation of collection states + collectionsStatesRef.set(null); simRunLeaderElection(Collections.singleton(collectionName), true); if (waitForFinalState) { boolean finished = finalStateLatch.await(cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, 60, TimeUnit.MILLISECONDS), @@ -680,11 +688,11 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @param async async id * @param results results of the operation */ - public void simDeleteCollection(String collection, String async, NamedList results) throws IOException { + public void simDeleteCollection(String collection, String async, NamedList results) throws Exception { if (async != null) { results.add(CoreAdminParams.REQUESTID, async); } - lock.lock(); + lock.lockInterruptibly(); try { collProperties.remove(collection); sliceProperties.remove(collection); @@ -722,7 +730,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { * Remove all collections. */ public void simDeleteAllCollections() throws Exception { - lock.lock(); + lock.lockInterruptibly(); try { nodeReplicaMap.clear(); collProperties.clear(); @@ -797,7 +805,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { String collectionName = message.getStr(COLLECTION_PROP); String sliceName = message.getStr(SHARD_ID_PROP); ClusterState clusterState = getClusterState(); - lock.lock(); + lock.lockInterruptibly(); try { ZkWriteCommand cmd = new CollectionMutator(cloudManager).createShard(clusterState, message); if (cmd.noop) { @@ -865,6 +873,10 @@ public class SimClusterStateProvider implements ClusterStateProvider { AtomicReference sliceName = new AtomicReference<>(); sliceName.set(message.getStr(SHARD_ID_PROP)); String splitKey = message.getStr("split.key"); + + // always invalidate cached collection states to get up-to-date metrics + collectionsStatesRef.set(null); + ClusterState clusterState = getClusterState(); DocCollection collection = clusterState.getCollection(collectionName); Slice parentSlice = SplitShardCmd.getParentSlice(clusterState, collectionName, sliceName, splitKey); @@ -887,6 +899,18 @@ public class SimClusterStateProvider implements ClusterStateProvider { PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getLastSessionWrapper(true); if (sessionWrapper != null) sessionWrapper.release(); + // adjust numDocs / deletedDocs / maxDoc + Replica leader = parentSlice.getLeader(); + // XXX leader election may not have happened yet - should we require it? + if (leader == null) { + leader = parentSlice.getReplicas().iterator().next(); + } + String numDocsStr = leader.getStr("SEARCHER.searcher.numDocs", "0"); + long numDocs = Long.parseLong(numDocsStr); + long newNumDocs = numDocs / subSlices.size(); + long remainder = numDocs % subSlices.size(); + String remainderSlice = null; + for (ReplicaPosition replicaPosition : replicaPositions) { String subSliceName = replicaPosition.shard; String subShardNodeName = replicaPosition.node; @@ -897,15 +921,32 @@ public class SimClusterStateProvider implements ClusterStateProvider { replicaProps.put(ZkStateReader.REPLICA_TYPE, replicaPosition.type.toString()); replicaProps.put(ZkStateReader.BASE_URL_PROP, Utils.getBaseUrlForNodeName(subShardNodeName, "http")); + long replicasNumDocs = newNumDocs; + if (remainderSlice == null) { + remainderSlice = subSliceName; + } + if (remainderSlice.equals(subSliceName)) { // only add to one sub slice + replicasNumDocs += remainder; + } + replicaProps.put("SEARCHER.searcher.numDocs", replicasNumDocs); + replicaProps.put("SEARCHER.searcher.maxDoc", replicasNumDocs); + replicaProps.put("SEARCHER.searcher.deletedDocs", 0); + ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0), solrCoreName, collectionName, replicaPosition.shard, replicaPosition.type, subShardNodeName, replicaProps); simAddReplica(replicaPosition.node, ri, false); } // mark the old slice as inactive - Map props = sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>()) - .computeIfAbsent(sliceName.get(), s -> new ConcurrentHashMap<>()); - props.put(ZkStateReader.STATE_PROP, Slice.State.INACTIVE.toString()); - props.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs())); + lock.lockInterruptibly(); + try { + Map props = sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>()) + .computeIfAbsent(sliceName.get(), s -> new ConcurrentHashMap<>()); + props.put(ZkStateReader.STATE_PROP, Slice.State.INACTIVE.toString()); + props.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs())); + // XXX also mark replicas as down? currently SplitShardCmd doesn't do this + } finally { + lock.unlock(); + } // add slice props for (int i = 0; i < subRanges.size(); i++) { String subSlice = subSlices.get(i); @@ -915,8 +956,9 @@ public class SimClusterStateProvider implements ClusterStateProvider { sliceProps.put(Slice.RANGE, range); sliceProps.put(Slice.PARENT, sliceName.get()); sliceProps.put(ZkStateReader.STATE_PROP, Slice.State.ACTIVE.toString()); - props.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs())); + sliceProps.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs())); } + collectionsStatesRef.set(null); simRunLeaderElection(Collections.singleton(collectionName), true); results.add("success", ""); @@ -945,7 +987,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { opDelay(collectionName, CollectionParams.CollectionAction.DELETESHARD.name()); - lock.lock(); + lock.lockInterruptibly(); try { sliceProperties.computeIfAbsent(collectionName, coll -> new ConcurrentHashMap<>()).remove(sliceName); nodeReplicaMap.forEach((n, replicas) -> { @@ -966,6 +1008,122 @@ public class SimClusterStateProvider implements ClusterStateProvider { } } + /** + * Simulate an update by modifying replica metrics. + * The following core metrics are updated: + *
    + *
  • SEARCHER.searcher.numDocs - increased by added docs, decreased by deleteById and deleteByQuery
  • + *
  • SEARCHER.searcher.deletedDocs - decreased by deleteById and deleteByQuery by up to numDocs
  • + *
  • SEARCHER.searcher.maxDoc - always increased by the number of added docs.
  • + *
+ *

IMPORTANT limitations:

+ *
    + *
  • document replacements are always counted as new docs
  • + *
  • delete by ID always succeeds (unless numDocs == 0)
  • + *
  • deleteByQuery is not supported unless the query is *:*
  • + *
+ * @param req update request. This request MUST have the collection param set. + * @return {@link UpdateResponse} + * @throws SolrException on errors, such as nonexistent collection or unsupported deleteByQuery + */ + public UpdateResponse simUpdate(UpdateRequest req) throws SolrException, InterruptedException, IOException { + String collection = req.getCollection(); + if (collection == null) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection not set"); + } + if (!simListCollections().contains(collection)) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection '" + collection + "' doesn't exist"); + } + // always reset first to get the current metrics - it's easier than to keep matching + // Replica with ReplicaInfo where the current real counts are stored + collectionsStatesRef.set(null); + DocCollection coll = getClusterState().getCollection(collection); + DocRouter router = coll.getRouter(); + + boolean modified = false; + + lock.lockInterruptibly(); + try { + List deletes = req.getDeleteById(); + if (deletes != null && !deletes.isEmpty()) { + for (String id : deletes) { + Slice s = router.getTargetSlice(id, null, null, req.getParams(), coll); + // NOTE: we don't use getProperty because it uses PROPERTY_PROP_PREFIX + String numDocsStr = s.getLeader().getStr("SEARCHER.searcher.numDocs"); + if (numDocsStr == null) { + LOG.debug("-- no docs in " + s.getLeader()); + continue; + } + long numDocs = Long.parseLong(numDocsStr); + if (numDocs == 0) { + LOG.debug("-- attempting to delete nonexistent doc " + id + " from " + s.getLeader()); + continue; + } + if (numDocsStr != null) { + modified = true; + try { + simSetShardValue(collection, s.getName(), "SEARCHER.searcher.deletedDocs", 1, true, false); + simSetShardValue(collection, s.getName(), "SEARCHER.searcher.numDocs", -1, true, false); + } catch (Exception e) { + throw new IOException(e); + } + } + } + } + deletes = req.getDeleteQuery(); + if (deletes != null && !deletes.isEmpty()) { + for (String q : deletes) { + if (!"*:*".equals(q)) { + throw new UnsupportedOperationException("Only '*:*' query is supported in deleteByQuery"); + } + for (Slice s : coll.getSlices()) { + String numDocsStr = s.getLeader().getStr("SEARCHER.searcher.numDocs"); + if (numDocsStr == null) { + continue; + } + long numDocs = Long.parseLong(numDocsStr); + if (numDocs == 0) { + continue; + } + modified = true; + try { + simSetShardValue(collection, s.getName(), "SEARCHER.searcher.deletedDocs", numDocs, false, false); + simSetShardValue(collection, s.getName(), "SEARCHER.searcher.numDocs", 0, false, false); + } catch (Exception e) { + throw new IOException(e); + } + } + } + } + List docs = req.getDocuments(); + if (docs != null && !docs.isEmpty()) { + for (SolrInputDocument doc : docs) { + String id = (String) doc.getFieldValue("id"); + if (id == null) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Document without id: " + doc); + } + Slice s = router.getTargetSlice(id, null, null, req.getParams(), coll); + modified = true; + try { + simSetShardValue(collection, s.getName(), "SEARCHER.searcher.numDocs", 1, true, false); + simSetShardValue(collection, s.getName(), "SEARCHER.searcher.maxDoc", 1, true, false); + // Policy reuses this value and expects it to be in GB units!!! + // the idea here is to increase the index size by 500 bytes with each doc + // simSetShardValue(collection, s.getName(), "INDEX.sizeInBytes", 500, true, false); + } catch (Exception e) { + throw new IOException(e); + } + } + } + if (modified) { + collectionsStatesRef.set(null); + } + } finally { + lock.unlock(); + } + return new UpdateResponse(); + } + /** * Saves cluster properties to clusterprops.json. * @return current properties @@ -988,7 +1146,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @param properties properties to set */ public void simSetClusterProperties(Map properties) throws Exception { - lock.lock(); + lock.lockInterruptibly(); try { clusterProperties.clear(); if (properties != null) { @@ -1007,7 +1165,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @param value property value */ public void simSetClusterProperty(String key, Object value) throws Exception { - lock.lock(); + lock.lockInterruptibly(); try { if (value != null) { clusterProperties.put(key, value); @@ -1026,7 +1184,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @param properties properties */ public void simSetCollectionProperties(String coll, Map properties) throws Exception { - lock.lock(); + lock.lockInterruptibly(); try { if (properties == null) { collProperties.remove(coll); @@ -1049,7 +1207,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { */ public void simSetCollectionProperty(String coll, String key, String value) throws Exception { Map props = collProperties.computeIfAbsent(coll, c -> new HashMap<>()); - lock.lock(); + lock.lockInterruptibly(); try { if (value == null) { props.remove(key); @@ -1070,7 +1228,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { */ public void simSetSliceProperties(String coll, String slice, Map properties) throws Exception { Map sliceProps = sliceProperties.computeIfAbsent(coll, c -> new HashMap<>()).computeIfAbsent(slice, s -> new HashMap<>()); - lock.lock(); + lock.lockInterruptibly(); try { sliceProps.clear(); if (properties != null) { @@ -1089,7 +1247,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @param value property value */ public void simSetCollectionValue(String collection, String key, Object value) throws Exception { - simSetCollectionValue(collection, key, value, false); + simSetCollectionValue(collection, key, value, false, false); } /** @@ -1100,8 +1258,8 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @param divide if the value is a {@link Number} and this param is true, then the value will be evenly * divided by the number of replicas. */ - public void simSetCollectionValue(String collection, String key, Object value, boolean divide) throws Exception { - simSetShardValue(collection, null, key, value, divide); + public void simSetCollectionValue(String collection, String key, Object value, boolean delta, boolean divide) throws Exception { + simSetShardValue(collection, null, key, value, delta, divide); } /** @@ -1112,7 +1270,7 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @param value property value */ public void simSetShardValue(String collection, String shard, String key, Object value) throws Exception { - simSetShardValue(collection, shard, key, value, false); + simSetShardValue(collection, shard, key, value, false, false); } /** @@ -1121,10 +1279,12 @@ public class SimClusterStateProvider implements ClusterStateProvider { * @param shard shard name. If null then all shards will be affected. * @param key property name * @param value property value + * @param delta if true then treat the numeric value as delta to add to the existing value + * (or set the value to delta if missing) * @param divide if the value is a {@link Number} and this is true, then the value will be evenly * divided by the number of replicas. */ - public void simSetShardValue(String collection, String shard, String key, Object value, boolean divide) throws Exception { + public void simSetShardValue(String collection, String shard, String key, Object value, boolean delta, boolean divide) throws Exception { List infos = new ArrayList<>(); nodeReplicaMap.forEach((n, replicas) -> { replicas.forEach(r -> { @@ -1140,14 +1300,38 @@ public class SimClusterStateProvider implements ClusterStateProvider { throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection " + collection + " doesn't exist."); } if (divide && value != null && (value instanceof Number)) { - value = ((Number)value).doubleValue() / infos.size(); + if ((value instanceof Long) || (value instanceof Integer)) { + value = ((Number) value).longValue() / infos.size(); + } else { + value = ((Number) value).doubleValue() / infos.size(); + } } for (ReplicaInfo r : infos) { synchronized (r) { if (value == null) { r.getVariables().remove(key); } else { - r.getVariables().put(key, value); + if (delta) { + Object prevValue = r.getVariables().get(key); + if (prevValue != null) { + if ((prevValue instanceof Number) && (value instanceof Number)) { + if (((prevValue instanceof Long) || (prevValue instanceof Integer)) && + ((value instanceof Long) || (value instanceof Integer))) { + Long newValue = ((Number)prevValue).longValue() + ((Number)value).longValue(); + r.getVariables().put(key, newValue); + } else { + Double newValue = ((Number)prevValue).doubleValue() + ((Number)value).doubleValue(); + r.getVariables().put(key, newValue); + } + } else { + throw new UnsupportedOperationException("delta cannot be applied to non-numeric values: " + prevValue + " and " + value); + } + } else { + r.getVariables().put(key, value); + } + } else { + r.getVariables().put(key, value); + } } } } @@ -1172,9 +1356,9 @@ public class SimClusterStateProvider implements ClusterStateProvider { * List collections. * @return list of existing collections. */ - public List simListCollections() { + public List simListCollections() throws InterruptedException { final Set collections = new HashSet<>(); - lock.lock(); + lock.lockInterruptibly(); try { nodeReplicaMap.forEach((n, replicas) -> { replicas.forEach(ri -> collections.add(ri.getCollection())); @@ -1216,6 +1400,8 @@ public class SimClusterStateProvider implements ClusterStateProvider { return state; } + // this method uses a simple cache in collectionsStatesRef. Operations that modify + // cluster state should always reset this cache so that the changes become visible private Map getCollectionStates() { Map collectionStates = collectionsStatesRef.get(); if (collectionStates != null) { @@ -1263,7 +1449,9 @@ public class SimClusterStateProvider implements ClusterStateProvider { slices.put(s, slice); }); Map collProps = collProperties.computeIfAbsent(coll, c -> new ConcurrentHashMap<>()); - DocCollection dc = new DocCollection(coll, slices, collProps, DocRouter.DEFAULT, clusterStateVersion, ZkStateReader.CLUSTER_STATE); + Map routerProp = (Map) collProps.getOrDefault(DocCollection.DOC_ROUTER, Collections.singletonMap("name", DocRouter.DEFAULT_NAME)); + DocRouter router = DocRouter.getDocRouter((String)routerProp.getOrDefault("name", DocRouter.DEFAULT_NAME)); + DocCollection dc = new DocCollection(coll, slices, collProps, router, clusterStateVersion, ZkStateReader.CLUSTER_STATE); res.put(coll, dc); }); collectionsStatesRef.set(res); diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java index 6d1f68a589b..b9169eb2263 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java @@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory; /** * Simulated {@link NodeStateProvider}. * Note: in order to setup node-level metrics use {@link #simSetNodeValues(String, Map)}. However, in order - * to setup core-level metrics use {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean)}. + * to setup core-level metrics use {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean, boolean)}. */ public class SimNodeStateProvider implements NodeStateProvider { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -204,7 +204,7 @@ public class SimNodeStateProvider implements NodeStateProvider { /** * Simulate getting replica metrics values. This uses per-replica properties set in - * {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean)} and + * {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean, boolean)} and * similar methods. * @param node node id * @param tags metrics names diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java index f18234a9321..757e2975cd9 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java @@ -22,15 +22,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo; import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; @@ -79,59 +73,7 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 { public void tearDown() throws Exception { super.tearDown(); if (cluster != null) { - log.info("\n"); - log.info("#############################################"); - log.info("############ FINAL CLUSTER STATS ############"); - log.info("#############################################\n"); - log.info("## Live nodes:\t\t" + cluster.getLiveNodesSet().size()); - int emptyNodes = 0; - int maxReplicas = 0; - int minReplicas = Integer.MAX_VALUE; - Map> replicaStates = new TreeMap<>(); - int numReplicas = 0; - for (String node : cluster.getLiveNodesSet().get()) { - List replicas = cluster.getSimClusterStateProvider().simGetReplicaInfos(node); - numReplicas += replicas.size(); - if (replicas.size() > maxReplicas) { - maxReplicas = replicas.size(); - } - if (minReplicas > replicas.size()) { - minReplicas = replicas.size(); - } - for (ReplicaInfo ri : replicas) { - replicaStates.computeIfAbsent(ri.getCollection(), c -> new TreeMap<>()) - .computeIfAbsent(ri.getState(), s -> new AtomicInteger()) - .incrementAndGet(); - } - if (replicas.isEmpty()) { - emptyNodes++; - } - } - if (minReplicas == Integer.MAX_VALUE) { - minReplicas = 0; - } - log.info("## Empty nodes:\t" + emptyNodes); - Set deadNodes = cluster.getSimNodeStateProvider().simGetDeadNodes(); - log.info("## Dead nodes:\t\t" + deadNodes.size()); - deadNodes.forEach(n -> log.info("##\t\t" + n)); - log.info("## Collections:\t" + cluster.getSimClusterStateProvider().simListCollections()); - log.info("## Max replicas per node:\t" + maxReplicas); - log.info("## Min replicas per node:\t" + minReplicas); - log.info("## Total replicas:\t\t" + numReplicas); - replicaStates.forEach((c, map) -> { - AtomicInteger repCnt = new AtomicInteger(); - map.forEach((s, cnt) -> repCnt.addAndGet(cnt.get())); - log.info("## * " + c + "\t\t" + repCnt.get()); - map.forEach((s, cnt) -> log.info("##\t\t- " + String.format(Locale.ROOT, "%-12s %4d", s, cnt.get()))); - }); - log.info("######### Final Solr op counts ##########"); - cluster.simGetOpCounts().forEach((k, cnt) -> log.info("##\t\t- " + String.format(Locale.ROOT, "%-14s %4d", k, cnt.get()))); - log.info("######### Autoscaling event counts ###########"); - Map> counts = cluster.simGetEventCounts(); - counts.forEach((trigger, map) -> { - log.info("## * Trigger: " + trigger); - map.forEach((s, cnt) -> log.info("##\t\t- " + String.format(Locale.ROOT, "%-11s %4d", s, cnt.get()))); - }); + log.info(cluster.dumpClusterState(false)); } } diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java index 14ac40f23c6..129f18c0079 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java @@ -540,7 +540,7 @@ public class TestLargeCluster extends SimSolrCloudTestCase { String metricName = "QUERY./select.requestTimes:1minRate"; // simulate search traffic - cluster.getSimClusterStateProvider().simSetShardValue(collectionName, "shard1", metricName, 40, true); + cluster.getSimClusterStateProvider().simSetShardValue(collectionName, "shard1", metricName, 40, false, true); // now define the trigger. doing it earlier may cause partial events to be generated (where only some // nodes / replicas exceeded the threshold). @@ -592,7 +592,19 @@ public class TestLargeCluster extends SimSolrCloudTestCase { ops.forEach(op -> { assertEquals(CollectionParams.CollectionAction.ADDREPLICA, op.getAction()); assertEquals(1, op.getHints().size()); - Pair hint = (Pair)op.getHints().get(Suggester.Hint.COLL_SHARD); + Object o = op.getHints().get(Suggester.Hint.COLL_SHARD); + // this may be a pair or a HashSet of pairs with size 1 + Pair hint = null; + if (o instanceof Pair) { + hint = (Pair)o; + } else if (o instanceof Set) { + assertEquals("unexpected number of hints: " + o, 1, ((Set)o).size()); + o = ((Set)o).iterator().next(); + assertTrue("unexpected hint: " + o, o instanceof Pair); + hint = (Pair)o; + } else { + fail("unexpected hints: " + o); + } assertNotNull(hint); assertEquals(collectionName, hint.first()); assertEquals("shard1", hint.second()); diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java index 31e36367cbe..c898dbc213c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java +++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java @@ -1192,7 +1192,7 @@ public class TestTriggerIntegration extends SimSolrCloudTestCase { // solrClient.query(COLL1, query); // } - cluster.getSimClusterStateProvider().simSetCollectionValue(COLL1, "QUERY./select.requestTimes:1minRate", 500, true); + cluster.getSimClusterStateProvider().simSetCollectionValue(COLL1, "QUERY./select.requestTimes:1minRate", 500, false, true); boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS); assertTrue("The trigger did not fire at all", await); diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc index 3165669fee7..432a5e9eeb0 100644 --- a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc +++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc @@ -34,6 +34,8 @@ Currently the following event types (and corresponding trigger implementations) * `nodeAdded`: generated when a new node joins the cluster * `nodeLost`: generated when a node leaves the cluster * `metric`: generated when the configured metric crosses a configured lower or upper threshold value +* `indexSize`: generated when a shard size (defined as index size in bytes or number of documents) +exceeds upper or lower threshold values * `searchRate`: generated when the 1-minute average search rate exceeds configured upper threshold * `scheduled`: generated according to a scheduled time period such as every 24 hours etc @@ -105,6 +107,81 @@ This trigger supports the following configuration: } ---- +== Index Size Trigger +This trigger can be used for monitoring the size of collection shards, measured either by the +number of documents in a shard or the physical size of the shard's index in bytes. + +When either of the upper thresholds is exceeded the trigger will generate an event with +a (configurable) requested operation to perform on the offending shards - by default +this is a SPLITSHARD operation. + +Similarly, when either of the lower thresholds is exceeded the trigger will generate an +event with a (configurable) requested operation to perform on two of the smallest +shards - by default this is a MERGESHARDS operation (which is currently ignored because +it's not yet implemented - SOLR-9407) + +Additionally, monitoring can be restricted to a list of collections - by default +all collections are monitored. + +This trigger supports the following configuration parameters (all thresholds are exclusive): + +`aboveBytes`:: upper threshold in bytes. This value is compared to the `INDEX.sizeInBytes` metric. + +`belowBytes`:: lower threshold in bytes. Note that this value should be at least 2x smaller than +`aboveBytes` + +`aboveDocs`:: upper threshold expressed as the number of documents. This value is compared with `SEARCHER.searcher.numDocs` metric. +Note: due to the way Lucene indexes work a shard may exceed the `aboveBytes` threshold +even if the number of documents is relatively small, because replaced and deleted documents keep +occupying disk space until they are actually removed during Lucene index merging. + +`belowDocs`:: lower threshold expressed as the number of documents. + +`aboveOp`:: operation to request when an upper threshold is exceeded. If not specified the +default value is `SPLITSHARD`. + +`belowOp`:: operation to request when a lower threshold is exceeded. If not specified +the default value is `MERGESHARDS` (but see the note above). + +`collections`:: comma-separated list of collection names that this trigger should monitor. If not +specified or empty all collections are monitored. + +Events generated by this trigger contain additional details about the shards +that exceeded thresholds and the types of violations (upper / lower bounds, bytes / docs metrics). + +.Example: +This configuration specifies an index size trigger that monitors collections "test1" and "test2", +with both bytes (1GB) and number of docs (1 mln) upper limits, and a custom `belowOp` +operation `NONE` (which still can be monitored and acted upon by an appropriate trigger listener): + +[source,json] +---- +{ + "set-trigger": { + "name" : "index_size_trigger", + "event" : "indexSize", + "collections" : "test1,test2", + "aboveBytes" : 1000000000, + "aboveDocs" : 1000000000, + "belowBytes" : 200000, + "belowDocs" : 200000, + "belopOp" : "NONE", + "waitFor" : "1m", + "enabled" : true, + "actions" : [ + { + "name" : "compute_plan", + "class": "solr.ComputePlanAction" + }, + { + "name" : "execute_plan", + "class": "solr.ExecutePlanAction" + } + ] + } +} +---- + == Search Rate Trigger The search rate trigger can be used for monitoring 1-minute average search rates in a selected diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java index bee69c83aaa..9496b0ff1d2 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java @@ -466,7 +466,10 @@ public class Policy implements MapWriter { static { ops.put(CollectionAction.ADDREPLICA, () -> new AddReplicaSuggester()); + ops.put(CollectionAction.DELETEREPLICA, () -> new UnsupportedSuggester(CollectionAction.DELETEREPLICA)); ops.put(CollectionAction.MOVEREPLICA, () -> new MoveReplicaSuggester()); + ops.put(CollectionAction.SPLITSHARD, () -> new SplitShardSuggester()); + ops.put(CollectionAction.MERGESHARDS, () -> new UnsupportedSuggester(CollectionAction.MERGESHARDS)); } public Map> getPolicies() { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java index 8c1fba3f22b..e1d8281592b 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java @@ -66,6 +66,10 @@ public class ReplicaInfo implements MapWriter { this.node = node; } + public Object clone() { + return new ReplicaInfo(name, core, collection, shard, type, node, variables); + } + @Override public void writeMap(EntryWriter ew) throws IOException { ew.put(name, (MapWriter) ew1 -> { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SplitShardSuggester.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SplitShardSuggester.java new file mode 100644 index 00000000000..2a42d27a909 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/SplitShardSuggester.java @@ -0,0 +1,43 @@ +/* + * 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.solr.client.solrj.cloud.autoscaling; + +import java.util.Collections; +import java.util.Set; + +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.common.util.Pair; + +/** + * This suggester produces a SPLITSHARD request using provided {@link org.apache.solr.client.solrj.cloud.autoscaling.Suggester.Hint#COLL_SHARD} value. + */ +class SplitShardSuggester extends Suggester { + + @Override + SolrRequest init() { + Set> shards = (Set>) hints.getOrDefault(Hint.COLL_SHARD, Collections.emptySet()); + if (shards.isEmpty()) { + throw new RuntimeException("split-shard requires 'collection' and 'shard'"); + } + if (shards.size() > 1) { + throw new RuntimeException("split-shard requires exactly one pair of 'collection' and 'shard'"); + } + Pair collShard = shards.iterator().next(); + return CollectionAdminRequest.splitShard(collShard.first()).setShardName(collShard.second()); + } +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java index 96bc773c111..a983bf02602 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/TriggerEventType.java @@ -28,5 +28,6 @@ public enum TriggerEventType { SEARCHRATE, INDEXRATE, INVALID, - METRIC + METRIC, + INDEXSIZE } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/UnsupportedSuggester.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/UnsupportedSuggester.java new file mode 100644 index 00000000000..9d44ae41128 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/UnsupportedSuggester.java @@ -0,0 +1,59 @@ +/* + * 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.solr.client.solrj.cloud.autoscaling; + +import java.lang.invoke.MethodHandles; + +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.common.params.CollectionParams; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This suggester simply logs the request but does not produce any suggestions. + */ +public class UnsupportedSuggester extends Suggester { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final CollectionParams.CollectionAction action; + + public static UnsupportedSuggester get(Policy.Session session, CollectionParams.CollectionAction action) { + UnsupportedSuggester suggester = new UnsupportedSuggester(action); + suggester._init(session); + return suggester; + } + + public UnsupportedSuggester(CollectionParams.CollectionAction action) { + this.action = action; + } + + @Override + public CollectionParams.CollectionAction getAction() { + return action; + } + + @Override + SolrRequest init() { + log.warn("Unsupported suggester for action " + action + " with hints " + hints + " - no suggestion available"); + return null; + } + + @Override + public SolrRequest getSuggestion() { + return null; + } +} diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java index f473ee4097b..6fb348f25c8 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java @@ -119,7 +119,9 @@ public interface CollectionParams { REPLACENODE(true, LockLevel.NONE), DELETENODE(true, LockLevel.NONE), MOCK_REPLICA_TASK(false, LockLevel.REPLICA), - NONE(false, LockLevel.NONE) + NONE(false, LockLevel.NONE), + // TODO: not implemented yet + MERGESHARDS(true, LockLevel.SHARD) ; public final boolean isWrite; diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java index e0f3d7878a6..a4c3b6d29d9 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java @@ -281,7 +281,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 { /** * Return a {@link CollectionStatePredicate} that returns true if a collection has the expected - * number of shards and replicas + * number of active shards and active replicas */ public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas) { return (liveNodes, collectionState) -> {