mirror of https://github.com/apache/lucene.git
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr
This commit is contained in:
commit
5bd7b03e71
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 + ")";
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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"
|
||||
};
|
||||
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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()) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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;
|
||||
};
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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!");
|
||||
|
|
|
@ -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' : " +
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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 -> {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -28,5 +28,6 @@ public enum TriggerEventType {
|
|||
SEARCHRATE,
|
||||
INDEXRATE,
|
||||
INVALID,
|
||||
METRIC
|
||||
METRIC,
|
||||
INDEXSIZE
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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) -> {
|
||||
|
|
Loading…
Reference in New Issue