Merge branch 'master' into feature/rank-eval

This commit is contained in:
Christoph Büscher 2016-08-15 11:26:34 +02:00
commit a7e56d7fa5
150 changed files with 1196 additions and 633 deletions

View File

@ -42,14 +42,14 @@ public class DocsTestPlugin extends RestTestPlugin {
'List snippets that probably should be marked // CONSOLE'
listConsoleCandidates.perSnippet {
if (
it.console // Already marked, nothing to do
|| it.testResponse // It is a response
it.console != null // Already marked, nothing to do
|| it.testResponse // It is a response
) {
return
}
List<String> languages = [
// These languages should almost always be marked console
'js', 'json',
// This language should almost always be marked console
'js',
// These are often curl commands that should be converted but
// are probably false positives
'sh', 'shell',

View File

@ -32,6 +32,12 @@ import java.util.regex.Matcher
* Generates REST tests for each snippet marked // TEST.
*/
public class RestTestsFromSnippetsTask extends SnippetsTask {
/**
* These languages aren't supported by the syntax highlighter so we
* shouldn't use them.
*/
private static final List BAD_LANGUAGES = ['json', 'javascript']
@Input
Map<String, String> setups = new HashMap()
@ -87,9 +93,9 @@ public class RestTestsFromSnippetsTask extends SnippetsTask {
* calls buildTest to actually build the test.
*/
void handleSnippet(Snippet snippet) {
if (snippet.language == 'json') {
if (BAD_LANGUAGES.contains(snippet.language)) {
throw new InvalidUserDataException(
"$snippet: Use `js` instead of `json`.")
"$snippet: Use `js` instead of `${snippet.language}`.")
}
if (snippet.testSetup) {
setup(snippet)

View File

@ -114,23 +114,38 @@ public class SnippetsTask extends DefaultTask {
return
}
if (line ==~ /\/\/\s*AUTOSENSE\s*/) {
throw new InvalidUserDataException("AUTOSENSE has been " +
"replaced by CONSOLE. Use that instead at " +
"$file:$lineNumber")
throw new InvalidUserDataException("$file:$lineNumber: "
+ "AUTOSENSE has been replaced by CONSOLE.")
}
if (line ==~ /\/\/\s*CONSOLE\s*/) {
if (snippet == null) {
throw new InvalidUserDataException("CONSOLE not " +
"paired with a snippet at $file:$lineNumber")
throw new InvalidUserDataException("$file:$lineNumber: "
+ "CONSOLE not paired with a snippet")
}
if (snippet.console != null) {
throw new InvalidUserDataException("$file:$lineNumber: "
+ "Can't be both CONSOLE and NOTCONSOLE")
}
snippet.console = true
return
}
if (line ==~ /\/\/\s*NOTCONSOLE\s*/) {
if (snippet == null) {
throw new InvalidUserDataException("$file:$lineNumber: "
+ "NOTCONSOLE not paired with a snippet")
}
if (snippet.console != null) {
throw new InvalidUserDataException("$file:$lineNumber: "
+ "Can't be both CONSOLE and NOTCONSOLE")
}
snippet.console = false
return
}
matcher = line =~ /\/\/\s*TEST(\[(.+)\])?\s*/
if (matcher.matches()) {
if (snippet == null) {
throw new InvalidUserDataException("TEST not " +
"paired with a snippet at $file:$lineNumber")
throw new InvalidUserDataException("$file:$lineNumber: "
+ "TEST not paired with a snippet at ")
}
snippet.test = true
if (matcher.group(2) != null) {
@ -172,8 +187,8 @@ public class SnippetsTask extends DefaultTask {
matcher = line =~ /\/\/\s*TESTRESPONSE(\[(.+)\])?\s*/
if (matcher.matches()) {
if (snippet == null) {
throw new InvalidUserDataException("TESTRESPONSE not " +
"paired with a snippet at $file:$lineNumber")
throw new InvalidUserDataException("$file:$lineNumber: "
+ "TESTRESPONSE not paired with a snippet")
}
snippet.testResponse = true
if (matcher.group(2) != null) {
@ -226,7 +241,7 @@ public class SnippetsTask extends DefaultTask {
int end = NOT_FINISHED
String contents
boolean console = false
Boolean console = null
boolean test = false
boolean testResponse = false
boolean testSetup = false
@ -243,8 +258,8 @@ public class SnippetsTask extends DefaultTask {
if (language != null) {
result += "($language)"
}
if (console) {
result += '// CONSOLE'
if (console != null) {
result += console ? '// CONSOLE' : '// NOTCONSOLE'
}
if (test) {
result += '// TEST'

View File

@ -261,6 +261,7 @@ class ClusterFormationTasks {
'node.attr.testattr' : 'test',
'repositories.url.allowed_urls': 'http://snapshot.test*'
]
esConfig['node.max_local_storage_nodes'] = node.config.numNodes
esConfig['http.port'] = node.config.httpPort
esConfig['transport.tcp.port'] = node.config.transportPort
esConfig.putAll(node.config.settings)

View File

@ -575,18 +575,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]fetch[/\\]FetchSearchResult.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]fetch[/\\]FetchSubPhase.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]fetch[/\\]FetchSubPhaseParseElement.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]fetch[/\\]explain[/\\]ExplainFetchSubPhase.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]fetch[/\\]fielddata[/\\]FieldDataFieldsParseElement.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]fetch[/\\]innerhits[/\\]InnerHitsParseElement.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]fetch[/\\]source[/\\]FetchSourceContext.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]highlight[/\\]FastVectorHighlighter.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]highlight[/\\]HighlightPhase.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]highlight[/\\]HighlightUtils.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]highlight[/\\]HighlighterParseElement.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]highlight[/\\]PlainHighlighter.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]highlight[/\\]PostingsHighlighter.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]highlight[/\\]vectorhighlight[/\\]SimpleFragmentsBuilder.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]highlight[/\\]vectorhighlight[/\\]SourceScoreOrderFragmentsBuilder.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]internal[/\\]DefaultSearchContext.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]internal[/\\]FilteredSearchContext.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]internal[/\\]InternalSearchHit.java" checks="LineLength" />

View File

@ -20,7 +20,7 @@
package org.apache.lucene.search.postingshighlight;
import org.apache.lucene.search.highlight.Encoder;
import org.elasticsearch.search.highlight.HighlightUtils;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightUtils;
/**
Custom passage formatter that allows us to:

View File

@ -467,17 +467,21 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
// Need to set anonymous "name" of char_filter
charFilterFactories[i] = charFilterFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_charfilter_[" + i + "]", settings);
} else {
AnalysisModule.AnalysisProvider<CharFilterFactory> charFilterFactoryFactory;
if (analysisService == null) {
AnalysisModule.AnalysisProvider<CharFilterFactory> charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilter.name);
charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilter.name);
if (charFilterFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global char filter under [" + charFilter.name + "]");
}
charFilterFactories[i] = charFilterFactoryFactory.get(environment, charFilter.name);
} else {
charFilterFactories[i] = analysisService.charFilter(charFilter.name);
if (charFilterFactories[i] == null) {
charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilter.name, analysisService.getIndexSettings());
if (charFilterFactoryFactory == null) {
throw new IllegalArgumentException("failed to find char filter under [" + charFilter.name + "]");
}
charFilterFactories[i] = charFilterFactoryFactory.get(analysisService.getIndexSettings(), environment, charFilter.name,
AnalysisRegistry.getSettingsFromIndexSettings(analysisService.getIndexSettings(),
AnalysisRegistry.INDEX_ANALYSIS_CHAR_FILTER + "." + charFilter.name));
}
}
if (charFilterFactories[i] == null) {
@ -509,18 +513,21 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
// Need to set anonymous "name" of tokenfilter
tokenFilterFactories[i] = tokenFilterFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_tokenfilter_[" + i + "]", settings);
} else {
AnalysisModule.AnalysisProvider<TokenFilterFactory> tokenFilterFactoryFactory;
if (analysisService == null) {
AnalysisModule.AnalysisProvider<TokenFilterFactory> tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilter.name);
tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilter.name);
if (tokenFilterFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global token filter under [" + tokenFilter.name + "]");
}
tokenFilterFactories[i] = tokenFilterFactoryFactory.get(environment, tokenFilter.name);
} else {
tokenFilterFactories[i] = analysisService.tokenFilter(tokenFilter.name);
if (tokenFilterFactories[i] == null) {
tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilter.name, analysisService.getIndexSettings());
if (tokenFilterFactoryFactory == null) {
throw new IllegalArgumentException("failed to find token filter under [" + tokenFilter.name + "]");
}
tokenFilterFactories[i] = tokenFilterFactoryFactory.get(analysisService.getIndexSettings(), environment, tokenFilter.name,
AnalysisRegistry.getSettingsFromIndexSettings(analysisService.getIndexSettings(),
AnalysisRegistry.INDEX_ANALYSIS_FILTER + "." + tokenFilter.name));
}
}
if (tokenFilterFactories[i] == null) {
@ -550,17 +557,21 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
// Need to set anonymous "name" of tokenizer
tokenizerFactory = tokenizerFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_tokenizer", settings);
} else {
AnalysisModule.AnalysisProvider<TokenizerFactory> tokenizerFactoryFactory;
if (analysisService == null) {
AnalysisModule.AnalysisProvider<TokenizerFactory> tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(tokenizer.name);
tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(tokenizer.name);
if (tokenizerFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global tokenizer under [" + tokenizer.name + "]");
}
tokenizerFactory = tokenizerFactoryFactory.get(environment, tokenizer.name);
} else {
tokenizerFactory = analysisService.tokenizer(tokenizer.name);
if (tokenizerFactory == null) {
tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(tokenizer.name, analysisService.getIndexSettings());
if (tokenizerFactoryFactory == null) {
throw new IllegalArgumentException("failed to find tokenizer under [" + tokenizer.name + "]");
}
tokenizerFactory = tokenizerFactoryFactory.get(analysisService.getIndexSettings(), environment, tokenizer.name,
AnalysisRegistry.getSettingsFromIndexSettings(analysisService.getIndexSettings(),
AnalysisRegistry.INDEX_ANALYSIS_TOKENIZER + "." + tokenizer.name));
}
}
return tokenizerFactory;

View File

@ -26,7 +26,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import java.io.IOException;

View File

@ -24,7 +24,7 @@ import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
/**
* A builder for {@link ExplainRequest}.

View File

@ -28,7 +28,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import java.io.IOException;

View File

@ -24,7 +24,7 @@ import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
/**
* A get document action request builder.

View File

@ -38,7 +38,7 @@ import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import java.io.IOException;
import java.util.ArrayList;

View File

@ -322,11 +322,8 @@ abstract class AbstractSearchAsyncAction<FirstResult extends SearchPhaseResult>
// we only release search context that we did not fetch from if we are not scrolling
if (request.scroll() == null) {
for (AtomicArray.Entry<? extends QuerySearchResultProvider> entry : queryResults.asList()) {
QuerySearchResult queryResult = entry.value.queryResult().queryResult();
final TopDocs topDocs = queryResult.topDocs();
final Suggest suggest = queryResult.suggest();
if (((topDocs != null && topDocs.scoreDocs.length > 0) // the shard had matches
||suggest != null && suggest.hasScoreDocs()) // or had suggest docs
QuerySearchResult queryResult = entry.value.queryResult();
if (queryResult.hasHits()
&& docIdsToLoad.get(entry.index) == null) { // but none of them made it to the global top docs
try {
DiscoveryNode node = nodes.get(entry.value.queryResult().shardTarget().nodeId());

View File

@ -31,7 +31,7 @@ import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
import org.elasticsearch.search.slice.SliceBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.sort.SortOrder;

View File

@ -48,7 +48,7 @@ import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.lookup.SourceLookup;
import java.util.ArrayList;

View File

@ -546,24 +546,15 @@ public class RoutingNodes implements Iterable<RoutingNode> {
assert failedShard.active();
if (failedShard.primary()) {
// promote active replica to primary if active replica exists
ShardRouting candidate = activeReplica(failedShard.shardId());
if (candidate == null) {
ShardRouting activeReplica = activeReplica(failedShard.shardId());
if (activeReplica == null) {
moveToUnassigned(failedShard, unassignedInfo);
} else {
// if the activeReplica was relocating before this call to failShard, its relocation was cancelled above when we
// failed initializing replica shards (and moved replica relocation source back to started)
assert activeReplica.started() : "replica relocation should have been cancelled: " + activeReplica;
movePrimaryToUnassignedAndDemoteToReplica(failedShard, unassignedInfo);
ShardRouting primarySwappedCandidate = promoteAssignedReplicaShardToPrimary(candidate);
if (primarySwappedCandidate.relocatingNodeId() != null) {
// its also relocating, make sure to move the other routing to primary
RoutingNode node = node(primarySwappedCandidate.relocatingNodeId());
if (node != null) {
for (ShardRouting shardRouting : node) {
if (shardRouting.shardId().equals(primarySwappedCandidate.shardId()) && !shardRouting.primary()) {
promoteAssignedReplicaShardToPrimary(shardRouting);
break;
}
}
}
}
ShardRouting primarySwappedCandidate = promoteActiveReplicaShardToPrimary(activeReplica);
if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.getSettings())) {
reinitShadowPrimary(primarySwappedCandidate);
}
@ -621,8 +612,8 @@ public class RoutingNodes implements Iterable<RoutingNode> {
* @param replicaShard the replica shard to be promoted to primary
* @return the resulting primary shard
*/
private ShardRouting promoteAssignedReplicaShardToPrimary(ShardRouting replicaShard) {
assert replicaShard.unassigned() == false : "unassigned shard cannot be promoted to primary: " + replicaShard;
private ShardRouting promoteActiveReplicaShardToPrimary(ShardRouting replicaShard) {
assert replicaShard.active() : "non-active shard cannot be promoted to primary: " + replicaShard;
assert replicaShard.primary() == false : "primary shard cannot be promoted to primary: " + replicaShard;
ShardRouting primaryShard = replicaShard.moveToPrimary();
updateAssigned(replicaShard, primaryShard);
@ -729,7 +720,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
/**
* Moves assigned primary to unassigned and demotes it to a replica.
* Used in conjunction with {@link #promoteAssignedReplicaShardToPrimary} when an active replica is promoted to primary.
* Used in conjunction with {@link #promoteActiveReplicaShardToPrimary} when an active replica is promoted to primary.
*/
private ShardRouting movePrimaryToUnassignedAndDemoteToReplica(ShardRouting shard, UnassignedInfo unassignedInfo) {
assert shard.unassigned() == false : "only assigned shards can be moved to unassigned (" + shard + ")";

View File

@ -68,6 +68,7 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
import java.util.Set;
@ -151,7 +152,7 @@ public final class NodeEnvironment implements Closeable {
/**
* Maximum number of data nodes that should run in an environment.
*/
public static final Setting<Integer> MAX_LOCAL_STORAGE_NODES_SETTING = Setting.intSetting("node.max_local_storage_nodes", 50, 1,
public static final Setting<Integer> MAX_LOCAL_STORAGE_NODES_SETTING = Setting.intSetting("node.max_local_storage_nodes", 1, 1,
Property.NodeScope);
/**
@ -244,8 +245,15 @@ public final class NodeEnvironment implements Closeable {
}
if (locks[0] == null) {
throw new IllegalStateException("Failed to obtain node lock, is the following location writable?: "
+ Arrays.toString(environment.dataWithClusterFiles()), lastException);
final String message = String.format(
Locale.ROOT,
"failed to obtain node locks, tried [%s] with lock id%s;" +
" maybe these locations are not writable or multiple nodes were started without increasing [%s] (was [%d])?",
Arrays.toString(environment.dataWithClusterFiles()),
maxLocalStorageNodes == 1 ? " [0]" : "s [0--" + (maxLocalStorageNodes - 1) + "]",
MAX_LOCAL_STORAGE_NODES_SETTING.getKey(),
maxLocalStorageNodes);
throw new IllegalStateException(message, lastException);
}
this.nodeMetaData = loadOrCreateNodeMetaData(settings, startupTraceLogger, nodePaths);
this.logger = Loggers.getLogger(getClass(), Node.addNodeNameIfNeeded(settings, this.nodeMetaData.nodeId()));

View File

@ -33,7 +33,6 @@ import java.util.concurrent.TimeUnit;
/**
*/
public final class SearchSlowLog implements SearchOperationListener {
private final Index index;
private boolean reformat;
private long queryWarnThreshold;
@ -84,10 +83,8 @@ public final class SearchSlowLog implements SearchOperationListener {
public SearchSlowLog(IndexSettings indexSettings) {
this.queryLogger = Loggers.getLogger(INDEX_SEARCH_SLOWLOG_PREFIX + ".query");
this.fetchLogger = Loggers.getLogger(INDEX_SEARCH_SLOWLOG_PREFIX + ".fetch");
this.index = indexSettings.getIndex();
this.queryLogger = Loggers.getLogger(INDEX_SEARCH_SLOWLOG_PREFIX + ".query", indexSettings.getSettings());
this.fetchLogger = Loggers.getLogger(INDEX_SEARCH_SLOWLOG_PREFIX + ".fetch", indexSettings.getSettings());
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_SEARCH_SLOWLOG_REFORMAT, this::setReformat);
this.reformat = indexSettings.getValue(INDEX_SEARCH_SLOWLOG_REFORMAT);
@ -122,38 +119,36 @@ public final class SearchSlowLog implements SearchOperationListener {
@Override
public void onQueryPhase(SearchContext context, long tookInNanos) {
if (queryWarnThreshold >= 0 && tookInNanos > queryWarnThreshold) {
queryLogger.warn("{}", new SlowLogSearchContextPrinter(index, context, tookInNanos, reformat));
queryLogger.warn("{}", new SlowLogSearchContextPrinter(context, tookInNanos, reformat));
} else if (queryInfoThreshold >= 0 && tookInNanos > queryInfoThreshold) {
queryLogger.info("{}", new SlowLogSearchContextPrinter(index, context, tookInNanos, reformat));
queryLogger.info("{}", new SlowLogSearchContextPrinter(context, tookInNanos, reformat));
} else if (queryDebugThreshold >= 0 && tookInNanos > queryDebugThreshold) {
queryLogger.debug("{}", new SlowLogSearchContextPrinter(index, context, tookInNanos, reformat));
queryLogger.debug("{}", new SlowLogSearchContextPrinter(context, tookInNanos, reformat));
} else if (queryTraceThreshold >= 0 && tookInNanos > queryTraceThreshold) {
queryLogger.trace("{}", new SlowLogSearchContextPrinter(index, context, tookInNanos, reformat));
queryLogger.trace("{}", new SlowLogSearchContextPrinter(context, tookInNanos, reformat));
}
}
@Override
public void onFetchPhase(SearchContext context, long tookInNanos) {
if (fetchWarnThreshold >= 0 && tookInNanos > fetchWarnThreshold) {
fetchLogger.warn("{}", new SlowLogSearchContextPrinter(index, context, tookInNanos, reformat));
fetchLogger.warn("{}", new SlowLogSearchContextPrinter(context, tookInNanos, reformat));
} else if (fetchInfoThreshold >= 0 && tookInNanos > fetchInfoThreshold) {
fetchLogger.info("{}", new SlowLogSearchContextPrinter(index, context, tookInNanos, reformat));
fetchLogger.info("{}", new SlowLogSearchContextPrinter(context, tookInNanos, reformat));
} else if (fetchDebugThreshold >= 0 && tookInNanos > fetchDebugThreshold) {
fetchLogger.debug("{}", new SlowLogSearchContextPrinter(index, context, tookInNanos, reformat));
fetchLogger.debug("{}", new SlowLogSearchContextPrinter(context, tookInNanos, reformat));
} else if (fetchTraceThreshold >= 0 && tookInNanos > fetchTraceThreshold) {
fetchLogger.trace("{}", new SlowLogSearchContextPrinter(index, context, tookInNanos, reformat));
fetchLogger.trace("{}", new SlowLogSearchContextPrinter(context, tookInNanos, reformat));
}
}
static final class SlowLogSearchContextPrinter {
private final SearchContext context;
private final Index index;
private final long tookInNanos;
private final boolean reformat;
public SlowLogSearchContextPrinter(Index index, SearchContext context, long tookInNanos, boolean reformat) {
public SlowLogSearchContextPrinter(SearchContext context, long tookInNanos, boolean reformat) {
this.context = context;
this.index = index;
this.tookInNanos = tookInNanos;
this.reformat = reformat;
}
@ -161,7 +156,7 @@ public final class SearchSlowLog implements SearchOperationListener {
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(index).append(" ");
sb.append(context.indexShard().shardId()).append(" ");
sb.append("took[").append(TimeValue.timeValueNanos(tookInNanos)).append("], took_millis[").append(TimeUnit.NANOSECONDS.toMillis(tookInNanos)).append("], ");
if (context.getQueryShardContext().getTypes() == null) {
sb.append("types[], ");

View File

@ -49,6 +49,9 @@ import static java.util.Collections.unmodifiableMap;
* This class exists per node and allows to create per-index {@link AnalysisService} via {@link #build(IndexSettings)}
*/
public final class AnalysisRegistry implements Closeable {
public static final String INDEX_ANALYSIS_CHAR_FILTER = "index.analysis.char_filter";
public static final String INDEX_ANALYSIS_FILTER = "index.analysis.filter";
public static final String INDEX_ANALYSIS_TOKENIZER = "index.analysis.tokenizer";
private final PrebuiltAnalysis prebuiltAnalysis = new PrebuiltAnalysis();
private final Map<String, Analyzer> cachedAnalyzer = new ConcurrentHashMap<>();
@ -70,6 +73,20 @@ public final class AnalysisRegistry implements Closeable {
this.analyzers = unmodifiableMap(analyzers);
}
/**
* Returns a {@link Settings} by groupName from {@link IndexSettings} or a default {@link Settings}
* @param indexSettings an index settings
* @param groupName tokenizer/token filter/char filter name
* @return {@link Settings}
*/
public static Settings getSettingsFromIndexSettings(IndexSettings indexSettings, String groupName) {
Settings settings = indexSettings.getSettings().getAsSettings(groupName);
if (settings.isEmpty()) {
settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, indexSettings.getIndexVersionCreated()).build();
}
return settings;
}
/**
* Returns a registered {@link TokenizerFactory} provider by name or <code>null</code> if the tokenizer was not registered
*/
@ -122,9 +139,9 @@ public final class AnalysisRegistry implements Closeable {
* Creates an index-level {@link AnalysisService} from this registry using the given index settings
*/
public AnalysisService build(IndexSettings indexSettings) throws IOException {
final Map<String, Settings> charFiltersSettings = indexSettings.getSettings().getGroups("index.analysis.char_filter");
final Map<String, Settings> tokenFiltersSettings = indexSettings.getSettings().getGroups("index.analysis.filter");
final Map<String, Settings> tokenizersSettings = indexSettings.getSettings().getGroups("index.analysis.tokenizer");
final Map<String, Settings> charFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_CHAR_FILTER);
final Map<String, Settings> tokenFiltersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_FILTER);
final Map<String, Settings> tokenizersSettings = indexSettings.getSettings().getGroups(INDEX_ANALYSIS_TOKENIZER);
final Map<String, Settings> analyzersSettings = indexSettings.getSettings().getGroups("index.analysis.analyzer");
final Map<String, CharFilterFactory> charFilterFactories = buildMapping(false, "charfilter", indexSettings, charFiltersSettings, charFilters, prebuiltAnalysis.charFilterFactories);
@ -136,13 +153,76 @@ public final class AnalysisRegistry implements Closeable {
* instead of building the infrastructure for plugins we rather make it a real exception to not pollute the general interface and
* hide internal data-structures as much as possible.
*/
tokenFilters.put("synonym", requriesAnalysisSettings((is, env, name, settings) -> new SynonymTokenFilterFactory(is, env, tokenizerFactories, name, settings)));
tokenFilters.put("synonym", requriesAnalysisSettings((is, env, name, settings) -> new SynonymTokenFilterFactory(is, env, this, name, settings)));
final Map<String, TokenFilterFactory> tokenFilterFactories = buildMapping(false, "tokenfilter", indexSettings, tokenFiltersSettings, Collections.unmodifiableMap(tokenFilters), prebuiltAnalysis.tokenFilterFactories);
final Map<String, AnalyzerProvider<?>> analyzierFactories = buildMapping(true, "analyzer", indexSettings, analyzersSettings,
analyzers, prebuiltAnalysis.analyzerProviderFactories);
return new AnalysisService(indexSettings, analyzierFactories, tokenizerFactories, charFilterFactories, tokenFilterFactories);
}
/**
* Returns a registered {@link TokenizerFactory} provider by {@link IndexSettings}
* or a registered {@link TokenizerFactory} provider by predefined name
* or <code>null</code> if the tokenizer was not registered
* @param tokenizer global or defined tokenizer name
* @param indexSettings an index settings
* @return {@link TokenizerFactory} provider or <code>null</code>
*/
public AnalysisProvider<TokenizerFactory> getTokenizerProvider(String tokenizer, IndexSettings indexSettings) {
final Map<String, Settings> tokenizerSettings = indexSettings.getSettings().getGroups("index.analysis.tokenizer");
if (tokenizerSettings.containsKey(tokenizer)) {
Settings currentSettings = tokenizerSettings.get(tokenizer);
return getAnalysisProvider("tokenizer", tokenizers, tokenizer, currentSettings.get("type"));
} else {
return prebuiltAnalysis.tokenizerFactories.get(tokenizer);
}
}
/**
* Returns a registered {@link TokenFilterFactory} provider by {@link IndexSettings}
* or a registered {@link TokenFilterFactory} provider by predefined name
* or <code>null</code> if the tokenFilter was not registered
* @param tokenFilter global or defined tokenFilter name
* @param indexSettings an index settings
* @return {@link TokenFilterFactory} provider or <code>null</code>
*/
public AnalysisProvider<TokenFilterFactory> getTokenFilterProvider(String tokenFilter, IndexSettings indexSettings) {
final Map<String, Settings> tokenFilterSettings = indexSettings.getSettings().getGroups("index.analysis.filter");
if (tokenFilterSettings.containsKey(tokenFilter)) {
Settings currentSettings = tokenFilterSettings.get(tokenFilter);
String typeName = currentSettings.get("type");
/*
* synonym is different than everything else since it needs access to the tokenizer factories for this index.
* instead of building the infrastructure for plugins we rather make it a real exception to not pollute the general interface and
* hide internal data-structures as much as possible.
*/
if ("synonym".equals(typeName)) {
return requriesAnalysisSettings((is, env, name, settings) -> new SynonymTokenFilterFactory(is, env, this, name, settings));
} else {
return getAnalysisProvider("tokenfilter", tokenFilters, tokenFilter, typeName);
}
} else {
return prebuiltAnalysis.tokenFilterFactories.get(tokenFilter);
}
}
/**
* Returns a registered {@link CharFilterFactory} provider by {@link IndexSettings}
* or a registered {@link CharFilterFactory} provider by predefined name
* or <code>null</code> if the charFilter was not registered
* @param charFilter global or defined charFilter name
* @param indexSettings an index settings
* @return {@link CharFilterFactory} provider or <code>null</code>
*/
public AnalysisProvider<CharFilterFactory> getCharFilterProvider(String charFilter, IndexSettings indexSettings) {
final Map<String, Settings> tokenFilterSettings = indexSettings.getSettings().getGroups("index.analysis.char_filter");
if (tokenFilterSettings.containsKey(charFilter)) {
Settings currentSettings = tokenFilterSettings.get(charFilter);
return getAnalysisProvider("charfilter", charFilters, charFilter, currentSettings.get("type"));
} else {
return prebuiltAnalysis.charFilterFactories.get(charFilter);
}
}
private static <T> AnalysisModule.AnalysisProvider<T> requriesAnalysisSettings(AnalysisModule.AnalysisProvider<T> provider) {
return new AnalysisModule.AnalysisProvider<T>() {
@ -185,13 +265,7 @@ public final class AnalysisRegistry implements Closeable {
}
factories.put(name, factory);
} else {
if (typeName == null) {
throw new IllegalArgumentException(toBuild + " [" + name + "] must specify either an analyzer type, or a tokenizer");
}
AnalysisModule.AnalysisProvider<T> type = providerMap.get(typeName);
if (type == null) {
throw new IllegalArgumentException("Unknown " + toBuild + " type [" + typeName + "] for [" + name + "]");
}
AnalysisProvider<T> type = getAnalysisProvider(toBuild, providerMap, name, typeName);
final T factory = type.get(settings, environment, name, currentSettings);
factories.put(name, factory);
}
@ -232,6 +306,17 @@ public final class AnalysisRegistry implements Closeable {
return factories;
}
private <T> AnalysisProvider<T> getAnalysisProvider(String toBuild, Map<String, AnalysisProvider<T>> providerMap, String name, String typeName) {
if (typeName == null) {
throw new IllegalArgumentException(toBuild + " [" + name + "] must specify either an analyzer type, or a tokenizer");
}
AnalysisProvider<T> type = providerMap.get(typeName);
if (type == null) {
throw new IllegalArgumentException("Unknown " + toBuild + " type [" + typeName + "] for [" + name + "]");
}
return type;
}
private static class PrebuiltAnalysis implements Closeable {
final Map<String, AnalysisModule.AnalysisProvider<AnalyzerProvider<?>>> analyzerProviderFactories;

View File

@ -32,18 +32,18 @@ import org.elasticsearch.common.io.FastStringReader;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.indices.analysis.AnalysisModule;
import java.io.IOException;
import java.io.Reader;
import java.util.List;
import java.util.Map;
public class SynonymTokenFilterFactory extends AbstractTokenFilterFactory {
private final SynonymMap synonymMap;
private final boolean ignoreCase;
public SynonymTokenFilterFactory(IndexSettings indexSettings, Environment env, Map<String, TokenizerFactory> tokenizerFactories,
public SynonymTokenFilterFactory(IndexSettings indexSettings, Environment env, AnalysisRegistry analysisRegistry,
String name, Settings settings) throws IOException {
super(indexSettings, name, settings);
@ -65,11 +65,13 @@ public class SynonymTokenFilterFactory extends AbstractTokenFilterFactory {
boolean expand = settings.getAsBoolean("expand", true);
String tokenizerName = settings.get("tokenizer", "whitespace");
final TokenizerFactory tokenizerFactory = tokenizerFactories.get(tokenizerName);
if (tokenizerFactory == null) {
AnalysisModule.AnalysisProvider<TokenizerFactory> tokenizerFactoryFactory =
analysisRegistry.getTokenizerProvider(tokenizerName, indexSettings);
if (tokenizerFactoryFactory == null) {
throw new IllegalArgumentException("failed to find tokenizer [" + tokenizerName + "] for synonym token filter");
}
final TokenizerFactory tokenizerFactory = tokenizerFactoryFactory.get(indexSettings, env, tokenizerName,
AnalysisRegistry.getSettingsFromIndexSettings(indexSettings, AnalysisRegistry.INDEX_ANALYSIS_TOKENIZER + "." + tokenizerName));
Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName) {

View File

@ -50,8 +50,8 @@ import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.search.fetch.parent.ParentFieldSubFetchPhase;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.ParentFieldSubFetchPhase;
import org.elasticsearch.search.lookup.LeafSearchLookup;
import org.elasticsearch.search.lookup.SearchLookup;

View File

@ -34,11 +34,11 @@ import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsContext;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsContext;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.sort.SortAndFormats;
import org.elasticsearch.search.sort.SortBuilder;
@ -585,7 +585,7 @@ public final class InnerHitBuilder extends ToXContentToBytes implements Writeabl
for (ScriptField field : scriptFields) {
SearchScript searchScript = innerHitsContext.scriptService().search(innerHitsContext.lookup(), field.script(),
ScriptContext.Standard.SEARCH, Collections.emptyMap());
innerHitsContext.scriptFields().add(new org.elasticsearch.search.fetch.script.ScriptFieldsContext.ScriptField(
innerHitsContext.scriptFields().add(new org.elasticsearch.search.fetch.subphase.ScriptFieldsContext.ScriptField(
field.fieldName(), searchScript, field.ignoreFailure()));
}
}

View File

@ -37,7 +37,7 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.Signi
import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgPipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModel;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.highlight.Highlighter;
import org.elasticsearch.search.fetch.subphase.highlight.Highlighter;
import org.elasticsearch.search.suggest.Suggester;
import java.util.List;
@ -143,9 +143,9 @@ public interface SearchPlugin {
/**
* Specification of search time behavior extension like a custom {@link MovAvgModel} or {@link ScoreFunction}.
*
* @param W the type of the main {@link NamedWriteable} for this spec. All specs have this but it isn't always *for* the same thing
* @param <W> the type of the main {@link NamedWriteable} for this spec. All specs have this but it isn't always *for* the same thing
* though, usually it is some sort of builder sent from the coordinating node to the data nodes executing the behavior
* @param P the type of the parser for this spec. The parser runs on the coordinating node, converting {@link XContent} into the
* @param <P> the type of the parser for this spec. The parser runs on the coordinating node, converting {@link XContent} into the
* behavior to execute
*/
class SearchExtensionSpec<W extends NamedWriteable, P> {

View File

@ -39,7 +39,7 @@ import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.RestResponse;
import org.elasticsearch.rest.action.support.RestActions;
import org.elasticsearch.rest.action.support.RestBuilderListener;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import java.io.IOException;

View File

@ -35,7 +35,7 @@ import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.RestResponse;
import org.elasticsearch.rest.action.support.RestActions;
import org.elasticsearch.rest.action.support.RestBuilderListener;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import static org.elasticsearch.rest.RestRequest.Method.GET;
import static org.elasticsearch.rest.RestStatus.NOT_FOUND;

View File

@ -33,7 +33,7 @@ import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.RestResponse;
import org.elasticsearch.rest.action.support.RestResponseListener;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import java.io.IOException;

View File

@ -31,7 +31,7 @@ import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.support.RestActions;
import org.elasticsearch.rest.action.support.RestToXContentListener;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import static org.elasticsearch.rest.RestRequest.Method.GET;
import static org.elasticsearch.rest.RestRequest.Method.POST;

View File

@ -43,7 +43,7 @@ import org.elasticsearch.rest.action.support.RestStatusToXContentListener;
import org.elasticsearch.search.Scroll;
import org.elasticsearch.search.aggregations.AggregatorParsers;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.search.suggest.SuggestBuilder;

View File

@ -25,7 +25,7 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.search.highlight.HighlightField;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightField;
import java.util.Map;

View File

@ -19,13 +19,6 @@
package org.elasticsearch.search;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.function.Consumer;
import java.util.function.Function;
import org.apache.lucene.search.BooleanQuery;
import org.elasticsearch.common.NamedRegistry;
import org.elasticsearch.common.ParseField;
@ -253,18 +246,18 @@ import org.elasticsearch.search.aggregations.pipeline.serialdiff.SerialDiffPipel
import org.elasticsearch.search.controller.SearchPhaseController;
import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.explain.ExplainFetchSubPhase;
import org.elasticsearch.search.fetch.matchedqueries.MatchedQueriesFetchSubPhase;
import org.elasticsearch.search.fetch.parent.ParentFieldSubFetchPhase;
import org.elasticsearch.search.fetch.script.ScriptFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.source.FetchSourceSubPhase;
import org.elasticsearch.search.fetch.version.VersionFetchSubPhase;
import org.elasticsearch.search.highlight.FastVectorHighlighter;
import org.elasticsearch.search.highlight.HighlightPhase;
import org.elasticsearch.search.highlight.Highlighter;
import org.elasticsearch.search.highlight.PlainHighlighter;
import org.elasticsearch.search.highlight.PostingsHighlighter;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.ExplainFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.FetchSourceSubPhase;
import org.elasticsearch.search.fetch.subphase.MatchedQueriesFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.ParentFieldSubFetchPhase;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.VersionFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.highlight.FastVectorHighlighter;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightPhase;
import org.elasticsearch.search.fetch.subphase.highlight.Highlighter;
import org.elasticsearch.search.fetch.subphase.highlight.PlainHighlighter;
import org.elasticsearch.search.fetch.subphase.highlight.PostingsHighlighter;
import org.elasticsearch.search.rescore.QueryRescorerBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.sort.FieldSortBuilder;
@ -283,6 +276,13 @@ import org.elasticsearch.search.suggest.phrase.SmoothingModel;
import org.elasticsearch.search.suggest.phrase.StupidBackoff;
import org.elasticsearch.search.suggest.term.TermSuggester;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.function.Consumer;
import java.util.function.Function;
import static java.util.Collections.unmodifiableMap;
import static java.util.Objects.requireNonNull;
@ -361,7 +361,7 @@ public class SearchModule extends AbstractModule {
/**
* The registry of {@link MovAvgModel}s.
*/
public ParseFieldRegistry<MovAvgModel.AbstractModelParser> getMovingAverageMdelParserRegistry() {
public ParseFieldRegistry<MovAvgModel.AbstractModelParser> getMovingAverageModelParserRegistry() {
return movingAverageModelParserRegistry;
}

View File

@ -67,11 +67,11 @@ import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.QueryFetchSearchResult;
import org.elasticsearch.search.fetch.ScrollQueryFetchSearchResult;
import org.elasticsearch.search.fetch.ShardFetchRequest;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsContext;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsContext.DocValueField;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.script.ScriptFieldsContext.ScriptField;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsContext;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsContext.DocValueField;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext.ScriptField;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.DefaultSearchContext;
import org.elasticsearch.search.internal.InternalScrollSearchRequest;
import org.elasticsearch.search.internal.ScrollContext;
@ -269,7 +269,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
loadOrExecuteQueryPhase(request, context);
if (hasHits(context.queryResult()) == false && context.scrollContext() == null) {
if (context.queryResult().hasHits() == false && context.scrollContext() == null) {
freeContext(context.id());
} else {
contextProcessedSuccessfully(context);
@ -324,7 +324,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
operationListener.onPreQueryPhase(context);
long time = System.nanoTime();
queryPhase.execute(context);
if (hasHits(context.queryResult()) == false && context.scrollContext() == null) {
if (context.queryResult().hasHits() == false && context.scrollContext() == null) {
// no hits, we can release the context since there will be no fetch phase
freeContext(context.id());
} else {
@ -861,11 +861,6 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
context.docIdsToLoad(docIdsToLoad, 0, docIdsToLoad.length);
}
private static boolean hasHits(final QuerySearchResult searchResult) {
return searchResult.topDocs().scoreDocs.length > 0 ||
(searchResult.suggest() != null && searchResult.suggest().hasScoreDocs());
}
private void processScroll(InternalScrollSearchRequest request, SearchContext context) {
// process scroll
context.from(context.from() + context.size());

View File

@ -38,8 +38,8 @@ import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.sort.SortBuilders;

View File

@ -28,11 +28,11 @@ import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsContext;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsContext.DocValueField;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsContext;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsContext.DocValueField;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.SubSearchContext;
import org.elasticsearch.search.sort.SortAndFormats;
import org.elasticsearch.search.sort.SortBuilder;
@ -107,7 +107,7 @@ public class TopHitsAggregatorFactory extends AggregatorFactory<TopHitsAggregato
for (ScriptField field : scriptFields) {
SearchScript searchScript = subSearchContext.scriptService().search(subSearchContext.lookup(), field.script(),
ScriptContext.Standard.SEARCH, Collections.emptyMap());
subSearchContext.scriptFields().add(new org.elasticsearch.search.fetch.script.ScriptFieldsContext.ScriptField(
subSearchContext.scriptFields().add(new org.elasticsearch.search.fetch.subphase.ScriptFieldsContext.ScriptField(
field.fieldName(), searchScript, field.ignoreFailure()));
}
}

View File

@ -44,8 +44,8 @@ import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorParsers;
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.searchafter.SearchAfterBuilder;

View File

@ -181,7 +181,7 @@ public class SearchPhaseController extends AbstractComponent {
} else {
// lets see if we only got hits from a single shard, if so, we can optimize...
for (AtomicArray.Entry<? extends QuerySearchResultProvider> entry : results) {
if (entry.value.queryResult().topDocs().scoreDocs.length > 0) {
if (entry.value.queryResult().hasHits()) {
if (result != null) { // we already have one, can't really optimize
canOptimize = false;
break;

View File

@ -45,8 +45,8 @@ import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchParseElement;
import org.elasticsearch.search.SearchPhase;
import org.elasticsearch.search.fetch.innerhits.InnerHitsFetchSubPhase;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.InnerHitsFetchSubPhase;
import org.elasticsearch.search.internal.InternalSearchHit;
import org.elasticsearch.search.internal.InternalSearchHitField;
import org.elasticsearch.search.internal.InternalSearchHits;

View File

@ -31,7 +31,7 @@ import java.util.HashMap;
import java.util.Map;
/**
* Sub phase within the fetch phase used to fetch things *about* the documents highlghting.
* Sub phase within the fetch phase used to fetch things *about* the documents like highlghting or matched queries.
*/
public interface FetchSubPhase {

View File

@ -19,7 +19,7 @@
package org.elasticsearch.search.fetch;
import org.elasticsearch.search.fetch.docvalues.DocValueFieldsContext;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsContext;
/**
* All configuration and context needed by the FetchSubPhase to execute on hits.

View File

@ -0,0 +1,25 @@
/*
* 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.
*/
/**
* Search phase that fetches the top hits from the shards after the results of the query phase have been merged. Pluggable by implementing
* {@link org.elasticsearch.search.fetch.FetchSubPhase} and
* {@link org.elasticsearch.plugins.SearchPlugin#getFetchSubPhases(org.elasticsearch.plugins.SearchPlugin.FetchPhaseConstructionContext)}.
*/
package org.elasticsearch.search.fetch;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.fetch.docvalues;
package org.elasticsearch.search.fetch.subphase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.fetch.docvalues;
package org.elasticsearch.search.fetch.subphase;
import org.elasticsearch.index.fielddata.AtomicFieldData;
import org.elasticsearch.index.fielddata.ScriptDocValues;

View File

@ -16,19 +16,18 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.fetch.explain;
package org.elasticsearch.search.fetch.subphase;
import org.apache.lucene.search.Explanation;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.internal.InternalSearchHit;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.rescore.RescoreSearchContext;
import java.io.IOException;
/**
*
* Explains the scoring calculations for the top hits.
*/
public final class ExplainFetchSubPhase implements FetchSubPhase {
@ -47,7 +46,8 @@ public final class ExplainFetchSubPhase implements FetchSubPhase {
// we use the top level doc id, since we work with the top level searcher
hitContext.hit().explanation(explanation);
} catch (IOException e) {
throw new FetchPhaseExecutionException(context, "Failed to explain doc [" + hitContext.hit().type() + "#" + hitContext.hit().id() + "]", e);
throw new FetchPhaseExecutionException(context, "Failed to explain doc [" + hitContext.hit().type() + "#"
+ hitContext.hit().id() + "]", e);
} finally {
context.clearReleasables(SearchContext.Lifetime.COLLECTION);
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.fetch.source;
package org.elasticsearch.search.fetch.subphase;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.ParseField;
@ -38,6 +38,7 @@ import java.util.Arrays;
import java.util.List;
/**
* Context used to fetch the {@code _source}.
*/
public class FetchSourceContext implements Streamable, ToXContent {
@ -175,8 +176,8 @@ public class FetchSourceContext implements Streamable, ToXContent {
if (token == XContentParser.Token.VALUE_STRING) {
includesList.add(parser.text());
} else {
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token
+ " in [" + currentFieldName + "].", parser.getTokenLocation());
}
}
includes = includesList.toArray(new String[includesList.size()]);
@ -186,14 +187,14 @@ public class FetchSourceContext implements Streamable, ToXContent {
if (token == XContentParser.Token.VALUE_STRING) {
excludesList.add(parser.text());
} else {
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token
+ " in [" + currentFieldName + "].", parser.getTokenLocation());
}
}
excludes = excludesList.toArray(new String[excludesList.size()]);
} else {
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token
+ " in [" + currentFieldName + "].", parser.getTokenLocation());
}
} else if (token == XContentParser.Token.VALUE_STRING) {
if (context.getParseFieldMatcher().match(currentFieldName, INCLUDES_FIELD)) {

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.fetch.source;
package org.elasticsearch.search.fetch.subphase;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.io.stream.BytesStreamOutput;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.fetch.innerhits;
package org.elasticsearch.search.fetch.subphase;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.fetch.innerhits;
package org.elasticsearch.search.fetch.subphase;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.ScoreDoc;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.fetch.matchedqueries;
package org.elasticsearch.search.fetch.subphase;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.fetch.parent;
package org.elasticsearch.search.fetch.subphase;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.SortedDocValues;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.fetch.script;
package org.elasticsearch.search.fetch.subphase;
import org.elasticsearch.script.SearchScript;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.fetch.script;
package org.elasticsearch.search.fetch.subphase;
import org.elasticsearch.script.LeafSearchScript;
import org.elasticsearch.search.SearchHitField;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.fetch.version;
package org.elasticsearch.search.fetch.subphase;
import org.apache.lucene.index.NumericDocValues;
import org.elasticsearch.ElasticsearchException;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.search.highlight.SimpleFragmenter;
import org.apache.lucene.search.highlight.SimpleSpanFragmenter;
@ -32,7 +32,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.highlight.HighlightBuilder.Order;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.Order;
import java.io.IOException;
import java.util.Arrays;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Query;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.search.highlight.Encoder;
import org.apache.lucene.search.vectorhighlight.BaseFragmentsBuilder;
@ -37,9 +37,6 @@ import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.highlight.vectorhighlight.SimpleFragmentsBuilder;
import org.elasticsearch.search.highlight.vectorhighlight.SourceScoreOrderFragmentsBuilder;
import org.elasticsearch.search.highlight.vectorhighlight.SourceSimpleFragmentsBuilder;
import org.elasticsearch.search.internal.SearchContext;
import java.util.Collections;
@ -68,7 +65,8 @@ public class FastVectorHighlighter implements Highlighter {
FieldMapper mapper = highlighterContext.mapper;
if (canHighlight(mapper) == false) {
throw new IllegalArgumentException("the field [" + highlighterContext.fieldName + "] should be indexed with term vector with position offsets to be used with fast vector highlighter");
throw new IllegalArgumentException("the field [" + highlighterContext.fieldName
+ "] should be indexed with term vector with position offsets to be used with fast vector highlighter");
}
Encoder encoder = field.fieldOptions().encoder().equals("html") ? HighlightUtils.Encoders.HTML : HighlightUtils.Encoders.DEFAULT;
@ -82,14 +80,22 @@ public class FastVectorHighlighter implements Highlighter {
FieldQuery fieldQuery;
if (field.fieldOptions().requireFieldMatch()) {
if (cache.fieldMatchFieldQuery == null) {
// we use top level reader to rewrite the query against all readers, with use caching it across hits (and across readers...)
cache.fieldMatchFieldQuery = new CustomFieldQuery(highlighterContext.query, hitContext.topLevelReader(), true, field.fieldOptions().requireFieldMatch());
/*
* we use top level reader to rewrite the query against all readers, with use caching it across hits (and across
* readers...)
*/
cache.fieldMatchFieldQuery = new CustomFieldQuery(highlighterContext.query, hitContext.topLevelReader(),
true, field.fieldOptions().requireFieldMatch());
}
fieldQuery = cache.fieldMatchFieldQuery;
} else {
if (cache.noFieldMatchFieldQuery == null) {
// we use top level reader to rewrite the query against all readers, with use caching it across hits (and across readers...)
cache.noFieldMatchFieldQuery = new CustomFieldQuery(highlighterContext.query, hitContext.topLevelReader(), true, field.fieldOptions().requireFieldMatch());
/*
* we use top level reader to rewrite the query against all readers, with use caching it across hits (and across
* readers...)
*/
cache.noFieldMatchFieldQuery = new CustomFieldQuery(highlighterContext.query, hitContext.topLevelReader(),
true, field.fieldOptions().requireFieldMatch());
}
fieldQuery = cache.noFieldMatchFieldQuery;
}
@ -100,31 +106,40 @@ public class FastVectorHighlighter implements Highlighter {
BaseFragmentsBuilder fragmentsBuilder;
BoundaryScanner boundaryScanner = DEFAULT_BOUNDARY_SCANNER;
if (field.fieldOptions().boundaryMaxScan() != SimpleBoundaryScanner.DEFAULT_MAX_SCAN || field.fieldOptions().boundaryChars() != SimpleBoundaryScanner.DEFAULT_BOUNDARY_CHARS) {
boundaryScanner = new SimpleBoundaryScanner(field.fieldOptions().boundaryMaxScan(), field.fieldOptions().boundaryChars());
if (field.fieldOptions().boundaryMaxScan() != SimpleBoundaryScanner.DEFAULT_MAX_SCAN
|| field.fieldOptions().boundaryChars() != SimpleBoundaryScanner.DEFAULT_BOUNDARY_CHARS) {
boundaryScanner = new SimpleBoundaryScanner(field.fieldOptions().boundaryMaxScan(),
field.fieldOptions().boundaryChars());
}
boolean forceSource = context.highlight().forceSource(field);
if (field.fieldOptions().numberOfFragments() == 0) {
fragListBuilder = new SingleFragListBuilder();
if (!forceSource && mapper.fieldType().stored()) {
fragmentsBuilder = new SimpleFragmentsBuilder(mapper, field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
fragmentsBuilder = new SimpleFragmentsBuilder(mapper, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), boundaryScanner);
} else {
fragmentsBuilder = new SourceSimpleFragmentsBuilder(mapper, context, field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
fragmentsBuilder = new SourceSimpleFragmentsBuilder(mapper, context,
field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
}
} else {
fragListBuilder = field.fieldOptions().fragmentOffset() == -1 ? new SimpleFragListBuilder() : new SimpleFragListBuilder(field.fieldOptions().fragmentOffset());
fragListBuilder = field.fieldOptions().fragmentOffset() == -1 ?
new SimpleFragListBuilder() : new SimpleFragListBuilder(field.fieldOptions().fragmentOffset());
if (field.fieldOptions().scoreOrdered()) {
if (!forceSource && mapper.fieldType().stored()) {
fragmentsBuilder = new ScoreOrderFragmentsBuilder(field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
fragmentsBuilder = new ScoreOrderFragmentsBuilder(field.fieldOptions().preTags(),
field.fieldOptions().postTags(), boundaryScanner);
} else {
fragmentsBuilder = new SourceScoreOrderFragmentsBuilder(mapper, context, field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
fragmentsBuilder = new SourceScoreOrderFragmentsBuilder(mapper, context,
field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
}
} else {
if (!forceSource && mapper.fieldType().stored()) {
fragmentsBuilder = new SimpleFragmentsBuilder(mapper, field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
fragmentsBuilder = new SimpleFragmentsBuilder(mapper, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), boundaryScanner);
} else {
fragmentsBuilder = new SourceSimpleFragmentsBuilder(mapper, context, field.fieldOptions().preTags(), field.fieldOptions().postTags(), boundaryScanner);
fragmentsBuilder = new SourceSimpleFragmentsBuilder(mapper, context, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), boundaryScanner);
}
}
}
@ -146,16 +161,20 @@ public class FastVectorHighlighter implements Highlighter {
String[] fragments;
// a HACK to make highlighter do highlighting, even though its using the single frag list builder
int numberOfFragments = field.fieldOptions().numberOfFragments() == 0 ? Integer.MAX_VALUE : field.fieldOptions().numberOfFragments();
int fragmentCharSize = field.fieldOptions().numberOfFragments() == 0 ? Integer.MAX_VALUE : field.fieldOptions().fragmentCharSize();
int numberOfFragments = field.fieldOptions().numberOfFragments() == 0 ?
Integer.MAX_VALUE : field.fieldOptions().numberOfFragments();
int fragmentCharSize = field.fieldOptions().numberOfFragments() == 0 ?
Integer.MAX_VALUE : field.fieldOptions().fragmentCharSize();
// we highlight against the low level reader and docId, because if we load source, we want to reuse it if possible
// Only send matched fields if they were requested to save time.
if (field.fieldOptions().matchedFields() != null && !field.fieldOptions().matchedFields().isEmpty()) {
fragments = cache.fvh.getBestFragments(fieldQuery, hitContext.reader(), hitContext.docId(), mapper.fieldType().name(), field.fieldOptions().matchedFields(), fragmentCharSize,
numberOfFragments, entry.fragListBuilder, entry.fragmentsBuilder, field.fieldOptions().preTags(), field.fieldOptions().postTags(), encoder);
fragments = cache.fvh.getBestFragments(fieldQuery, hitContext.reader(), hitContext.docId(), mapper.fieldType().name(),
field.fieldOptions().matchedFields(), fragmentCharSize, numberOfFragments, entry.fragListBuilder,
entry.fragmentsBuilder, field.fieldOptions().preTags(), field.fieldOptions().postTags(), encoder);
} else {
fragments = cache.fvh.getBestFragments(fieldQuery, hitContext.reader(), hitContext.docId(), mapper.fieldType().name(), fragmentCharSize,
numberOfFragments, entry.fragListBuilder, entry.fragmentsBuilder, field.fieldOptions().preTags(), field.fieldOptions().postTags(), encoder);
fragments = cache.fvh.getBestFragments(fieldQuery, hitContext.reader(), hitContext.docId(), mapper.fieldType().name(),
fragmentCharSize, numberOfFragments, entry.fragListBuilder, entry.fragmentsBuilder, field.fieldOptions().preTags(),
field.fieldOptions().postTags(), encoder);
}
if (fragments != null && fragments.length > 0) {
@ -183,14 +202,13 @@ public class FastVectorHighlighter implements Highlighter {
@Override
public boolean canHighlight(FieldMapper fieldMapper) {
return fieldMapper.fieldType().storeTermVectors() && fieldMapper.fieldType().storeTermVectorOffsets() && fieldMapper.fieldType().storeTermVectorPositions();
return fieldMapper.fieldType().storeTermVectors() && fieldMapper.fieldType().storeTermVectorOffsets()
&& fieldMapper.fieldType().storeTermVectorPositions();
}
private class MapperHighlightEntry {
public FragListBuilder fragListBuilder;
public FragmentsBuilder fragmentsBuilder;
public org.apache.lucene.search.highlight.Highlighter highlighter;
}
private class HighlighterEntry {

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.highlight.vectorhighlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.document.Field;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.vectorhighlight.SimpleBoundaryScanner;
@ -32,7 +32,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.highlight.SearchContextHighlight.FieldOptions;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight.FieldOptions;
import java.io.IOException;
import java.util.ArrayList;
@ -279,7 +279,7 @@ public class HighlightBuilder extends AbstractHighlighterBuilder<HighlightBuilde
globalOptionsBuilder.merge(defaultOptions);
// create field options
Collection<org.elasticsearch.search.highlight.SearchContextHighlight.Field> fieldOptions = new ArrayList<>();
Collection<org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight.Field> fieldOptions = new ArrayList<>();
for (Field field : this.fields) {
final SearchContextHighlight.FieldOptions.Builder fieldOptionsBuilder = new SearchContextHighlight.FieldOptions.Builder();
fieldOptionsBuilder.fragmentOffset(field.fragmentOffset);

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.component.AbstractComponent;
@ -67,7 +67,8 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
if (context.highlight().forceSource(field)) {
SourceFieldMapper sourceFieldMapper = context.mapperService().documentMapper(hitContext.hit().type()).sourceMapper();
if (!sourceFieldMapper.enabled()) {
throw new IllegalArgumentException("source is forced for fields " + fieldNamesToHighlight + " but type [" + hitContext.hit().type() + "] has disabled _source");
throw new IllegalArgumentException("source is forced for fields " + fieldNamesToHighlight
+ " but type [" + hitContext.hit().type() + "] has disabled _source");
}
}
@ -105,11 +106,16 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
}
Highlighter highlighter = highlighters.get(highlighterType);
if (highlighter == null) {
throw new IllegalArgumentException("unknown highlighter type [" + highlighterType + "] for the field [" + fieldName + "]");
throw new IllegalArgumentException("unknown highlighter type [" + highlighterType
+ "] for the field [" + fieldName + "]");
}
Query highlightQuery = field.fieldOptions().highlightQuery() == null ? context.parsedQuery().query() : field.fieldOptions().highlightQuery();
HighlighterContext highlighterContext = new HighlighterContext(fieldName, field, fieldMapper, context, hitContext, highlightQuery);
Query highlightQuery = field.fieldOptions().highlightQuery();
if (highlightQuery == null) {
highlightQuery = context.parsedQuery().query();
}
HighlighterContext highlighterContext = new HighlighterContext(fieldName, field, fieldMapper, context,
hitContext, highlightQuery);
if ((highlighter.canHighlight(fieldMapper) == false) && fieldNameContainsWildcards) {
// if several fieldnames matched the wildcard then we want to skip those that we cannot highlight

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.search.highlight.DefaultEncoder;
import org.apache.lucene.search.highlight.Encoder;
@ -43,7 +43,11 @@ public final class HighlightUtils {
}
static List<Object> loadFieldValues(SearchContextHighlight.Field field, FieldMapper mapper, SearchContext searchContext, FetchSubPhase.HitContext hitContext) throws IOException {
/**
* Load field values for highlighting.
*/
public static List<Object> loadFieldValues(SearchContextHighlight.Field field, FieldMapper mapper, SearchContext searchContext,
FetchSubPhase.HitContext hitContext) throws IOException {
//percolator needs to always load from source, thus it sets the global force source to true
boolean forceSource = searchContext.highlight().forceSource(field);
List<Object> textsToHighlight;
@ -65,7 +69,7 @@ public final class HighlightUtils {
}
static class Encoders {
static Encoder DEFAULT = new DefaultEncoder();
static Encoder HTML = new SimpleHTMLEncoder();
static final Encoder DEFAULT = new DefaultEncoder();
static final Encoder HTML = new SimpleHTMLEncoder();
}
}

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.elasticsearch.index.mapper.FieldMapper;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.search.Query;
import org.elasticsearch.index.mapper.FieldMapper;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
@ -68,11 +68,13 @@ public class PlainHighlighter implements Highlighter {
hitContext.cache().put(CACHE_KEY, mappers);
}
@SuppressWarnings("unchecked")
Map<FieldMapper, org.apache.lucene.search.highlight.Highlighter> cache = (Map<FieldMapper, org.apache.lucene.search.highlight.Highlighter>) hitContext.cache().get(CACHE_KEY);
Map<FieldMapper, org.apache.lucene.search.highlight.Highlighter> cache =
(Map<FieldMapper, org.apache.lucene.search.highlight.Highlighter>) hitContext.cache().get(CACHE_KEY);
org.apache.lucene.search.highlight.Highlighter entry = cache.get(mapper);
if (entry == null) {
QueryScorer queryScorer = new CustomQueryScorer(highlighterContext.query, field.fieldOptions().requireFieldMatch() ? mapper.fieldType().name() : null);
QueryScorer queryScorer = new CustomQueryScorer(highlighterContext.query,
field.fieldOptions().requireFieldMatch() ? mapper.fieldType().name() : null);
queryScorer.setExpandMultiTermQuery(true);
Fragmenter fragmenter;
if (field.fieldOptions().numberOfFragments() == 0) {
@ -84,7 +86,8 @@ public class PlainHighlighter implements Highlighter {
} else if ("span".equals(field.fieldOptions().fragmenter())) {
fragmenter = new SimpleSpanFragmenter(queryScorer, field.fieldOptions().fragmentCharSize());
} else {
throw new IllegalArgumentException("unknown fragmenter option [" + field.fieldOptions().fragmenter() + "] for the field [" + highlighterContext.fieldName + "]");
throw new IllegalArgumentException("unknown fragmenter option [" + field.fieldOptions().fragmenter()
+ "] for the field [" + highlighterContext.fieldName + "]");
}
Formatter formatter = new SimpleHTMLFormatter(field.fieldOptions().preTags()[0], field.fieldOptions().postTags()[0]);
@ -181,7 +184,8 @@ public class PlainHighlighter implements Highlighter {
return true;
}
private static int findGoodEndForNoHighlightExcerpt(int noMatchSize, Analyzer analyzer, String fieldName, String contents) throws IOException {
private static int findGoodEndForNoHighlightExcerpt(int noMatchSize, Analyzer analyzer, String fieldName, String contents)
throws IOException {
try (TokenStream tokenStream = analyzer.tokenStream(fieldName, contents)) {
if (!tokenStream.hasAttribute(OffsetAttribute.class)) {
// Can't split on term boundaries without offsets

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.IndexOptions;
@ -33,6 +33,7 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightUtils.Encoders;
import java.io.IOException;
import java.text.BreakIterator;
@ -53,7 +54,8 @@ public class PostingsHighlighter implements Highlighter {
FieldMapper fieldMapper = highlighterContext.mapper;
SearchContextHighlight.Field field = highlighterContext.field;
if (canHighlight(fieldMapper) == false) {
throw new IllegalArgumentException("the field [" + highlighterContext.fieldName + "] should be indexed with positions and offsets in the postings list to be used with postings highlighter");
throw new IllegalArgumentException("the field [" + highlighterContext.fieldName
+ "] should be indexed with positions and offsets in the postings list to be used with postings highlighter");
}
SearchContext context = highlighterContext.context;
@ -67,8 +69,9 @@ public class PostingsHighlighter implements Highlighter {
MapperHighlighterEntry mapperHighlighterEntry = highlighterEntry.mappers.get(fieldMapper);
if (mapperHighlighterEntry == null) {
Encoder encoder = field.fieldOptions().encoder().equals("html") ? HighlightUtils.Encoders.HTML : HighlightUtils.Encoders.DEFAULT;
CustomPassageFormatter passageFormatter = new CustomPassageFormatter(field.fieldOptions().preTags()[0], field.fieldOptions().postTags()[0], encoder);
Encoder encoder = field.fieldOptions().encoder().equals("html") ? Encoders.HTML : Encoders.DEFAULT;
CustomPassageFormatter passageFormatter = new CustomPassageFormatter(
field.fieldOptions().preTags()[0], field.fieldOptions().postTags()[0], encoder);
mapperHighlighterEntry = new MapperHighlighterEntry(passageFormatter);
}
@ -83,17 +86,20 @@ public class PostingsHighlighter implements Highlighter {
//so we don't lose the distinction between the different values of a field and we get back a snippet per value
String fieldValue = mergeFieldValues(fieldValues, HighlightUtils.NULL_SEPARATOR);
CustomSeparatorBreakIterator breakIterator = new CustomSeparatorBreakIterator(HighlightUtils.NULL_SEPARATOR);
highlighter = new CustomPostingsHighlighter(analyzer, mapperHighlighterEntry.passageFormatter, breakIterator, fieldValue, field.fieldOptions().noMatchSize() > 0);
highlighter = new CustomPostingsHighlighter(analyzer, mapperHighlighterEntry.passageFormatter, breakIterator,
fieldValue, field.fieldOptions().noMatchSize() > 0);
numberOfFragments = fieldValues.size(); //we are highlighting the whole content, one snippet per value
} else {
//using paragraph separator we make sure that each field value holds a discrete passage for highlighting
String fieldValue = mergeFieldValues(fieldValues, HighlightUtils.PARAGRAPH_SEPARATOR);
highlighter = new CustomPostingsHighlighter(analyzer, mapperHighlighterEntry.passageFormatter, fieldValue, field.fieldOptions().noMatchSize() > 0);
highlighter = new CustomPostingsHighlighter(analyzer, mapperHighlighterEntry.passageFormatter,
fieldValue, field.fieldOptions().noMatchSize() > 0);
numberOfFragments = field.fieldOptions().numberOfFragments();
}
IndexSearcher searcher = new IndexSearcher(hitContext.reader());
Snippet[] fieldSnippets = highlighter.highlightField(fieldMapper.fieldType().name(), highlighterContext.query, searcher, hitContext.docId(), numberOfFragments);
Snippet[] fieldSnippets = highlighter.highlightField(fieldMapper.fieldType().name(), highlighterContext.query, searcher,
hitContext.docId(), numberOfFragments);
for (Snippet fieldSnippet : fieldSnippets) {
if (Strings.hasText(fieldSnippet.getText())) {
snippets.add(fieldSnippet);

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.search.highlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.search.Query;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight.vectorhighlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.highlight.Encoder;
@ -40,6 +40,7 @@ public class SimpleFragmentsBuilder extends org.apache.lucene.search.vectorhighl
@Override
protected String makeFragment( StringBuilder buffer, int[] index, Field[] values, WeightedFragInfo fragInfo,
String[] preTags, String[] postTags, Encoder encoder ){
return super.makeFragment(buffer, index, values, FragmentBuilderHelper.fixWeightedFragInfo(mapper, values, fragInfo), preTags, postTags, encoder);
return super.makeFragment(buffer, index, values, FragmentBuilderHelper.fixWeightedFragInfo(mapper, values, fragInfo),
preTags, postTags, encoder);
}
}

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight.vectorhighlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.TextField;
@ -66,6 +66,7 @@ public class SourceScoreOrderFragmentsBuilder extends ScoreOrderFragmentsBuilder
@Override
protected String makeFragment( StringBuilder buffer, int[] index, Field[] values, WeightedFragInfo fragInfo,
String[] preTags, String[] postTags, Encoder encoder ){
return super.makeFragment(buffer, index, values, FragmentBuilderHelper.fixWeightedFragInfo(mapper, values, fragInfo), preTags, postTags, encoder);
return super.makeFragment(buffer, index, values, FragmentBuilderHelper.fixWeightedFragInfo(mapper, values, fragInfo),
preTags, postTags, encoder);
}
}

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.highlight.vectorhighlight;
package org.elasticsearch.search.fetch.subphase.highlight;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.TextField;

View File

@ -0,0 +1,25 @@
/*
* 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.
*/
/**
* Fetch sub phase that extracts significant portions of string fields, marking the matches. Pluggable by implementing
* {@link org.elasticsearch.search.fetch.subphase.highlight.Highlighter} and
* {@link org.elasticsearch.plugins.SearchPlugin#getHighlighters()}.
*/
package org.elasticsearch.search.fetch.subphase.highlight;

View File

@ -0,0 +1,23 @@
/*
* 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.
*/
/**
* Built in {@link org.elasticsearch.search.fetch.FetchSubPhase}s like matched queries and fetching {@code _source}.
*/
package org.elasticsearch.search.fetch.subphase;

View File

@ -133,6 +133,10 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable {
@Override
public Explanation explain(Query query, int doc) throws IOException {
if (aggregatedDfs != null) {
// dfs data is needed to explain the score
return super.explain(createNormalizedWeight(query, true), doc);
}
return in.explain(query, doc);
}

View File

@ -60,9 +60,9 @@ import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
import org.elasticsearch.search.fetch.script.ScriptFieldsContext;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.SearchContextHighlight;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight;
import org.elasticsearch.search.lookup.SearchLookup;
import org.elasticsearch.search.profile.Profilers;
import org.elasticsearch.search.query.QueryPhaseExecutionException;

View File

@ -45,10 +45,10 @@ import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.script.ScriptFieldsContext;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.SearchContextHighlight;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight;
import org.elasticsearch.search.lookup.SearchLookup;
import org.elasticsearch.search.profile.Profilers;
import org.elasticsearch.search.query.QuerySearchResult;

View File

@ -38,7 +38,7 @@ import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.highlight.HighlightField;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightField;
import org.elasticsearch.search.internal.InternalSearchHits.StreamContext.ShardTargetType;
import org.elasticsearch.search.lookup.SourceLookup;
@ -55,7 +55,7 @@ import static java.util.Collections.singletonMap;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.lucene.Lucene.readExplanation;
import static org.elasticsearch.common.lucene.Lucene.writeExplanation;
import static org.elasticsearch.search.highlight.HighlightField.readHighlightField;
import static org.elasticsearch.search.fetch.subphase.highlight.HighlightField.readHighlightField;
import static org.elasticsearch.search.internal.InternalSearchHitField.readSearchHitField;
/**

View File

@ -49,10 +49,10 @@ import org.elasticsearch.search.fetch.FetchPhase;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.script.ScriptFieldsContext;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.SearchContextHighlight;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight;
import org.elasticsearch.search.lookup.SearchLookup;
import org.elasticsearch.search.profile.Profilers;
import org.elasticsearch.search.query.QuerySearchResult;

View File

@ -24,9 +24,9 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.script.ScriptFieldsContext;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.SearchContextHighlight;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight;
import org.elasticsearch.search.lookup.SearchLookup;
import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.search.rescore.RescoreSearchContext;

View File

@ -188,6 +188,12 @@ public class QuerySearchResult extends QuerySearchResultProvider {
return this;
}
/** Returns true iff the result has hits */
public boolean hasHits() {
return (topDocs != null && topDocs.scoreDocs.length > 0) ||
(suggest != null && suggest.hasScoreDocs());
}
public static QuerySearchResult readQuerySearchResult(StreamInput in) throws IOException {
QuerySearchResult result = new QuerySearchResult();
result.readFrom(in);

View File

@ -117,6 +117,9 @@ public class TribeService extends AbstractLifecycleComponent {
sb.put(Node.NODE_MASTER_SETTING.getKey(), false);
sb.put(Node.NODE_DATA_SETTING.getKey(), false);
sb.put(Node.NODE_INGEST_SETTING.getKey(), false);
if (!NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.exists(settings)) {
sb.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), nodesSettings.size());
}
sb.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local"); // a tribe node should not use zen discovery
// nothing is going to be discovered, since no master will be elected
sb.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);

View File

@ -36,7 +36,7 @@ import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.highlight.DefaultEncoder;
import org.apache.lucene.store.Directory;
import org.elasticsearch.search.highlight.HighlightUtils;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightUtils;
import org.elasticsearch.test.ESTestCase;
import static org.hamcrest.CoreMatchers.equalTo;

View File

@ -19,7 +19,7 @@
package org.apache.lucene.search.postingshighlight;
import org.elasticsearch.search.highlight.HighlightUtils;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightUtils;
import org.elasticsearch.test.ESTestCase;
import java.text.BreakIterator;

View File

@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest;
import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse;
import org.elasticsearch.action.admin.indices.analyze.TransportAnalyzeAction;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
@ -51,12 +52,21 @@ public class TransportAnalyzeActionTests extends ESTestCase {
Settings indexSettings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID())
.put("index.analysis.filter.wordDelimiter.type", "word_delimiter")
.put("index.analysis.filter.wordDelimiter.split_on_numerics", false)
.put("index.analysis.analyzer.custom_analyzer.tokenizer", "whitespace")
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter")
.put("index.analysis.analyzer.custom_analyzer.tokenizer", "whitespace")
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter").build();
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter")
.put("index.analysis.tokenizer.trigram.type", "ngram")
.put("index.analysis.tokenizer.trigram.min_gram", 3)
.put("index.analysis.tokenizer.trigram.max_gram", 3)
.put("index.analysis.filter.synonym.type", "synonym")
.putArray("index.analysis.filter.synonym.synonyms", "kimchy => shay")
.put("index.analysis.filter.synonym.tokenizer", "trigram")
.put("index.analysis.filter.synonym.min_gram", 3)
.put("index.analysis.filter.synonym.max_gram", 3).build();
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", indexSettings);
environment = new Environment(settings);
registry = new AnalysisModule(environment, emptyList()).getAnalysisRegistry();
@ -168,6 +178,16 @@ public class TransportAnalyzeActionTests extends ESTestCase {
assertEquals("brown", tokens.get(2).getTerm());
assertEquals("fox", tokens.get(3).getTerm());
assertEquals("dog", tokens.get(4).getTerm());
request.analyzer(null);
request.tokenizer("trigram");
request.addTokenFilter("synonym");
request.text("kimchy");
analyze = TransportAnalyzeAction.analyze(request, AllFieldMapper.NAME, null, analysisService, registry, environment);
tokens = analyze.getTokens();
assertEquals(2, tokens.size());
assertEquals("sha", tokens.get(0).getTerm());
assertEquals("hay", tokens.get(1).getTerm());
}
public void testGetIndexAnalyserWithoutAnalysisService() throws IOException {

View File

@ -22,7 +22,7 @@ package org.elasticsearch.action.get;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;

View File

@ -38,6 +38,7 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.store.Store;
@ -118,6 +119,7 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
// manual collection or upon cluster forming.
.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), 2)
.put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.getKey(), "1s")
.build();
}

View File

@ -374,50 +374,59 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> cancel the move of the replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node3", false)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
if (randomBoolean()) {
logger.info("--> cancel the primary allocation (with allow_primary set to true)");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node1", true)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(rerouteResult.changed(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).iterator().next().primary(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
} else {
logger.info("--> cancel the move of the replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node3", false)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
logger.info("--> move the replica shard again");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", 0, "node2", "node3")), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> move the replica shard again");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", 0, "node2", "node3")), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> cancel the source replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node2", false)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).get(0).relocatingNodeId(), nullValue());
logger.info("--> cancel the source replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node2", false)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).get(0).relocatingNodeId(), nullValue());
logger.info("--> start the former target replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(STARTED).size(), equalTo(1));
logger.info("--> start the former target replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(STARTED).size(), equalTo(1));
logger.info("--> cancel the primary allocation (with allow_primary set to true)");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node1", true)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(rerouteResult.changed(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(STARTED).iterator().next().primary(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
logger.info("--> cancel the primary allocation (with allow_primary set to true)");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node1", true)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(rerouteResult.changed(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(STARTED).iterator().next().primary(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
}
}
public void testSerialization() throws Exception {

View File

@ -55,6 +55,7 @@ import org.elasticsearch.discovery.zen.ping.ZenPing;
import org.elasticsearch.discovery.zen.ping.ZenPingService;
import org.elasticsearch.discovery.zen.ping.unicast.UnicastZenPing;
import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.indices.store.IndicesStoreIntegrationIT;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
@ -207,6 +208,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
// TODO: Rarely use default settings form some of these
Settings nodeSettings = Settings.builder()
.put(settings)
.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), numberOfNodes)
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minimumMasterNode)
.build();

View File

@ -74,18 +74,14 @@ public class NodeEnvironmentTests extends ESTestCase {
}
public void testNodeLockSingleEnvironment() throws IOException {
final Settings settings = buildEnvSettings(Settings.builder()
.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), 1).build());
final Settings settings = buildEnvSettings(Settings.builder().put("node.max_local_storage_nodes", 1).build());
NodeEnvironment env = newNodeEnvironment(settings);
List<String> dataPaths = Environment.PATH_DATA_SETTING.get(settings);
try {
// Reuse the same location and attempt to lock again
new NodeEnvironment(settings, new Environment(settings));
fail("env has already locked all the data directories it is allowed");
} catch (IllegalStateException ex) {
assertThat(ex.getMessage(), containsString("Failed to obtain node lock"));
}
// Reuse the same location and attempt to lock again
IllegalStateException ex =
expectThrows(IllegalStateException.class, () -> new NodeEnvironment(settings, new Environment(settings)));
assertThat(ex.getMessage(), containsString("failed to obtain node lock"));
// Close the environment that holds the lock and make sure we can get the lock after release
env.close();
@ -121,7 +117,7 @@ public class NodeEnvironmentTests extends ESTestCase {
}
public void testNodeLockMultipleEnvironment() throws IOException {
final Settings settings = buildEnvSettings(Settings.EMPTY);
final Settings settings = buildEnvSettings(Settings.builder().put("node.max_local_storage_nodes", 2).build());
final NodeEnvironment first = newNodeEnvironment(settings);
List<String> dataPaths = Environment.PATH_DATA_SETTING.get(settings);
NodeEnvironment second = new NodeEnvironment(settings, new Environment(settings));

View File

@ -41,7 +41,6 @@ import java.io.IOException;
import static org.hamcrest.Matchers.startsWith;
public class SearchSlowLogTests extends ESSingleNodeTestCase {
@Override
protected SearchContext createSearchContext(IndexService indexService) {
@ -54,7 +53,7 @@ public class SearchSlowLogTests extends ESSingleNodeTestCase {
return new ShardSearchRequest() {
@Override
public ShardId shardId() {
return null;
return new ShardId(indexService.index(), 0);
}
@Override
@ -129,8 +128,8 @@ public class SearchSlowLogTests extends ESSingleNodeTestCase {
IndexService index = createIndex("foo");
// Turning off document logging doesn't log source[]
SearchContext searchContext = createSearchContext(index);
SearchSlowLog.SlowLogSearchContextPrinter p = new SearchSlowLog.SlowLogSearchContextPrinter(index.index(), searchContext, 10, true);
assertThat(p.toString(), startsWith(index.index().toString()));
SearchSlowLog.SlowLogSearchContextPrinter p = new SearchSlowLog.SlowLogSearchContextPrinter(searchContext, 10, true);
assertThat(p.toString(), startsWith("[foo][0]"));
}
public void testReformatSetting() {

View File

@ -25,7 +25,7 @@ import org.elasticsearch.common.geo.builders.ShapeBuilders;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.test.ESIntegTestCase;
import java.util.Arrays;

View File

@ -42,7 +42,7 @@ import org.elasticsearch.index.mapper.TypeFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.SortOrder;

View File

@ -30,7 +30,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.SortOrder;

View File

@ -51,8 +51,8 @@ import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilderTests;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilderTests;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.query;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.join.ScoreMode;
@ -27,7 +28,7 @@ import org.apache.lucene.search.join.ToParentBlockJoinQuery;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.subphase.InnerHitsContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.SortOrder;

View File

@ -22,6 +22,7 @@ package org.elasticsearch.indices.memory.breaker;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;
@ -40,6 +41,7 @@ public class CircuitBreakerNoopIT extends ESIntegTestCase {
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), 2)
.put(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING.getKey(), "noop")
// This is set low, because if the "noop" is not a noop, it will break
.put(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), "10b")

View File

@ -27,7 +27,7 @@ import org.elasticsearch.action.get.MultiGetResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.test.ESIntegTestCase;
import java.io.IOException;

View File

@ -36,12 +36,12 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.Signi
import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModel;
import org.elasticsearch.search.aggregations.pipeline.movavg.models.SimpleModel;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.explain.ExplainFetchSubPhase;
import org.elasticsearch.search.highlight.CustomHighlighter;
import org.elasticsearch.search.highlight.FastVectorHighlighter;
import org.elasticsearch.search.highlight.Highlighter;
import org.elasticsearch.search.highlight.PlainHighlighter;
import org.elasticsearch.search.highlight.PostingsHighlighter;
import org.elasticsearch.search.fetch.subphase.ExplainFetchSubPhase;
import org.elasticsearch.search.fetch.subphase.highlight.CustomHighlighter;
import org.elasticsearch.search.fetch.subphase.highlight.FastVectorHighlighter;
import org.elasticsearch.search.fetch.subphase.highlight.Highlighter;
import org.elasticsearch.search.fetch.subphase.highlight.PlainHighlighter;
import org.elasticsearch.search.fetch.subphase.highlight.PostingsHighlighter;
import org.elasticsearch.search.suggest.CustomSuggester;
import org.elasticsearch.search.suggest.Suggester;
import org.elasticsearch.search.suggest.completion.CompletionSuggester;

View File

@ -44,8 +44,8 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory.ExecutionMode;
import org.elasticsearch.search.aggregations.metrics.max.Max;
import org.elasticsearch.search.aggregations.metrics.tophits.TopHits;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.highlight.HighlightField;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightField;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;

View File

@ -27,8 +27,8 @@ import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.BaseAggregationTestCase;
import org.elasticsearch.search.aggregations.metrics.tophits.TopHitsAggregationBuilder;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilderTests;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilderTests;
import org.elasticsearch.search.sort.ScriptSortBuilder.ScriptSortType;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;

View File

@ -60,6 +60,7 @@ import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.endsWith;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
@ -147,6 +148,10 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase {
for (int i = 0; i < hits.length; ++i) {
SearchHit hit = hits[i];
assertThat(hit.explanation(), notNullValue());
assertThat(hit.explanation().getDetails().length, equalTo(1));
assertThat(hit.explanation().getDetails()[0].getDetails().length, equalTo(2));
assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDescription(),
endsWith("idf(docFreq=100, docCount=100)"));
assertThat("id[" + hit.id() + "] -> " + hit.explanation().toString(), hit.id(), equalTo(Integer.toString(100 - total - i - 1)));
}
total += hits.length;
@ -171,6 +176,10 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase {
for (int i = 0; i < hits.length; ++i) {
SearchHit hit = hits[i];
assertThat(hit.explanation(), notNullValue());
assertThat(hit.explanation().getDetails().length, equalTo(1));
assertThat(hit.explanation().getDetails()[0].getDetails().length, equalTo(2));
assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDescription(),
endsWith("idf(docFreq=100, docCount=100)"));
assertThat("id[" + hit.id() + "]", hit.id(), equalTo(Integer.toString(total + i)));
}
total += hits.length;
@ -317,6 +326,10 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase {
SearchHit hit = searchResponse.getHits().hits()[i];
// System.out.println(hit.shard() + ": " + hit.explanation());
assertThat(hit.explanation(), notNullValue());
assertThat(hit.explanation().getDetails().length, equalTo(1));
assertThat(hit.explanation().getDetails()[0].getDetails().length, equalTo(2));
assertThat(hit.explanation().getDetails()[0].getDetails()[0].getDescription(),
endsWith("idf(docFreq=100, docCount=100)"));
// assertThat("id[" + hit.id() + "]", hit.id(), equalTo(Integer.toString(100 - i - 1)));
assertThat("make sure we don't have duplicates", expectedIds.remove(hit.id()), notNullValue());
}

View File

@ -60,8 +60,8 @@ import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.AggregatorParsers;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilderTests;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilderTests;
import org.elasticsearch.search.rescore.QueryRescoreBuilderTests;
import org.elasticsearch.search.rescore.QueryRescorerBuilder;
import org.elasticsearch.search.searchafter.SearchAfterBuilder;

View File

@ -47,8 +47,8 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter;
import org.elasticsearch.search.aggregations.bucket.global.Global;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.highlight.HighlightField;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightField;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;

View File

@ -17,18 +17,17 @@
* under the License.
*/
package org.elasticsearch.search.innerhits;
package org.elasticsearch.search.fetch.subphase;
import org.apache.lucene.search.join.ScoreMode;
import org.apache.lucene.util.ArrayUtil;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.InnerHitBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.MockScriptEngine;
import org.elasticsearch.script.MockScriptPlugin;
@ -36,8 +35,7 @@ import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
@ -168,14 +166,16 @@ public class InnerHitsIT extends ESIntegTestCase {
new InnerHitBuilder().setHighlightBuilder(new HighlightBuilder().field("comments.message"))
.setExplain(true)
.addDocValueField("comments.message")
.addScriptField("script", new Script("5", ScriptService.ScriptType.INLINE, MockScriptEngine.NAME, Collections.emptyMap()))
.addScriptField("script",
new Script("5", ScriptService.ScriptType.INLINE, MockScriptEngine.NAME, Collections.emptyMap()))
.setSize(1)
)).get();
assertNoFailures(response);
innerHits = response.getHits().getAt(0).getInnerHits().get("comments");
assertThat(innerHits.getTotalHits(), equalTo(2L));
assertThat(innerHits.getHits().length, equalTo(1));
assertThat(innerHits.getAt(0).getHighlightFields().get("comments.message").getFragments()[0].string(), equalTo("<em>fox</em> eat quick"));
assertThat(innerHits.getAt(0).getHighlightFields().get("comments.message").getFragments()[0].string(),
equalTo("<em>fox</em> eat quick"));
assertThat(innerHits.getAt(0).explanation().toString(), containsString("weight(comments.message:fox in"));
assertThat(innerHits.getAt(0).getFields().get("comments.message").getValue().toString(), equalTo("eat"));
assertThat(innerHits.getAt(0).getFields().get("script").getValue().toString(), equalTo("5"));
@ -334,12 +334,14 @@ public class InnerHitsIT extends ESIntegTestCase {
int numChildDocs = child1InnerObjects[parent] = scaledRandomIntBetween(1, numDocs);
int limit = child1 + numChildDocs;
for (; child1 < limit; child1++) {
requestBuilders.add(client().prepareIndex("idx", "child1", String.format(Locale.ENGLISH, "%04d", child1)).setParent(parentId).setSource("{}"));
requestBuilders.add(client().prepareIndex("idx", "child1",
String.format(Locale.ENGLISH, "%04d", child1)).setParent(parentId).setSource("{}"));
}
numChildDocs = child2InnerObjects[parent] = scaledRandomIntBetween(1, numDocs);
limit = child2 + numChildDocs;
for (; child2 < limit; child2++) {
requestBuilders.add(client().prepareIndex("idx", "child2", String.format(Locale.ENGLISH, "%04d", child2)).setParent(parentId).setSource("{}"));
requestBuilders.add(client().prepareIndex("idx", "child2",
String.format(Locale.ENGLISH, "%04d", child2)).setParent(parentId).setSource("{}"));
}
}
indexRandom(true, requestBuilders);
@ -399,10 +401,14 @@ public class InnerHitsIT extends ESIntegTestCase {
.addMapping("answer", "_parent", "type=question", "body", "type=text")
);
List<IndexRequestBuilder> requests = new ArrayList<>();
requests.add(client().prepareIndex("stack", "question", "1").setSource("body", "I'm using HTTPS + Basic authentication to protect a resource. How can I throttle authentication attempts to protect against brute force attacks?"));
requests.add(client().prepareIndex("stack", "answer", "1").setParent("1").setSource("body", "install fail2ban and enable rules for apache"));
requests.add(client().prepareIndex("stack", "question", "2").setSource("body", "I have firewall rules set up and also denyhosts installed.\\ndo I also need to install fail2ban?"));
requests.add(client().prepareIndex("stack", "answer", "2").setParent("2").setSource("body", "Denyhosts protects only ssh; Fail2Ban protects all daemons."));
requests.add(client().prepareIndex("stack", "question", "1").setSource("body", "I'm using HTTPS + Basic authentication "
+ "to protect a resource. How can I throttle authentication attempts to protect against brute force attacks?"));
requests.add(client().prepareIndex("stack", "answer", "1").setParent("1").setSource("body",
"install fail2ban and enable rules for apache"));
requests.add(client().prepareIndex("stack", "question", "2").setSource("body",
"I have firewall rules set up and also denyhosts installed.\\ndo I also need to install fail2ban?"));
requests.add(client().prepareIndex("stack", "answer", "2").setParent("2").setSource("body",
"Denyhosts protects only ssh; Fail2Ban protects all daemons."));
indexRandom(true, requests);
SearchResponse response = client().prepareSearch("stack")
@ -491,24 +497,25 @@ public class InnerHitsIT extends ESIntegTestCase {
}
public void testNestedMultipleLayers() throws Exception {
assertAcked(prepareCreate("articles").addMapping("article", jsonBuilder().startObject().startObject("article").startObject("properties")
.startObject("comments")
.field("type", "nested")
.startObject("properties")
.startObject("message")
.field("type", "text")
.endObject()
.startObject("remarks")
.field("type", "nested")
.startObject("properties")
.startObject("message").field("type", "text").endObject()
assertAcked(prepareCreate("articles").addMapping("article", jsonBuilder().startObject()
.startObject("article").startObject("properties")
.startObject("comments")
.field("type", "nested")
.startObject("properties")
.startObject("message")
.field("type", "text")
.endObject()
.startObject("remarks")
.field("type", "nested")
.startObject("properties")
.startObject("message").field("type", "text").endObject()
.endObject()
.endObject()
.endObject()
.endObject()
.endObject()
.startObject("title")
.field("type", "text")
.endObject()
.startObject("title")
.field("type", "text")
.endObject()
.endObject().endObject().endObject()));
List<IndexRequestBuilder> requests = new ArrayList<>();
@ -560,8 +567,8 @@ public class InnerHitsIT extends ESIntegTestCase {
// Directly refer to the second level:
response = client().prepareSearch("articles")
.setQuery(nestedQuery("comments.remarks", matchQuery("comments.remarks.message", "bad"), ScoreMode.Avg).innerHit(new InnerHitBuilder()))
.get();
.setQuery(nestedQuery("comments.remarks", matchQuery("comments.remarks.message", "bad"), ScoreMode.Avg)
.innerHit(new InnerHitBuilder())).get();
assertNoFailures(response);
assertHitCount(response, 1);
assertSearchHit(response, 1, hasId("2"));
@ -621,7 +628,8 @@ public class InnerHitsIT extends ESIntegTestCase {
assertThat(response.getHits().getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getTotalHits(), equalTo(1L));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).getNestedIdentity().getField().string(), equalTo("comments"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).getNestedIdentity().getField().string(),
equalTo("comments"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).getNestedIdentity().getOffset(), equalTo(0));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments").getAt(0).getNestedIdentity().getChild(), nullValue());
}
@ -654,28 +662,32 @@ public class InnerHitsIT extends ESIntegTestCase {
indexRandom(true, requests);
SearchResponse response = client().prepareSearch("articles")
.setQuery(nestedQuery("comments.messages", matchQuery("comments.messages.message", "fox"), ScoreMode.Avg).innerHit(new InnerHitBuilder()))
.get();
.setQuery(nestedQuery("comments.messages", matchQuery("comments.messages.message", "fox"), ScoreMode.Avg)
.innerHit(new InnerHitBuilder())).get();
assertNoFailures(response);
assertHitCount(response, 1);
assertThat(response.getHits().getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getTotalHits(), equalTo(1L));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getOffset(), equalTo(0));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getChild(), nullValue());
SearchHit hit = response.getHits().getAt(0);
assertThat(hit.id(), equalTo("1"));
SearchHits messages = hit.getInnerHits().get("comments.messages");
assertThat(messages.getTotalHits(), equalTo(1L));
assertThat(messages.getAt(0).id(), equalTo("1"));
assertThat(messages.getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages"));
assertThat(messages.getAt(0).getNestedIdentity().getOffset(), equalTo(0));
assertThat(messages.getAt(0).getNestedIdentity().getChild(), nullValue());
response = client().prepareSearch("articles")
.setQuery(nestedQuery("comments.messages", matchQuery("comments.messages.message", "bear"), ScoreMode.Avg).innerHit(new InnerHitBuilder()))
.get();
.setQuery(nestedQuery("comments.messages", matchQuery("comments.messages.message", "bear"), ScoreMode.Avg)
.innerHit(new InnerHitBuilder())).get();
assertNoFailures(response);
assertHitCount(response, 1);
assertThat(response.getHits().getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getTotalHits(), equalTo(1L));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getOffset(), equalTo(1));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getChild(), nullValue());
hit = response.getHits().getAt(0);
assertThat(hit.id(), equalTo("1"));
messages = hit.getInnerHits().get("comments.messages");
assertThat(messages.getTotalHits(), equalTo(1L));
assertThat(messages.getAt(0).id(), equalTo("1"));
assertThat(messages.getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages"));
assertThat(messages.getAt(0).getNestedIdentity().getOffset(), equalTo(1));
assertThat(messages.getAt(0).getNestedIdentity().getChild(), nullValue());
// index the message in an object form instead of an array
requests = new ArrayList<>();
@ -685,16 +697,18 @@ public class InnerHitsIT extends ESIntegTestCase {
.endObject()));
indexRandom(true, requests);
response = client().prepareSearch("articles")
.setQuery(nestedQuery("comments.messages", matchQuery("comments.messages.message", "fox"), ScoreMode.Avg).innerHit(new InnerHitBuilder()))
.get();
.setQuery(nestedQuery("comments.messages", matchQuery("comments.messages.message", "fox"), ScoreMode.Avg)
.innerHit(new InnerHitBuilder())).get();
assertNoFailures(response);
assertHitCount(response, 1);
assertThat(response.getHits().getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getTotalHits(), equalTo(1L));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).id(), equalTo("1"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages"));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getOffset(), equalTo(0));
assertThat(response.getHits().getAt(0).getInnerHits().get("comments.messages").getAt(0).getNestedIdentity().getChild(), nullValue());
hit = response.getHits().getAt(0);;
assertThat(hit.id(), equalTo("1"));
messages = hit.getInnerHits().get("comments.messages");
assertThat(messages.getTotalHits(), equalTo(1L));
assertThat(messages.getAt(0).id(), equalTo("1"));
assertThat(messages.getAt(0).getNestedIdentity().getField().string(), equalTo("comments.messages"));
assertThat(messages.getAt(0).getNestedIdentity().getOffset(), equalTo(0));
assertThat(messages.getAt(0).getNestedIdentity().getChild(), nullValue());
}
public void testRoyals() throws Exception {
@ -841,12 +855,14 @@ public class InnerHitsIT extends ESIntegTestCase {
indexRandom(true, requests);
waitForRelocation(ClusterHealthStatus.GREEN);
QueryBuilder query = boolQuery()
.should(termQuery("nested1.n_field1", "n_value1_1").queryName("test1"))
.should(termQuery("nested1.n_field1", "n_value1_3").queryName("test2"))
.should(termQuery("nested1.n_field2", "n_value2_2").queryName("test3"));
query = nestedQuery("nested1", query, ScoreMode.Avg).innerHit(
new InnerHitBuilder().addSort(new FieldSortBuilder("nested1.n_field1").order(SortOrder.ASC)));
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(nestedQuery("nested1", boolQuery()
.should(termQuery("nested1.n_field1", "n_value1_1").queryName("test1"))
.should(termQuery("nested1.n_field1", "n_value1_3").queryName("test2"))
.should(termQuery("nested1.n_field2", "n_value2_2").queryName("test3")),
ScoreMode.Avg).innerHit(new InnerHitBuilder().addSort(new FieldSortBuilder("nested1.n_field1").order(SortOrder.ASC))))
.setQuery(query)
.setSize(numDocs)
.addSort("field1", SortOrder.ASC)
.get();
@ -885,7 +901,8 @@ public class InnerHitsIT extends ESIntegTestCase {
indexRandom(true, requests);
SearchResponse response = client().prepareSearch("index")
.setQuery(hasChildQuery("child", matchQuery("field", "value1").queryName("_name1"), ScoreMode.None).innerHit(new InnerHitBuilder()))
.setQuery(hasChildQuery("child", matchQuery("field", "value1").queryName("_name1"), ScoreMode.None)
.innerHit(new InnerHitBuilder()))
.addSort("_uid", SortOrder.ASC)
.get();
assertHitCount(response, 2);
@ -899,8 +916,10 @@ public class InnerHitsIT extends ESIntegTestCase {
assertThat(response.getHits().getAt(1).getInnerHits().get("child").getAt(0).getMatchedQueries().length, equalTo(1));
assertThat(response.getHits().getAt(1).getInnerHits().get("child").getAt(0).getMatchedQueries()[0], equalTo("_name1"));
QueryBuilder query = hasChildQuery("child", matchQuery("field", "value2").queryName("_name2"), ScoreMode.None)
.innerHit(new InnerHitBuilder());
response = client().prepareSearch("index")
.setQuery(hasChildQuery("child", matchQuery("field", "value2").queryName("_name2"), ScoreMode.None).innerHit(new InnerHitBuilder()))
.setQuery(query)
.addSort("_uid", SortOrder.ASC)
.get();
assertHitCount(response, 1);
@ -917,8 +936,10 @@ public class InnerHitsIT extends ESIntegTestCase {
requests.add(client().prepareIndex("index1", "child", "1").setParent("1").setSource("field", "value1"));
indexRandom(true, requests);
QueryBuilder query = hasChildQuery("child", matchQuery("field", "value1"), ScoreMode.None)
.innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1));
SearchResponse response = client().prepareSearch("index1")
.setQuery(hasChildQuery("child", matchQuery("field", "value1"), ScoreMode.None).innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1)))
.setQuery(query)
.addSort("_uid", SortOrder.ASC)
.get();
assertNoFailures(response);
@ -935,8 +956,10 @@ public class InnerHitsIT extends ESIntegTestCase {
.setRefreshPolicy(IMMEDIATE)
.get();
query = nestedQuery("nested", matchQuery("nested.field", "value1"), ScoreMode.Avg)
.innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1));
response = client().prepareSearch("index2")
.setQuery(nestedQuery("nested", matchQuery("nested.field", "value1"), ScoreMode.Avg).innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1)))
.setQuery(query)
.addSort("_uid", SortOrder.ASC)
.get();
assertNoFailures(response);

Some files were not shown because too many files have changed in this diff Show More