Move CompletionStats into the Engine (#33847)
By moving CompletionStats into the engine we can easily cache the stats for read-only engines if necessary. It also moves the responsibiltiy out of IndexShard which has quiet some complexity already. Relates to #33835
This commit is contained in:
parent
0fa5758bc6
commit
a92dda2e7e
|
@ -19,9 +19,11 @@
|
|||
|
||||
package org.elasticsearch.index.engine;
|
||||
|
||||
import com.carrotsearch.hppc.ObjectLongHashMap;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexCommit;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
@ -32,8 +34,10 @@ import org.apache.lucene.index.SegmentCommitInfo;
|
|||
import org.apache.lucene.index.SegmentInfos;
|
||||
import org.apache.lucene.index.SegmentReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.ReferenceManager;
|
||||
import org.apache.lucene.search.suggest.document.CompletionTerms;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
@ -42,6 +46,7 @@ import org.apache.lucene.util.Accountables;
|
|||
import org.apache.lucene.util.SetOnce;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.common.CheckedRunnable;
|
||||
import org.elasticsearch.common.FieldMemoryStats;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
|
@ -56,6 +61,7 @@ import org.elasticsearch.common.lucene.uid.Versions;
|
|||
import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver;
|
||||
import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion;
|
||||
import org.elasticsearch.common.metrics.CounterMetric;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ReleasableLock;
|
||||
import org.elasticsearch.index.VersionType;
|
||||
|
@ -71,6 +77,7 @@ import org.elasticsearch.index.shard.ShardId;
|
|||
import org.elasticsearch.index.store.Store;
|
||||
import org.elasticsearch.index.translog.Translog;
|
||||
import org.elasticsearch.index.translog.TranslogStats;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionStats;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.FileNotFoundException;
|
||||
|
@ -176,6 +183,34 @@ public abstract class Engine implements Closeable {
|
|||
/** Returns how many bytes we are currently moving from heap to disk */
|
||||
public abstract long getWritingBytes();
|
||||
|
||||
/**
|
||||
* Returns the {@link CompletionStats} for this engine
|
||||
*/
|
||||
public CompletionStats completionStats(String... fieldNamePatterns) throws IOException {
|
||||
try (Engine.Searcher currentSearcher = acquireSearcher("completion_stats", SearcherScope.INTERNAL)) {
|
||||
long sizeInBytes = 0;
|
||||
ObjectLongHashMap<String> completionFields = null;
|
||||
if (fieldNamePatterns != null && fieldNamePatterns.length > 0) {
|
||||
completionFields = new ObjectLongHashMap<>(fieldNamePatterns.length);
|
||||
}
|
||||
for (LeafReaderContext atomicReaderContext : currentSearcher.reader().leaves()) {
|
||||
LeafReader atomicReader = atomicReaderContext.reader();
|
||||
for (FieldInfo info : atomicReader.getFieldInfos()) {
|
||||
Terms terms = atomicReader.terms(info.name);
|
||||
if (terms instanceof CompletionTerms) {
|
||||
// TODO: currently we load up the suggester for reporting its size
|
||||
long fstSize = ((CompletionTerms) terms).suggester().ramBytesUsed();
|
||||
if (Regex.simpleMatch(fieldNamePatterns, info.name)) {
|
||||
completionFields.addTo(info.name, fstSize);
|
||||
}
|
||||
sizeInBytes += fstSize;
|
||||
}
|
||||
}
|
||||
}
|
||||
return new CompletionStats(sizeInBytes, completionFields == null ? null : new FieldMemoryStats(completionFields));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link DocsStats} for this engine
|
||||
*/
|
||||
|
|
|
@ -128,13 +128,13 @@ import org.elasticsearch.indices.recovery.RecoveryState;
|
|||
import org.elasticsearch.repositories.RepositoriesService;
|
||||
import org.elasticsearch.repositories.Repository;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionFieldStats;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionStats;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
|
@ -955,14 +955,16 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
}
|
||||
|
||||
public CompletionStats completionStats(String... fields) {
|
||||
CompletionStats completionStats = new CompletionStats();
|
||||
try (Engine.Searcher currentSearcher = acquireSearcher("completion_stats")) {
|
||||
readAllowed();
|
||||
try {
|
||||
CompletionStats stats = getEngine().completionStats(fields);
|
||||
// we don't wait for a pending refreshes here since it's a stats call instead we mark it as accessed only which will cause
|
||||
// the next scheduled refresh to go through and refresh the stats as well
|
||||
markSearcherAccessed();
|
||||
completionStats.add(CompletionFieldStats.completionStats(currentSearcher.reader(), fields));
|
||||
return stats;
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
return completionStats;
|
||||
}
|
||||
|
||||
public Engine.SyncedFlushResult syncFlush(String syncId, Engine.CommitId expectedCommitId) {
|
||||
|
|
|
@ -1,70 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.search.suggest.completion;
|
||||
|
||||
import com.carrotsearch.hppc.ObjectLongHashMap;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.search.suggest.document.CompletionTerms;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.FieldMemoryStats;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class CompletionFieldStats {
|
||||
|
||||
/**
|
||||
* Returns total in-heap bytes used by all suggesters. This method has CPU cost <code>O(numIndexedFields)</code>.
|
||||
*
|
||||
* @param fieldNamePatterns if non-null, any completion field name matching any of these patterns will break out its in-heap bytes
|
||||
* separately in the returned {@link CompletionStats}
|
||||
*/
|
||||
public static CompletionStats completionStats(IndexReader indexReader, String ... fieldNamePatterns) {
|
||||
long sizeInBytes = 0;
|
||||
ObjectLongHashMap<String> completionFields = null;
|
||||
if (fieldNamePatterns != null && fieldNamePatterns.length > 0) {
|
||||
completionFields = new ObjectLongHashMap<>(fieldNamePatterns.length);
|
||||
}
|
||||
for (LeafReaderContext atomicReaderContext : indexReader.leaves()) {
|
||||
LeafReader atomicReader = atomicReaderContext.reader();
|
||||
try {
|
||||
for (FieldInfo info : atomicReader.getFieldInfos()) {
|
||||
Terms terms = atomicReader.terms(info.name);
|
||||
if (terms instanceof CompletionTerms) {
|
||||
// TODO: currently we load up the suggester for reporting its size
|
||||
long fstSize = ((CompletionTerms) terms).suggester().ramBytesUsed();
|
||||
if (fieldNamePatterns != null && fieldNamePatterns.length > 0 && Regex.simpleMatch(fieldNamePatterns, info.name)) {
|
||||
completionFields.addTo(info.name, fstSize);
|
||||
}
|
||||
sizeInBytes += fstSize;
|
||||
}
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new ElasticsearchException(ioe);
|
||||
}
|
||||
}
|
||||
return new CompletionStats(sizeInBytes, completionFields == null ? null : new FieldMemoryStats(completionFields));
|
||||
}
|
||||
}
|
|
@ -2438,6 +2438,23 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
closeShards(sourceShard, targetShard);
|
||||
}
|
||||
|
||||
public void testCompletionStatsMarksSearcherAccessed() throws Exception {
|
||||
IndexShard indexShard = null;
|
||||
try {
|
||||
indexShard = newStartedShard();
|
||||
IndexShard shard = indexShard;
|
||||
assertBusy(() -> {
|
||||
ThreadPool threadPool = shard.getThreadPool();
|
||||
assertThat(threadPool.relativeTimeInMillis(), greaterThan(shard.getLastSearcherAccess()));
|
||||
});
|
||||
long prevAccessTime = shard.getLastSearcherAccess();
|
||||
indexShard.completionStats();
|
||||
assertThat("searcher was not marked as accessed", shard.getLastSearcherAccess(), greaterThan(prevAccessTime));
|
||||
} finally {
|
||||
closeShards(indexShard);
|
||||
}
|
||||
}
|
||||
|
||||
public void testDocStats() throws Exception {
|
||||
IndexShard indexShard = null;
|
||||
try {
|
||||
|
|
Loading…
Reference in New Issue