Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Move CompletionStats into the Engine #33847

Merged
merged 7 commits into from
Sep 19, 2018
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
35 changes: 35 additions & 0 deletions server/src/main/java/org/elasticsearch/index/engine/Engine.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -32,8 +34,10 @@
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;
Expand All @@ -42,6 +46,7 @@
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;
Expand All @@ -56,6 +61,7 @@
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;
Expand All @@ -71,6 +77,7 @@
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;
Expand Down Expand Up @@ -176,6 +183,34 @@ public MergeStats getMergeStats() {
/** 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 (fieldNamePatterns != null && fieldNamePatterns.length > 0 && Regex.simpleMatch(fieldNamePatterns, info.name)) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: we don't need to check for non null non empty string array here as Regex.simpleMatch(fieldNamePatterns, info.name) takes care of it

completionFields.addTo(info.name, fstSize);
}
sizeInBytes += fstSize;
}
}
}
return new CompletionStats(sizeInBytes, completionFields == null ? null : new FieldMemoryStats(completionFields));
}
}

/**
* Returns the {@link DocsStats} for this engine
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,13 +128,13 @@
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;
Expand Down Expand Up @@ -954,14 +954,15 @@ public TranslogStats translogStats() {
}

public CompletionStats completionStats(String... fields) {
CompletionStats completionStats = new CompletionStats();
try (Engine.Searcher currentSearcher = acquireSearcher("completion_stats")) {
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) {
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -2438,6 +2438,23 @@ public void testRecoverFromLocalShard() throws IOException {
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 {
Expand Down