This commit is contained in:
Karl Wright 2018-04-11 17:54:30 -04:00
commit 5bd7b03e71
62 changed files with 3194 additions and 181 deletions

View File

@ -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

View File

@ -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;
}
}

View File

@ -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<numSegments;i++) {
final SegmentCommitInfo info = segmentInfos.info(i);
int delCount = writer.numDeletedDocs(info);
int delCount = writer.numDeletesToMerge(info);
if (delCount > 0) {
if (verbose(writer)) {
message(" segment " + info.info.name + " has deletions", writer);

View File

@ -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;
/**
* <p>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<CodecReader> readerSupplier) throws IOException {
return info.getDelCount() + pendingDeleteCount;
}
}

View File

@ -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<CodecReader> readerSupplier) throws IOException {
return in.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
}
}

View File

@ -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<CodecReader> readerSupplier) throws IOException {
return super.numDeletesToMerge(info, pendingDeleteCount, readerSupplier);
}
@Override
public String toString() {
return "NoMergePolicy";

View File

@ -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<CodecReader> 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
}
}

View File

@ -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<CodecReader> 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;
}
}

View File

@ -597,7 +597,7 @@ public class TieredMergePolicy extends MergePolicy {
final List<SegmentCommitInfo> eligible = new ArrayList<>();
final Set<SegmentCommitInfo> 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);
}

View File

@ -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> matches = new ArrayList<>();
int shouldMatchCount = 0;
Iterator<Weight> wIt = weights.iterator();
Iterator<BooleanClause> 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() {

View File

@ -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);

View File

@ -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<Term> 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<Term> 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<MatchesIterator> 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<MatchesIterator> mis) throws IOException {
if (mis.size() == 0)
return null;
if (mis.size() == 1)
return mis.get(0);
return new DisjunctionMatchesIterator(mis);
}
private final PriorityQueue<MatchesIterator> queue;
private boolean started = false;
private DisjunctionMatchesIterator(List<MatchesIterator> matches) throws IOException {
queue = new PriorityQueue<MatchesIterator>(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();
}
}

View File

@ -118,6 +118,18 @@ public final class DisjunctionMaxQuery extends Query implements Iterable<Query>
}
}
@Override
public Matches matches(LeafReaderContext context, int doc) throws IOException {
List<Matches> 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 {

View File

@ -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

View File

@ -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);
}
}

View File

@ -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

View File

@ -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;

View File

@ -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<String> {
/**
* 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<String> 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<Matches> subMatches) {
if (subMatches == null || subMatches.size() == 0) {
return null;
}
List<Matches> 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<String> 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<MatchesIterator> 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<String> 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<String> iterator() {
return Collections.singleton(field).iterator();
}
};
}
}

View File

@ -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();
}

View File

@ -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<Q extends MultiTermQuery> 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);

View File

@ -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);

View File

@ -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.

View File

@ -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;
}
}

View File

@ -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 + ")";

View File

@ -69,6 +69,35 @@ public abstract class Weight implements SegmentCacheable {
*/
public abstract void extractTerms(Set<Term> 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.
*

View File

@ -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 <T> the suppliers result type.
*/
@FunctionalInterface
public interface IOSupplier<T>{
/**
* Gets the result.
* @return the result
* @throws IOException if producing the result throws an {@link IOException}
*/
T get() throws IOException;
}

View File

@ -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();

View File

@ -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<String> 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"
};
}

View File

@ -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 {

View File

@ -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);

View File

@ -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<Matches> 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);

View File

@ -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<String> iterator() {
return in.iterator();
}
}

View File

@ -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();
}
}

View File

@ -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()) {

View File

@ -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);

View File

@ -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);

View File

@ -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
----------------------

View File

@ -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();

View File

@ -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);
}

View File

@ -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<TriggerEvent.Op> ops = (List<TriggerEvent.Op>)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<TriggerEvent.Op> unsupportedOps = (List<TriggerEvent.Op>)context.getProperties().computeIfAbsent("unsupportedOps", k -> new ArrayList<TriggerEvent.Op>());
unsupportedOps.add(op);
}
for (Map.Entry<Suggester.Hint, Object> 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;
}

View File

@ -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<String> collections = new HashSet<>();
private final Map<String, Long> 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<String, Object> 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<String, Object> getState() {
Map<String, Object> state = new HashMap<>();
state.put("lastEventMap", lastEventMap);
return state;
}
@Override
protected void setState(Map<String, Object> state) {
this.lastEventMap.clear();
Map<String, Long> replicaVsTime = (Map<String, Long>)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<String, ReplicaInfo> currentSizes = new HashMap<>();
try {
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
for (String node : clusterState.getLiveNodes()) {
Map<String, ReplicaInfo> metricTags = new HashMap<>();
// coll, shard, replica
Map<String, Map<String, List<ReplicaInfo>>> 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<String, Object> 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<String, List<ReplicaInfo>> 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<ReplicaInfo> 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<String, List<ReplicaInfo>> 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<ReplicaInfo> 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<TriggerEvent.Op> 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<String, Long> 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<Op> ops, Map<String, List<ReplicaInfo>> aboveSize,
Map<String, List<ReplicaInfo>> 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);
}
}
}

View File

@ -203,12 +203,12 @@ public class MetricTrigger extends TriggerBase {
List<Op> 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);

View File

@ -181,10 +181,11 @@ public class SearchRateTrigger extends TriggerBase {
} else {
Map<String, List<ReplicaInfo>> perCollection = collectionRates.computeIfAbsent(info.getCollection(), s -> new HashMap<>());
List<ReplicaInfo> 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());
}
});
}

View File

@ -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() {

View File

@ -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<Slice> 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;
};

View File

@ -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<String, List<CapturedEvent>> 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<String, Object> 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<TriggerEvent> 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<TriggerEvent.Op> ops = (List<TriggerEvent.Op>) 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<Pair<String, String>> hints = (Set<Pair<String, String>>)op.getHints().get(Suggester.Hint.COLL_SHARD);
assertNotNull("hints", hints);
assertEquals("hints", 1, hints.size());
Pair<String, String> 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<CapturedEvent> 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<Object> 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<CapturedEvent> 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<TriggerEvent.Op> ops = (List<TriggerEvent.Op>) 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<Pair<String, String>> hints = (Set<Pair<String, String>>)op.getHints().get(Suggester.Hint.COLL_SHARD);
assertNotNull("hints", hints);
assertEquals("hints", 1, hints.size());
Pair<String, String> 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<Object> 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<CapturedEvent> 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<TriggerEvent.Op> ops = (List<TriggerEvent.Op>) 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<Pair<String, String>> hints = (Set<Pair<String, String>>)op.getHints().get(Suggester.Hint.COLL_SHARD);
assertNotNull("hints", hints);
assertEquals("hints", 2, hints.size());
Pair<String, String> p = hints.iterator().next();
assertEquals(collectionName, p.first());
}
// TODO: fix this once MERGESHARDS is supported
List<TriggerEvent.Op> unsupportedOps = (List<TriggerEvent.Op>)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<Object> 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<CapturedEvent> 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<TriggerEvent.Op> ops = (List<TriggerEvent.Op>) 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<Pair<String, String>> hints = (Set<Pair<String, String>>)op.getHints().get(Suggester.Hint.COLL_SHARD);
assertNotNull("hints", hints);
assertEquals("hints", 1, hints.size());
Pair<String, String> 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<TriggerEvent.Op>) 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<Pair<String, String>> hints = (Set<Pair<String, String>>)op.getHints().get(Suggester.Hint.COLL_SHARD);
assertNotNull("hints", hints);
assertEquals("hints", 2, hints.size());
Pair<String, String> p = hints.iterator().next();
assertEquals(collectionName, p.first());
}
// TODO: fix this once MERGESHARDS is supported
List<TriggerEvent.Op> unsupportedOps = (List<TriggerEvent.Op>)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<String, Object> createTriggerProps(long waitForSeconds) {
Map<String, Object> 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<Map<String, String>> actions = new ArrayList<>(3);
Map<String, String> 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;
}
}

View File

@ -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!");

View File

@ -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' : " +

View File

@ -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<String, Map<Replica.State, AtomicInteger>> replicaStates = new TreeMap<>();
int numReplicas = 0;
for (String node : getLiveNodesSet().get()) {
List<ReplicaInfo> 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<String> 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<String, Map<String, AtomicInteger>> 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<Object> 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<SolrInputDocument> docs = ureq.getDocuments();
if (docs != null) {
systemColl.addAll(docs);
}
return new UpdateResponse();
}
List<SolrInputDocument> 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());

View File

@ -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<String> 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<String> 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<ReplicaInfo> 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<String> 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<String, Object> 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<String, Object> 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:
* <ul>
* <li><code>SEARCHER.searcher.numDocs</code> - increased by added docs, decreased by deleteById and deleteByQuery</li>
* <li><code>SEARCHER.searcher.deletedDocs</code> - decreased by deleteById and deleteByQuery by up to <code>numDocs</code></li>
* <li><code>SEARCHER.searcher.maxDoc</code> - always increased by the number of added docs.</li>
* </ul>
* <p>IMPORTANT limitations:</p>
* <ul>
* <li>document replacements are always counted as new docs</li>
* <li>delete by ID always succeeds (unless numDocs == 0)</li>
* <li>deleteByQuery is not supported unless the query is <code>*:*</code></li>
* </ul>
* @param req update request. This request MUST have the <code>collection</code> 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<String> 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<SolrInputDocument> 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> properties) throws Exception {
Map<String, Object> 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<ReplicaInfo> 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<String> simListCollections() {
public List<String> simListCollections() throws InterruptedException {
final Set<String> 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<String, DocCollection> getCollectionStates() {
Map<String, DocCollection> collectionStates = collectionsStatesRef.get();
if (collectionStates != null) {
@ -1263,7 +1449,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
slices.put(s, slice);
});
Map<String, Object> collProps = collProperties.computeIfAbsent(coll, c -> new ConcurrentHashMap<>());
DocCollection dc = new DocCollection(coll, slices, collProps, DocRouter.DEFAULT, clusterStateVersion, ZkStateReader.CLUSTER_STATE);
Map<String, Object> routerProp = (Map<String, Object>) 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);

View File

@ -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

View File

@ -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<String, Map<Replica.State, AtomicInteger>> replicaStates = new TreeMap<>();
int numReplicas = 0;
for (String node : cluster.getLiveNodesSet().get()) {
List<ReplicaInfo> 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<String> 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<String, Map<String, AtomicInteger>> 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));
}
}

View File

@ -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<String, String> hint = (Pair<String, String>)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<String, String> hint = null;
if (o instanceof Pair) {
hint = (Pair<String, String>)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<String, String>)o;
} else {
fail("unexpected hints: " + o);
}
assertNotNull(hint);
assertEquals(collectionName, hint.first());
assertEquals("shard1", hint.second());

View File

@ -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);

View File

@ -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

View File

@ -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<String, List<Clause>> getPolicies() {

View File

@ -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 -> {

View File

@ -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<Pair<String, String>> shards = (Set<Pair<String, String>>) 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<String, String> collShard = shards.iterator().next();
return CollectionAdminRequest.splitShard(collShard.first()).setShardName(collShard.second());
}
}

View File

@ -28,5 +28,6 @@ public enum TriggerEventType {
SEARCHRATE,
INDEXRATE,
INVALID,
METRIC
METRIC,
INDEXSIZE
}

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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) -> {